From 9c24c6756c011ccd9670006d1a0f8c8b2c6114c5 Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Fri, 31 May 2024 10:58:23 +0800 Subject: [PATCH 1/5] done Signed-off-by: wjhuang2016 --- MODULE.bazel.lock | 2 +- ddl/metadatalocktest/BUILD.bazel | 2 +- ddl/metadatalocktest/mdl_test.go | 97 ++ domain/plan_replayer_dump.go | 1 + executor/executor.go | 1 + executor/prepared.go | 5 +- executor/seqtest/seq_executor_test.go | 3 +- meta/meta.go | 2 + parser/model/model.go | 3 + .../core/tests/prepare/prepare_test.go | 1434 +++++++++++++++++ planner/core/plan_cache.go | 36 +- planner/core/plan_cache_utils.go | 65 +- planner/core/preprocess.go | 11 +- planner/optimize.go | 2 +- server/driver_tidb.go | 2 +- session/session.go | 2 +- sessionctx/stmtctx/stmtctx.go | 3 + 17 files changed, 1653 insertions(+), 18 deletions(-) create mode 100644 pkg/planner/core/tests/prepare/prepare_test.go diff --git a/MODULE.bazel.lock b/MODULE.bazel.lock index 11bf37a4bfc0e..1558a0cd726b6 100644 --- a/MODULE.bazel.lock +++ b/MODULE.bazel.lock @@ -1125,7 +1125,7 @@ }, "@@rules_java~//java:extensions.bzl%toolchains": { "general": { - "bzlTransitiveDigest": "tJHbmWnq7m+9eUBnUdv7jZziQ26FmcGL9C5/hU3Q9UQ=", + "bzlTransitiveDigest": "0N5b5J9fUzo0sgvH4F3kIEaeXunz4Wy2/UtSFV/eXUY=", "recordedFileInputs": {}, "recordedDirentsInputs": {}, "envVariables": {}, diff --git a/ddl/metadatalocktest/BUILD.bazel b/ddl/metadatalocktest/BUILD.bazel index 4f0e67a4e015d..5d95a8d23c397 100644 --- a/ddl/metadatalocktest/BUILD.bazel +++ b/ddl/metadatalocktest/BUILD.bazel @@ -7,7 +7,7 @@ go_test( "mdl_test.go", ], flaky = True, - shard_count = 34, + shard_count = 36, deps = [ "//config", "//ddl", diff --git a/ddl/metadatalocktest/mdl_test.go b/ddl/metadatalocktest/mdl_test.go index b27ac5e6a9064..0ed550da6c34e 100644 --- a/ddl/metadatalocktest/mdl_test.go +++ b/ddl/metadatalocktest/mdl_test.go @@ -872,6 +872,103 @@ func TestMDLPreparePlanCacheInvalid(t *testing.T) { tk.MustQuery(`execute stmt_test_1 using @a;`).Check(testkit.Rows("1 ", "2 ", "3 ", "4 ")) } +func TestMDLPreparePlanCacheExecute(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + defer ingesttestutil.InjectMockBackendMgr(t, store)() + + sv := server.CreateMockServer(t, store) + + sv.SetDomain(dom) + dom.InfoSyncer().SetSessionManager(sv) + defer sv.Close() + + conn1 := server.CreateMockConn(t, sv) + tk := testkit.NewTestKitWithSession(t, store, conn1.Context().Session) + conn2 := server.CreateMockConn(t, sv) + tkDDL := testkit.NewTestKitWithSession(t, store, conn2.Context().Session) + tk.MustExec("use test") + tk.MustExec("set global tidb_enable_metadata_lock=1") + tk.MustExec("create table t(a int);") + tk.MustExec("create table t2(a int);") + tk.MustExec("insert into t values(1), (2), (3), (4);") + + tk.MustExec(`prepare stmt_test_1 from 'update t set a = ? where a = ?';`) + tk.MustExec(`set @a = 1, @b = 3;`) + tk.MustExec(`execute stmt_test_1 using @a, @b;`) + + tk.MustExec("begin") + + ch := make(chan struct{}) + + var wg sync.WaitGroup + wg.Add(1) + go func() { + <-ch + tkDDL.MustExec("alter table test.t add index idx(a);") + wg.Done() + }() + + tk.MustQuery("select * from t2") + tk.MustExec(`set @a = 2, @b=4;`) + tk.MustExec(`execute stmt_test_1 using @a, @b;`) + 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{}{} + + time.Sleep(5 * time.Second) + + tk.MustExec("commit") + + wg.Wait() + + tk.MustExec("admin check table t") +} + +func TestMDLPreparePlanCacheExecute2(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + defer ingesttestutil.InjectMockBackendMgr(t, store)() + + sv := server.CreateMockServer(t, store) + + sv.SetDomain(dom) + dom.InfoSyncer().SetSessionManager(sv) + defer sv.Close() + + conn1 := server.CreateMockConn(t, sv) + tk := testkit.NewTestKitWithSession(t, store, conn1.Context().Session) + conn2 := server.CreateMockConn(t, sv) + tkDDL := testkit.NewTestKitWithSession(t, store, conn2.Context().Session) + tk.MustExec("use test") + tk.MustExec("set global tidb_enable_metadata_lock=1") + tk.MustExec("create table t(a int);") + tk.MustExec("create table t2(a int);") + tk.MustExec("insert into t values(1), (2), (3), (4);") + + tk.MustExec(`prepare stmt_test_1 from 'select * from t where a = ?';`) + tk.MustExec(`set @a = 1;`) + tk.MustExec(`execute stmt_test_1 using @a;`) + + tk.MustExec("begin") + tk.MustQuery("select * from t2") + + var wg sync.WaitGroup + wg.Add(1) + go func() { + tkDDL.MustExec("alter table test.t add index idx(a);") + wg.Done() + }() + + wg.Wait() + + tk.MustExec(`set @a = 2;`) + tk.MustExec(`execute stmt_test_1 using @a;`) + // The plan should not be from cache because the schema has changed. + tk.MustQuery("select @@last_plan_from_cache;").Check(testkit.Rows("0")) + tk.MustExec("commit") + + tk.MustExec("admin check table t") +} + func TestMDLDisable2Enable(t *testing.T) { store, dom := testkit.CreateMockStoreAndDomain(t) sv := server.CreateMockServer(t, store) diff --git a/domain/plan_replayer_dump.go b/domain/plan_replayer_dump.go index 575d0bc35264e..1281f1cb87c69 100644 --- a/domain/plan_replayer_dump.go +++ b/domain/plan_replayer_dump.go @@ -526,6 +526,7 @@ func dumpExplain(ctx sessionctx.Context, zw *zip.Writer, execStmts []ast.StmtNod return nil } +// extractTableNames extracts table names from the given stmts. func extractTableNames(ctx context.Context, sctx sessionctx.Context, ExecStmts []ast.StmtNode, curDB model.CIStr) (map[tableNamePair]struct{}, error) { tableExtractor := &tableNameExtractor{ diff --git a/executor/executor.go b/executor/executor.go index a6318563c2422..11c1a57bd6e25 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -1985,6 +1985,7 @@ func ResetContextOfStmt(ctx sessionctx.Context, s ast.StmtNode) (err error) { vars.DiskTracker.ResetMaxConsumed() vars.MemTracker.SessionID.Store(vars.ConnectionID) vars.StmtCtx.TableStats = make(map[int64]interface{}) + sc.MDLRelatedTableIDs = make(map[int64]int64) isAnalyze := false if execStmt, ok := s.(*ast.ExecuteStmt); ok { diff --git a/executor/prepared.go b/executor/prepared.go index ae742a30d8bb5..bb2c20262157b 100644 --- a/executor/prepared.go +++ b/executor/prepared.go @@ -26,6 +26,7 @@ import ( "github.com/pingcap/tidb/parser/mysql" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/sessiontxn" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/chunk" @@ -117,7 +118,7 @@ func (e *PrepareExec) Next(ctx context.Context, req *chunk.Chunk) error { return err } } - stmt, p, paramCnt, err := plannercore.GeneratePlanCacheStmtWithAST(ctx, e.ctx, stmt0) + stmt, p, paramCnt, err := plannercore.GeneratePlanCacheStmtWithAST(ctx, e.ctx, stmt0, sessiontxn.GetTxnManager(e.ctx).GetTxnInfoSchema()) if err != nil { return err } @@ -211,7 +212,7 @@ func (e *DeallocateExec) Next(ctx context.Context, req *chunk.Chunk) error { if e.ctx.GetSessionVars().EnablePreparedPlanCache { bindSQL, _ := plannercore.GetBindSQL4PlanCache(e.ctx, preparedObj) cacheKey, err := plannercore.NewPlanCacheKey(vars, preparedObj.StmtText, preparedObj.StmtDB, prepared.SchemaVersion, - 0, bindSQL) + 0, bindSQL, preparedObj.RelateVersion) if err != nil { return err } diff --git a/executor/seqtest/seq_executor_test.go b/executor/seqtest/seq_executor_test.go index bf78e40e7ac3d..744c58eae307c 100644 --- a/executor/seqtest/seq_executor_test.go +++ b/executor/seqtest/seq_executor_test.go @@ -940,8 +940,7 @@ func TestBatchInsertDelete(t *testing.T) { atomic.StoreUint64(&kv.TxnTotalSizeLimit, originLimit) }() // Set the limitation to a small value, make it easier to reach the limitation. - atomic.StoreUint64(&kv.TxnTotalSizeLimit, 5800) - + atomic.StoreUint64(&kv.TxnTotalSizeLimit, 7000) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists batch_insert") diff --git a/meta/meta.go b/meta/meta.go index 6c79ac23b6659..fa4a8bc975f5f 100644 --- a/meta/meta.go +++ b/meta/meta.go @@ -822,6 +822,8 @@ func (m *Meta) UpdateTable(dbID int64, tableInfo *model.TableInfo) error { return errors.Trace(err) } + tableInfo.Revision++ + data, err := json.Marshal(tableInfo) if err != nil { return errors.Trace(err) diff --git a/parser/model/model.go b/parser/model/model.go index 89db1169ebc9e..81c65b96894a2 100644 --- a/parser/model/model.go +++ b/parser/model/model.go @@ -567,6 +567,9 @@ type TableInfo struct { ExchangePartitionInfo *ExchangePartitionInfo `json:"exchange_partition_info"` TTLInfo *TTLInfo `json:"ttl_info"` + + // Revision is per table schema's version, it will be increased when the schema changed. + Revision uint64 `json:"revision"` } // SepAutoInc decides whether _rowid and auto_increment id use separate allocator. 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..6d11919c502d6 --- /dev/null +++ b/pkg/planner/core/tests/prepare/prepare_test.go @@ -0,0 +1,1434 @@ +// 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" + "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/testkit" + "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) + } + + rand.Seed(time.Now().Unix()) + 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(), is, &hint.BlockHintProcessor{}) + 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.GetPlanFromSessionPlanCache(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`) + + tk.MustExec("use test") + for _, pruneMode := range []string{string(variable.Static), string(variable.Dynamic)} { + 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("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("execute stmt5 using @id").Check(testkit.Rows("abc")) + tk.MustExec("set @id=5") + tk.MustQuery("execute stmt5 using @id").Check(testkit.Rows("def")) + tk.MustQuery("execute stmt5 using @id").Check(testkit.Rows("def")) + tk.MustExec("set @id=13") + tk.MustQuery("execute stmt5 using @id").Check(testkit.Rows("xyz")) + tk.MustExec("set @id=17") + tk.MustQuery("execute stmt5 using @id").Check(testkit.Rows("hij")) + + 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("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("execute stmt6 using @id").Check(testkit.Rows("abc")) + tk.MustExec("set @id=5") + tk.MustQuery("execute stmt6 using @id").Check(testkit.Rows("def")) + tk.MustQuery("execute stmt6 using @id").Check(testkit.Rows("def")) + tk.MustExec("set @id=13") + tk.MustQuery("execute stmt6 using @id").Check(testkit.Rows("xyz")) + tk.MustExec("set @id=17") + tk.MustQuery("execute stmt6 using @id").Check(testkit.Rows("hij")) + + // 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("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("execute stmt7 using @id").Check(testkit.Rows("abc")) + tk.MustExec("set @id=5") + tk.MustQuery("execute stmt7 using @id").Check(testkit.Rows("def")) + tk.MustQuery("execute stmt7 using @id").Check(testkit.Rows("def")) + tk.MustExec("set @id=9") + tk.MustQuery("execute stmt7 using @id").Check(testkit.Rows("xyz")) + tk.MustExec("set @id=12") + tk.MustQuery("execute stmt7 using @id").Check(testkit.Rows("hij")) + tk.MustExec("set @id=100") + tk.MustQuery("execute stmt7 using @id").Check(testkit.Rows()) + + // 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("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("execute stmt8 using @id").Check(testkit.Rows("abc")) + tk.MustExec("set @id=5") + tk.MustQuery("execute stmt8 using @id").Check(testkit.Rows("def")) + tk.MustQuery("execute stmt8 using @id").Check(testkit.Rows("def")) + tk.MustExec("set @id=9") + tk.MustQuery("execute stmt8 using @id").Check(testkit.Rows("xyz")) + tk.MustExec("set @id=12") + tk.MustQuery("execute stmt8 using @id").Check(testkit.Rows("hij")) + tk.MustExec("set @id=100") + tk.MustQuery("execute stmt8 using @id").Check(testkit.Rows()) + + // 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(`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()) + 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")) + if pruneMode == string(variable.Dynamic) { + // When the temporary disabling of prepared plan cache for dynamic partition prune mode is disabled, change this to 1! + tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("0")) + } + } +} + +func newSession(t *testing.T, store kv.Storage, dbName string) session.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 session.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()) + err := counter.Write(pb) + require.NoError(t, err) + cnt := pb.GetCounter().GetValue() + require.Equal(t, float64(1), 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()) + 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") + + 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+rand.Intn(100), 1+rand.Intn(12), 1+rand.Intn(28), // date + rand.Intn(24), rand.Intn(60), rand.Intn(60)) // time + } + randDate := func() string { + return fmt.Sprintf("%v-%v-%v", 1950+rand.Intn(100), 1+rand.Intn(12), 1+rand.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)", rand.Intn(100000000), rand.Intn(100000000)) }, + func() string { return fmt.Sprintf("%v", rand.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)", rand.Intn(100000000), rand.Intn(100000000)) }, + func() string { return fmt.Sprintf("%v", rand.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")`, rand.Intn(1000), rand.Intn(1000)) }, + func() string { return fmt.Sprintf(`"%v"`, rand.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 + `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)", rand.Intn(20), rand.Intn(20)) }, + func() string { return fmt.Sprintf("%v", rand.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, ","))) + + // 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")) + + for i := 0; i < 100; i++ { + tk.MustExec(fmt.Sprintf("set @a=%v", tc.varGener())) + result1 := tk.MustQuery("execute stmt1 using @a").Sort().Rows() + // When https://github.com/pingcap/tidb/pull/33098 is reverted this should be 1 again + tk.MustQuery("select @@last_plan_from_cache /* i=" + strconv.Itoa(i) + " prepared statement: (t1) " + tc.query + "\n-- create table: " + tc.t1Create + "*/").Check(testkit.Rows("0")) + tk.MustQuery("execute stmt2 using @a").Sort().Check(result1) + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1")) + } + } +} + +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) + } + + // 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)) + } + for i := 0; i < 100; 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))) + tk.MustExec(fmt.Sprintf(`set @a0=%v, @a1=%v, @a2=%v`, rand.Intn(40000), rand.Intn(40000), rand.Intn(40000))) + + var rscan, rpoint, rbatch [][]interface{} + 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) + } + + point := tk.MustQuery(fmt.Sprintf(`execute stmt%v_pointget using @pointa`, tbl)).Sort() + if id == 0 { + rpoint = point.Rows() + } else { + point.Check(rpoint) + } + + 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) + } + } + } + + // 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)) + } + loops := 100 + missedPlanCache := make([]string, 0, 4) + 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))) + tk.MustExec(fmt.Sprintf(`set @a0=%v, @a1=%v, @a2=%v`, rand.Intn(40000), rand.Intn(40000), rand.Intn(40000))) + + var rscan, rlookup, rpoint, rbatch [][]interface{} + var expectedFromPlanCache string + for id, tbl := range []string{"trangeIdx", "thashIdx", "tnormalIdx"} { + scan := tk.MustQuery(fmt.Sprintf(`execute stmt%v_indexscan using @mina, @maxa`, tbl)).Sort() + if id == 2 { + expectedFromPlanCache = "1" + } else { + expectedFromPlanCache = "0" + } + tblStr := ` table: ` + tbl + " i :" + strconv.FormatInt(int64(i), 10) + " */" + if i > 0 { + missedPlanCache = helperCheckPlanCache(t, tk, `select @@last_plan_from_cache /* indexscan table: `+tblStr, expectedFromPlanCache, 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, expectedFromPlanCache, 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, expectedFromPlanCache, 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, expectedFromPlanCache, missedPlanCache) + } + if id == 0 { + rbatch = batch.Rows() + } else { + batch.Check(rbatch) + } + } + } + // Allow ~1% non-cached queries, due to background changes etc. + // (Actually just 1/3 %, since there are 3 tables * 4 queries per loop :) + if len(missedPlanCache) > (loops * 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")) +} diff --git a/planner/core/plan_cache.go b/planner/core/plan_cache.go index b84ed21320509..e7bf4d09078c2 100644 --- a/planner/core/plan_cache.go +++ b/planner/core/plan_cache.go @@ -76,8 +76,33 @@ func planCachePreprocess(ctx context.Context, sctx sessionctx.Context, isGeneral vars.PreparedParams = append(vars.PreparedParams, val) } - // step 3: check schema version - if stmtAst.SchemaVersion != is.SchemaMetaVersion() { + // step 3: add metadata lock and check each table's schema version + schemaNotMatch := false + for i := 0; i < len(stmt.dbName); i++ { + _, ok := is.TableByID(stmt.tbls[i].Meta().ID) + if !ok { + tblByName, err := is.TableByName(stmt.dbName[i], stmt.tbls[i].Meta().Name) + if err != nil { + return 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(sctx, stmt.dbName[i], stmt.tbls[i], is) + if err != nil { + schemaNotMatch = true + continue + } + if stmt.tbls[i].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.PreparedAst.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 @@ -99,7 +124,7 @@ func planCachePreprocess(ctx context.Context, sctx sessionctx.Context, isGeneral stmtAst.SchemaVersion = is.SchemaMetaVersion() } - // step 4: handle expiration + // 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. @@ -110,6 +135,7 @@ func planCachePreprocess(ctx context.Context, sctx sessionctx.Context, isGeneral stmtAst.CachedPlan = nil vars.LastUpdateTime4PC = expiredTimeStamp4PC } + return nil } @@ -151,7 +177,7 @@ func GetPlanFromSessionPlanCache(ctx context.Context, sctx sessionctx.Context, latestSchemaVersion = domain.GetDomain(sctx).InfoSchema().SchemaMetaVersion() } if cacheKey, err = NewPlanCacheKey(sctx.GetSessionVars(), stmt.StmtText, - stmt.StmtDB, stmtAst.SchemaVersion, latestSchemaVersion, bindSQL); err != nil { + stmt.StmtDB, stmtAst.SchemaVersion, latestSchemaVersion, bindSQL, stmt.RelateVersion); err != nil { return nil, nil, err } } @@ -293,7 +319,7 @@ func generateNewPlan(ctx context.Context, sctx sessionctx.Context, isGeneralPlan if _, isolationReadContainTiFlash := sessVars.IsolationReadEngines[kv.TiFlash]; isolationReadContainTiFlash && !IsReadOnly(stmtAst.Stmt, sessVars) { delete(sessVars.IsolationReadEngines, kv.TiFlash) if cacheKey, err = NewPlanCacheKey(sessVars, stmt.StmtText, stmt.StmtDB, - stmtAst.SchemaVersion, latestSchemaVersion, bindSQL); err != nil { + stmtAst.SchemaVersion, latestSchemaVersion, bindSQL, stmt.RelateVersion); err != nil { return nil, nil, err } sessVars.IsolationReadEngines[kv.TiFlash] = struct{}{} diff --git a/planner/core/plan_cache_utils.go b/planner/core/plan_cache_utils.go index f41f5a1c84659..6305f34e88f68 100644 --- a/planner/core/plan_cache_utils.go +++ b/planner/core/plan_cache_utils.go @@ -17,12 +17,14 @@ package core import ( "context" "math" + "sort" "strconv" "time" "unsafe" "github.com/pingcap/errors" "github.com/pingcap/tidb/expression" + "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/parser/ast" @@ -32,14 +34,17 @@ import ( "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/types" driver "github.com/pingcap/tidb/types/parser_driver" "github.com/pingcap/tidb/util/codec" "github.com/pingcap/tidb/util/hack" "github.com/pingcap/tidb/util/hint" "github.com/pingcap/tidb/util/kvcache" + "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/size" atomic2 "go.uber.org/atomic" + "go.uber.org/zap" "golang.org/x/exp/slices" ) @@ -67,7 +72,7 @@ func (e *paramMarkerExtractor) Leave(in ast.Node) (ast.Node, bool) { } // GeneratePlanCacheStmtWithAST generates the PlanCacheStmt structure for this AST. -func GeneratePlanCacheStmtWithAST(ctx context.Context, sctx sessionctx.Context, stmt ast.StmtNode) (*PlanCacheStmt, Plan, int, error) { +func GeneratePlanCacheStmtWithAST(ctx context.Context, sctx sessionctx.Context, stmt ast.StmtNode, is infoschema.InfoSchema) (*PlanCacheStmt, Plan, int, error) { vars := sctx.GetSessionVars() var extractor paramMarkerExtractor stmt.Accept(&extractor) @@ -150,6 +155,26 @@ func GeneratePlanCacheStmtWithAST(ctx context.Context, sctx sessionctx.Context, return nil, nil, 0, err } + // 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, dbID := 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(dbID) + if !ok { + logutil.BgLogger().Error("database not found in info schema", zap.Int64("dbID", 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, @@ -161,6 +186,9 @@ func GeneratePlanCacheStmtWithAST(ctx context.Context, sctx sessionctx.Context, SnapshotTSEvaluator: ret.SnapshotTSEvaluator, NormalizedSQL4PC: normalizedSQL4PC, SQLDigest4PC: digest4PC, + dbName: dbName, + tbls: tbls, + RelateVersion: relateVersion, } if err = CheckPreparedPriv(sctx, preparedObj, ret.InfoSchema); err != nil { return nil, nil, 0, err @@ -192,6 +220,7 @@ type planCacheKey struct { connID uint64 stmtText string schemaVersion int64 + tblVersionMap map[int64]uint64 // 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. @@ -211,6 +240,23 @@ type planCacheKey struct { hash []byte } +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 +} + // Hash implements Key interface. func (key *planCacheKey) Hash() []byte { if len(key.hash) == 0 { @@ -225,6 +271,7 @@ func (key *planCacheKey) Hash() []byte { key.hash = codec.EncodeInt(key.hash, int64(key.connID)) key.hash = append(key.hash, hack.Slice(key.stmtText)...) key.hash = codec.EncodeInt(key.hash, key.schemaVersion) + key.hash = hashInt64Uint64Map(key.hash, key.tblVersionMap) key.hash = codec.EncodeInt(key.hash, key.lastUpdatedSchemaVersion) key.hash = codec.EncodeInt(key.hash, int64(key.sqlMode)) key.hash = codec.EncodeInt(key.hash, int64(key.timezoneOffset)) @@ -283,7 +330,7 @@ func SetPstmtIDSchemaVersion(key kvcache.Key, stmtText string, schemaVersion int // 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. func NewPlanCacheKey(sessionVars *variable.SessionVars, stmtText, stmtDB string, schemaVersion int64, - lastUpdatedSchemaVersion int64, bindSQL string) (kvcache.Key, error) { + lastUpdatedSchemaVersion int64, bindSQL string, relatedSchemaVersion map[int64]uint64) (kvcache.Key, error) { if stmtText == "" { return nil, errors.New("no statement text") } @@ -302,6 +349,7 @@ func NewPlanCacheKey(sessionVars *variable.SessionVars, stmtText, stmtDB string, connID: sessionVars.ConnectionID, stmtText: stmtText, schemaVersion: schemaVersion, + tblVersionMap: make(map[int64]uint64), lastUpdatedSchemaVersion: lastUpdatedSchemaVersion, sqlMode: sessionVars.SQLMode, timezoneOffset: timezoneOffset, @@ -315,6 +363,9 @@ func NewPlanCacheKey(sessionVars *variable.SessionVars, stmtText, stmtDB string, for k, v := range sessionVars.IsolationReadEngines { key.isolationReadEngines[k] = v } + for k, v := range relatedSchemaVersion { + key.tblVersionMap[k] = v + } return key, nil } @@ -431,7 +482,11 @@ type PlanCacheStmt struct { // 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 interface{} + Executor interface{} + + // RelateVersion stores the true cache plan table schema version, since each table schema can be updated separately in transaction. + RelateVersion map[int64]uint64 + NormalizedSQL string NormalizedPlan string SQLDigest *parser.Digest @@ -447,6 +502,10 @@ type PlanCacheStmt struct { // 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. diff --git a/planner/core/preprocess.go b/planner/core/preprocess.go index eb21166d3194c..f0566c805acfa 100644 --- a/planner/core/preprocess.go +++ b/planner/core/preprocess.go @@ -1826,6 +1826,16 @@ func tryLockMDLAndUpdateSchemaIfNecessary(sctx sessionctx.Context, dbName model. return tbl, nil } tableInfo := tbl.Meta() + var err error + defer func() { + if err == nil && !skipLock { + dbID, ok := is.SchemaByName(dbName) + if !ok { + return + } + sctx.GetSessionVars().StmtCtx.MDLRelatedTableIDs[tbl.Meta().ID] = dbID.ID + } + }() if _, ok := sctx.GetSessionVars().GetRelatedTableForMDL().Load(tableInfo.ID); !ok { if se, ok := is.(*infoschema.SessionExtendedInfoSchema); ok && skipLock && se.MdlTables != nil { if _, ok := se.MdlTables.TableByID(tableInfo.ID); ok { @@ -1844,7 +1854,6 @@ func tryLockMDLAndUpdateSchemaIfNecessary(sctx sessionctx.Context, dbName model. dom := domain.GetDomain(sctx) domainSchema := dom.InfoSchema() domainSchemaVer := domainSchema.SchemaMetaVersion() - var err error tbl, err = domainSchema.TableByName(dbName, tableInfo.Name) if err != nil { return nil, err diff --git a/planner/optimize.go b/planner/optimize.go index b0cc8a20a2114..fbef265e32aa6 100644 --- a/planner/optimize.go +++ b/planner/optimize.go @@ -85,7 +85,7 @@ func getPlanFromGeneralPlanCache(ctx context.Context, sctx sessionctx.Context, s } val := sctx.GetSessionVars().GetGeneralPlanCacheStmt(paramSQL) if val == nil { - cachedStmt, _, _, err := core.GeneratePlanCacheStmtWithAST(ctx, sctx, stmt) + cachedStmt, _, _, err := core.GeneratePlanCacheStmtWithAST(ctx, sctx, stmt, is) if err != nil { return nil, nil, false, err } diff --git a/server/driver_tidb.go b/server/driver_tidb.go index 7f4a8e3460531..ea5e47965d68e 100644 --- a/server/driver_tidb.go +++ b/server/driver_tidb.go @@ -204,7 +204,7 @@ func (ts *TiDBStatement) Close() error { } bindSQL, _ := core.GetBindSQL4PlanCache(ts.ctx, preparedObj) cacheKey, err := core.NewPlanCacheKey(ts.ctx.GetSessionVars(), preparedObj.StmtText, preparedObj.StmtDB, - preparedObj.PreparedAst.SchemaVersion, 0, bindSQL) + preparedObj.PreparedAst.SchemaVersion, 0, bindSQL, preparedObj.RelateVersion) if err != nil { return err } diff --git a/session/session.go b/session/session.go index ca9d361893faa..ef665e50c07e2 100644 --- a/session/session.go +++ b/session/session.go @@ -397,7 +397,7 @@ func (s *session) cleanRetryInfo() { stmtText, stmtDB = preparedObj.StmtText, preparedObj.StmtDB bindSQL, _ := plannercore.GetBindSQL4PlanCache(s, preparedObj) cacheKey, err = plannercore.NewPlanCacheKey(s.sessionVars, stmtText, stmtDB, preparedAst.SchemaVersion, - 0, bindSQL) + 0, bindSQL, preparedObj.RelateVersion) if err != nil { logutil.Logger(s.currentCtx).Warn("clean cached plan failed", zap.Error(err)) return diff --git a/sessionctx/stmtctx/stmtctx.go b/sessionctx/stmtctx/stmtctx.go index d8e0bc7cd5fb7..d41f2ece45e30 100644 --- a/sessionctx/stmtctx/stmtctx.go +++ b/sessionctx/stmtctx/stmtctx.go @@ -383,6 +383,9 @@ type StatementContext struct { value *uint64 eval func() (uint64, error) } + + // MDLRelatedTableIDs is used to store the table IDs that are related to the current MDL lock. + MDLRelatedTableIDs map[int64]int64 } // StmtHints are SessionVars related sql hints. From 052c22a7a95800d29bb27942db44406e9c31319c Mon Sep 17 00:00:00 2001 From: wjhuang2016 Date: Tue, 4 Jun 2024 17:18:53 +0800 Subject: [PATCH 2/5] done Signed-off-by: wjhuang2016 --- .../core/tests/prepare/prepare_test.go | 1434 ----------------- 1 file changed, 1434 deletions(-) delete mode 100644 pkg/planner/core/tests/prepare/prepare_test.go diff --git a/pkg/planner/core/tests/prepare/prepare_test.go b/pkg/planner/core/tests/prepare/prepare_test.go deleted file mode 100644 index 6d11919c502d6..0000000000000 --- a/pkg/planner/core/tests/prepare/prepare_test.go +++ /dev/null @@ -1,1434 +0,0 @@ -// 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" - "github.com/pingcap/tidb/pkg/sessionctx/variable" - "github.com/pingcap/tidb/pkg/testkit" - "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) - } - - rand.Seed(time.Now().Unix()) - 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(), is, &hint.BlockHintProcessor{}) - 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.GetPlanFromSessionPlanCache(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`) - - tk.MustExec("use test") - for _, pruneMode := range []string{string(variable.Static), string(variable.Dynamic)} { - 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("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("execute stmt5 using @id").Check(testkit.Rows("abc")) - tk.MustExec("set @id=5") - tk.MustQuery("execute stmt5 using @id").Check(testkit.Rows("def")) - tk.MustQuery("execute stmt5 using @id").Check(testkit.Rows("def")) - tk.MustExec("set @id=13") - tk.MustQuery("execute stmt5 using @id").Check(testkit.Rows("xyz")) - tk.MustExec("set @id=17") - tk.MustQuery("execute stmt5 using @id").Check(testkit.Rows("hij")) - - 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("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("execute stmt6 using @id").Check(testkit.Rows("abc")) - tk.MustExec("set @id=5") - tk.MustQuery("execute stmt6 using @id").Check(testkit.Rows("def")) - tk.MustQuery("execute stmt6 using @id").Check(testkit.Rows("def")) - tk.MustExec("set @id=13") - tk.MustQuery("execute stmt6 using @id").Check(testkit.Rows("xyz")) - tk.MustExec("set @id=17") - tk.MustQuery("execute stmt6 using @id").Check(testkit.Rows("hij")) - - // 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("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("execute stmt7 using @id").Check(testkit.Rows("abc")) - tk.MustExec("set @id=5") - tk.MustQuery("execute stmt7 using @id").Check(testkit.Rows("def")) - tk.MustQuery("execute stmt7 using @id").Check(testkit.Rows("def")) - tk.MustExec("set @id=9") - tk.MustQuery("execute stmt7 using @id").Check(testkit.Rows("xyz")) - tk.MustExec("set @id=12") - tk.MustQuery("execute stmt7 using @id").Check(testkit.Rows("hij")) - tk.MustExec("set @id=100") - tk.MustQuery("execute stmt7 using @id").Check(testkit.Rows()) - - // 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("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("execute stmt8 using @id").Check(testkit.Rows("abc")) - tk.MustExec("set @id=5") - tk.MustQuery("execute stmt8 using @id").Check(testkit.Rows("def")) - tk.MustQuery("execute stmt8 using @id").Check(testkit.Rows("def")) - tk.MustExec("set @id=9") - tk.MustQuery("execute stmt8 using @id").Check(testkit.Rows("xyz")) - tk.MustExec("set @id=12") - tk.MustQuery("execute stmt8 using @id").Check(testkit.Rows("hij")) - tk.MustExec("set @id=100") - tk.MustQuery("execute stmt8 using @id").Check(testkit.Rows()) - - // 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(`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()) - 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")) - if pruneMode == string(variable.Dynamic) { - // When the temporary disabling of prepared plan cache for dynamic partition prune mode is disabled, change this to 1! - tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("0")) - } - } -} - -func newSession(t *testing.T, store kv.Storage, dbName string) session.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 session.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()) - err := counter.Write(pb) - require.NoError(t, err) - cnt := pb.GetCounter().GetValue() - require.Equal(t, float64(1), 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()) - 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") - - 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+rand.Intn(100), 1+rand.Intn(12), 1+rand.Intn(28), // date - rand.Intn(24), rand.Intn(60), rand.Intn(60)) // time - } - randDate := func() string { - return fmt.Sprintf("%v-%v-%v", 1950+rand.Intn(100), 1+rand.Intn(12), 1+rand.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)", rand.Intn(100000000), rand.Intn(100000000)) }, - func() string { return fmt.Sprintf("%v", rand.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)", rand.Intn(100000000), rand.Intn(100000000)) }, - func() string { return fmt.Sprintf("%v", rand.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")`, rand.Intn(1000), rand.Intn(1000)) }, - func() string { return fmt.Sprintf(`"%v"`, rand.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 - `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)", rand.Intn(20), rand.Intn(20)) }, - func() string { return fmt.Sprintf("%v", rand.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, ","))) - - // 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")) - - for i := 0; i < 100; i++ { - tk.MustExec(fmt.Sprintf("set @a=%v", tc.varGener())) - result1 := tk.MustQuery("execute stmt1 using @a").Sort().Rows() - // When https://github.com/pingcap/tidb/pull/33098 is reverted this should be 1 again - tk.MustQuery("select @@last_plan_from_cache /* i=" + strconv.Itoa(i) + " prepared statement: (t1) " + tc.query + "\n-- create table: " + tc.t1Create + "*/").Check(testkit.Rows("0")) - tk.MustQuery("execute stmt2 using @a").Sort().Check(result1) - tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1")) - } - } -} - -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) - } - - // 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)) - } - for i := 0; i < 100; 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))) - tk.MustExec(fmt.Sprintf(`set @a0=%v, @a1=%v, @a2=%v`, rand.Intn(40000), rand.Intn(40000), rand.Intn(40000))) - - var rscan, rpoint, rbatch [][]interface{} - 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) - } - - point := tk.MustQuery(fmt.Sprintf(`execute stmt%v_pointget using @pointa`, tbl)).Sort() - if id == 0 { - rpoint = point.Rows() - } else { - point.Check(rpoint) - } - - 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) - } - } - } - - // 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)) - } - loops := 100 - missedPlanCache := make([]string, 0, 4) - 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))) - tk.MustExec(fmt.Sprintf(`set @a0=%v, @a1=%v, @a2=%v`, rand.Intn(40000), rand.Intn(40000), rand.Intn(40000))) - - var rscan, rlookup, rpoint, rbatch [][]interface{} - var expectedFromPlanCache string - for id, tbl := range []string{"trangeIdx", "thashIdx", "tnormalIdx"} { - scan := tk.MustQuery(fmt.Sprintf(`execute stmt%v_indexscan using @mina, @maxa`, tbl)).Sort() - if id == 2 { - expectedFromPlanCache = "1" - } else { - expectedFromPlanCache = "0" - } - tblStr := ` table: ` + tbl + " i :" + strconv.FormatInt(int64(i), 10) + " */" - if i > 0 { - missedPlanCache = helperCheckPlanCache(t, tk, `select @@last_plan_from_cache /* indexscan table: `+tblStr, expectedFromPlanCache, 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, expectedFromPlanCache, 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, expectedFromPlanCache, 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, expectedFromPlanCache, missedPlanCache) - } - if id == 0 { - rbatch = batch.Rows() - } else { - batch.Check(rbatch) - } - } - } - // Allow ~1% non-cached queries, due to background changes etc. - // (Actually just 1/3 %, since there are 3 tables * 4 queries per loop :) - if len(missedPlanCache) > (loops * 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")) -} From 73a1a113956397ae89e35d287655fbd7acc1002c Mon Sep 17 00:00:00 2001 From: wjhuang2016 Date: Tue, 4 Jun 2024 17:31:11 +0800 Subject: [PATCH 3/5] fix Signed-off-by: wjhuang2016 --- ddl/metadatalocktest/mdl_test.go | 2 -- planner/core/plan_cache_utils_test.go | 8 ++++---- 2 files changed, 4 insertions(+), 6 deletions(-) diff --git a/ddl/metadatalocktest/mdl_test.go b/ddl/metadatalocktest/mdl_test.go index 0ed550da6c34e..4a4cc2a90c9a8 100644 --- a/ddl/metadatalocktest/mdl_test.go +++ b/ddl/metadatalocktest/mdl_test.go @@ -874,7 +874,6 @@ func TestMDLPreparePlanCacheInvalid(t *testing.T) { func TestMDLPreparePlanCacheExecute(t *testing.T) { store, dom := testkit.CreateMockStoreAndDomain(t) - defer ingesttestutil.InjectMockBackendMgr(t, store)() sv := server.CreateMockServer(t, store) @@ -926,7 +925,6 @@ func TestMDLPreparePlanCacheExecute(t *testing.T) { func TestMDLPreparePlanCacheExecute2(t *testing.T) { store, dom := testkit.CreateMockStoreAndDomain(t) - defer ingesttestutil.InjectMockBackendMgr(t, store)() sv := server.CreateMockServer(t, store) diff --git a/planner/core/plan_cache_utils_test.go b/planner/core/plan_cache_utils_test.go index 6f0938e447263..2ef6766231312 100644 --- a/planner/core/plan_cache_utils_test.go +++ b/planner/core/plan_cache_utils_test.go @@ -32,20 +32,20 @@ func TestCacheKey(t *testing.T) { ctx.GetSessionVars().InRestrictedSQL = false variable.RestrictedReadOnly.Store(false) variable.VarTiDBSuperReadOnly.Store(false) - key, err := NewPlanCacheKey(ctx.GetSessionVars(), "", "test", 1, 1, "") + key, err := NewPlanCacheKey(ctx.GetSessionVars(), "", "test", 1, 1, "", nil) if err.Error() != "no statement text" { t.Fail() // no statement text } - key, err = NewPlanCacheKey(ctx.GetSessionVars(), "select 1", "", 1, 1, "") + key, err = NewPlanCacheKey(ctx.GetSessionVars(), "select 1", "", 1, 1, "", nil) if err != nil { t.Fail() // schema can be nil } key, err = NewPlanCacheKey(ctx.GetSessionVars(), "select 1", "test", 1, 1, - "select /*+ ignore_plan_cache() */ * from t") + "select /*+ ignore_plan_cache() */ * from t", nil) if err != nil { t.Fail() } - key, err = NewPlanCacheKey(ctx.GetSessionVars(), "select 1", "test", 1, 1, "") + key, err = NewPlanCacheKey(ctx.GetSessionVars(), "select 1", "test", 1, 1, "", nil) if err != nil { t.Fail() } From ef9fb0f48c98fd853174595e5631e0b1def100c9 Mon Sep 17 00:00:00 2001 From: tangenta Date: Wed, 12 Jun 2024 09:45:34 +0800 Subject: [PATCH 4/5] cherry-pick #50969 to fix flaky test --- server/conn_stmt_test.go | 31 ++++++------------------------ util/chunk/row_container_reader.go | 7 +++++++ 2 files changed, 13 insertions(+), 25 deletions(-) diff --git a/server/conn_stmt_test.go b/server/conn_stmt_test.go index 6d91c5ae4071a..e835a700843e9 100644 --- a/server/conn_stmt_test.go +++ b/server/conn_stmt_test.go @@ -20,12 +20,6 @@ import ( "crypto/rand" "encoding/binary" "fmt" - "io/fs" - "os" - "path/filepath" - "strconv" - "strings" - "syscall" "testing" "github.com/pingcap/failpoint" @@ -554,30 +548,17 @@ func TestCursorFetchErrorInFetch(t *testing.T) { tk.MustExec(fmt.Sprintf("set tidb_mem_quota_query=%d", 1)) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/util/chunk/get-chunk-error", "return(true)")) + defer func() { + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/util/chunk/get-chunk-error")) + }() + require.NoError(t, c.Dispatch(ctx, append( appendUint32([]byte{mysql.ComStmtExecute}, uint32(stmt.ID())), mysql.CursorTypeReadOnly, 0x1, 0x0, 0x0, 0x0, ))) - // close these disk files to produce error - filepath.Walk("/proc/self/fd", func(path string, info fs.FileInfo, err error) error { - if err != nil { - return nil - } - target, err := os.Readlink(path) - if err != nil { - return nil - } - if strings.HasPrefix(target, tmpStoragePath) { - fd, err := strconv.Atoi(filepath.Base(path)) - require.NoError(t, err) - require.NoError(t, syscall.Close(fd)) - } - return nil - }) - - // it'll get "bad file descriptor", as it has been closed in the test. - require.Error(t, c.Dispatch(ctx, appendUint32(appendUint32([]byte{mysql.ComStmtFetch}, uint32(stmt.ID())), 1024))) + require.ErrorContains(t, c.Dispatch(ctx, appendUint32(appendUint32([]byte{mysql.ComStmtFetch}, uint32(stmt.ID())), 1024)), "fail to get chunk for test") // after getting a failed FETCH, the cursor should have been reseted require.False(t, stmt.GetCursorActive()) require.Len(t, tk.Session().GetSessionVars().MemTracker.GetChildrenForTest(), 0) diff --git a/util/chunk/row_container_reader.go b/util/chunk/row_container_reader.go index b96b20c6921c1..24a440f878f9d 100644 --- a/util/chunk/row_container_reader.go +++ b/util/chunk/row_container_reader.go @@ -19,6 +19,8 @@ import ( "runtime" "sync" + "github.com/pingcap/errors" + "github.com/pingcap/failpoint" "github.com/pingcap/tidb/util/logutil" ) @@ -122,6 +124,11 @@ func (reader *rowContainerReader) startWorker() { for chkIdx := 0; chkIdx < reader.rc.NumChunks(); chkIdx++ { chk, err := reader.rc.GetChunk(chkIdx) + failpoint.Inject("get-chunk-error", func(val failpoint.Value) { + if val.(bool) { + err = errors.New("fail to get chunk for test") + } + }) if err != nil { reader.err = err return From 6a2a562d9aa7c864acbeab220b3399930372b516 Mon Sep 17 00:00:00 2001 From: tangenta Date: Wed, 12 Jun 2024 12:01:01 +0800 Subject: [PATCH 5/5] fix TestCacheHitInForUpdateRead --- planner/core/prepare_test.go | 17 +++++++++-------- 1 file changed, 9 insertions(+), 8 deletions(-) diff --git a/planner/core/prepare_test.go b/planner/core/prepare_test.go index f9b651060f5db..d75e91f8d27ee 100644 --- a/planner/core/prepare_test.go +++ b/planner/core/prepare_test.go @@ -2674,14 +2674,15 @@ func verifyCache(ctx context.Context, t *testing.T, tk1 *testkit.TestKit, tk2 *t 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 - 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")) + // 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) {