Skip to content

Commit

Permalink
This is an automated cherry-pick of pingcap#40080
Browse files Browse the repository at this point in the history
Signed-off-by: ti-chi-bot <ti-community-prow-bot@tidb.io>
  • Loading branch information
qw4990 authored and ti-chi-bot committed Jan 18, 2023
1 parent 77cea7a commit b1adc61
Show file tree
Hide file tree
Showing 5 changed files with 1,148 additions and 6 deletions.
10 changes: 5 additions & 5 deletions executor/seqtest/prepared_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -495,14 +495,14 @@ func TestPreparedInsert(t *testing.T) {
err = counter.Write(pb)
require.NoError(t, err)
hit := pb.GetCounter().GetValue()
require.Equal(t, float64(1), hit)
require.Equal(t, float64(0), hit) // insert-values-stmt cannot use the plan cache
}
tk.MustExec(`set @a=3,@b=3; execute stmt_insert using @a, @b;`)
if flag {
err = counter.Write(pb)
require.NoError(t, err)
hit := pb.GetCounter().GetValue()
require.Equal(t, float64(2), hit)
require.Equal(t, float64(0), hit)
}

result := tk.MustQuery("select id, c1 from prepare_test where id = ?", 1)
Expand All @@ -518,21 +518,21 @@ func TestPreparedInsert(t *testing.T) {
err = counter.Write(pb)
require.NoError(t, err)
hit := pb.GetCounter().GetValue()
require.Equal(t, float64(2), hit)
require.Equal(t, float64(0), hit)
}
tk.MustExec(`set @a=2; execute stmt_insert_select using @a;`)
if flag {
err = counter.Write(pb)
require.NoError(t, err)
hit := pb.GetCounter().GetValue()
require.Equal(t, float64(3), hit)
require.Equal(t, float64(1), hit)
}
tk.MustExec(`set @a=3; execute stmt_insert_select using @a;`)
if flag {
err = counter.Write(pb)
require.NoError(t, err)
hit := pb.GetCounter().GetValue()
require.Equal(t, float64(4), hit)
require.Equal(t, float64(2), hit)
}

result = tk.MustQuery("select id, c1 from prepare_test where id = ?", 101)
Expand Down
15 changes: 15 additions & 0 deletions planner/core/cacheable_checker.go
Original file line number Diff line number Diff line change
Expand Up @@ -87,6 +87,21 @@ func (checker *cacheableChecker) Enter(in ast.Node) (out ast.Node, skipChildren
return in, true
}
}
case *ast.InsertStmt:
if node.Select == nil {
// do not cache insert-values-stmt like 'insert into t values (...)' since
// no performance benefit and to save memory.
checker.cacheable = false
checker.reason = "ignore insert-values-stmt"
return in, true
}
for _, hints := range node.TableHints {
if hints.HintName.L == HintIgnorePlanCache {
checker.cacheable = false
checker.reason = "ignore plan cache by hint"
return in, true
}
}
case *ast.VariableExpr, *ast.ExistsSubqueryExpr, *ast.SubqueryExpr:
checker.cacheable = false
return in, true
Expand Down
4 changes: 3 additions & 1 deletion planner/core/cacheable_checker_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -52,7 +52,9 @@ func TestCacheable(t *testing.T) {

tableRefsClause := &ast.TableRefsClause{TableRefs: &ast.Join{Left: &ast.TableSource{Source: tbl}}}
// test InsertStmt
stmt = &ast.InsertStmt{Table: tableRefsClause}
stmt = &ast.InsertStmt{Table: tableRefsClause} // insert-values-stmt
require.False(t, core.Cacheable(stmt, is))
stmt = &ast.InsertStmt{Table: tableRefsClause, Select: &ast.SelectStmt{}} // insert-select-stmt
require.True(t, core.Cacheable(stmt, is))

// test DeleteStmt
Expand Down
304 changes: 304 additions & 0 deletions planner/core/plan_cache_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,304 @@
// Copyright 2022 PingCAP, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.

package core_test

import (
"context"
"errors"
"fmt"
"math/rand"
"strconv"
"strings"
"testing"

"github.com/pingcap/tidb/expression"
"github.com/pingcap/tidb/parser/mysql"
plannercore "github.com/pingcap/tidb/planner/core"
"github.com/pingcap/tidb/testkit"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util"
"github.com/stretchr/testify/require"
)

type mockParameterizer struct {
action string
}

func (mp *mockParameterizer) Parameterize(originSQL string) (paramSQL string, params []expression.Expression, ok bool, err error) {
switch mp.action {
case "error":
return "", nil, false, errors.New("error")
case "not_support":
return "", nil, false, nil
}
// only support SQL like 'select * from t where col {op} {int} and ...'
prefix := "select * from t where "
if !strings.HasPrefix(originSQL, prefix) {
return "", nil, false, nil
}
buf := make([]byte, 0, 32)
buf = append(buf, prefix...)
for i, condStr := range strings.Split(originSQL[len(prefix):], "and") {
if i > 0 {
buf = append(buf, " and "...)
}
tmp := strings.Split(strings.TrimSpace(condStr), " ")
if len(tmp) != 3 { // col {op} {val}
return "", nil, false, nil
}
buf = append(buf, tmp[0]...)
buf = append(buf, tmp[1]...)
buf = append(buf, '?')

intParam, err := strconv.Atoi(tmp[2])
if err != nil {
return "", nil, false, nil
}
params = append(params, &expression.Constant{Value: types.NewDatum(intParam), RetType: types.NewFieldType(mysql.TypeLong)})
}
return string(buf), params, true, nil
}

func TestInitLRUWithSystemVar(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("set @@session.tidb_prepared_plan_cache_size = 0") // MinValue: 1
tk.MustQuery("select @@session.tidb_prepared_plan_cache_size").Check(testkit.Rows("1"))
sessionVar := tk.Session().GetSessionVars()

lru := plannercore.NewLRUPlanCache(uint(sessionVar.PreparedPlanCacheSize), 0, 0, plannercore.PickPlanFromBucket, tk.Session())
require.NotNil(t, lru)
}

func TestNonPreparedPlanCacheWithExplain(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec(`use test`)
tk.MustExec("create table t(a int)")
tk.MustExec("set tidb_enable_non_prepared_plan_cache=1")
tk.MustExec("select * from t where a=1") // cache this plan

tk.MustQuery("explain select * from t where a=2").Check(testkit.Rows(
`Selection_8 10.00 root eq(test.t.a, 2)`,
`└─TableReader_7 10.00 root data:Selection_6`,
` └─Selection_6 10.00 cop[tikv] eq(test.t.a, 2)`,
` └─TableFullScan_5 10000.00 cop[tikv] table:t keep order:false, stats:pseudo`))
tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1"))

tk.MustQuery("explain format=verbose select * from t where a=2").Check(testkit.Rows(
`Selection_8 10.00 169474.57 root eq(test.t.a, 2)`,
`└─TableReader_7 10.00 168975.57 root data:Selection_6`,
` └─Selection_6 10.00 2534000.00 cop[tikv] eq(test.t.a, 2)`,
` └─TableFullScan_5 10000.00 2035000.00 cop[tikv] table:t keep order:false, stats:pseudo`))
tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1"))

tk.MustQuery("explain analyze select * from t where a=2").CheckAt([]int{0, 1, 2, 3}, [][]interface{}{
{"Selection_8", "10.00", "0", "root"},
{"└─TableReader_7", "10.00", "0", "root"},
{" └─Selection_6", "10.00", "0", "cop[tikv]"},
{" └─TableFullScan_5", "10000.00", "0", "cop[tikv]"},
})
tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1"))
}

func TestNonPreparedPlanCacheFallback(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec(`use test`)
tk.MustExec(`create table t (a int)`)
for i := 0; i < 5; i++ {
tk.MustExec(fmt.Sprintf("insert into t values (%v)", i))
}
tk.MustExec("set tidb_enable_non_prepared_plan_cache=1")

// inject a fault to GeneratePlanCacheStmtWithAST
ctx := context.WithValue(context.Background(), "____GeneratePlanCacheStmtWithASTErr", struct{}{})
tk.MustQueryWithContext(ctx, "select * from t where a in (1, 2)").Sort().Check(testkit.Rows("1", "2"))
tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) // cannot generate PlanCacheStmt
tk.MustQueryWithContext(ctx, "select * from t where a in (1, 3)").Sort().Check(testkit.Rows("1", "3"))
tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) // cannot generate PlanCacheStmt
tk.MustQuery("select * from t where a in (1, 2)").Sort().Check(testkit.Rows("1", "2"))
tk.MustQuery("select * from t where a in (1, 3)").Sort().Check(testkit.Rows("1", "3"))
tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1")) // no error

// inject a fault to GetPlanFromSessionPlanCache
tk.MustQuery("select * from t where a=1").Check(testkit.Rows("1")) // cache this plan
tk.MustQuery("select * from t where a=2").Check(testkit.Rows("2")) // plan from cache
tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1"))
ctx = context.WithValue(context.Background(), "____GetPlanFromSessionPlanCacheErr", struct{}{})
tk.MustQueryWithContext(ctx, "select * from t where a=3").Check(testkit.Rows("3"))
tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) // fallback to the normal opt-path
tk.MustQueryWithContext(ctx, "select * from t where a=4").Check(testkit.Rows("4"))
tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) // fallback to the normal opt-path
tk.MustQueryWithContext(context.Background(), "select * from t where a=0").Check(testkit.Rows("0"))
tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1")) // use the cached plan if no error

// inject a fault to RestoreASTWithParams
ctx = context.WithValue(context.Background(), "____GetPlanFromSessionPlanCacheErr", struct{}{})
ctx = context.WithValue(ctx, "____RestoreASTWithParamsErr", struct{}{})
_, err := tk.ExecWithContext(ctx, "select * from t where a=1")
require.NotNil(t, err)
}

func TestNonPreparedPlanCacheBasically(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec(`use test`)
tk.MustExec(`create table t (a int, b int, c int, d int, primary key(a), key(b), key(c, d))`)
for i := 0; i < 20; i++ {
tk.MustExec(fmt.Sprintf("insert into t values (%v, %v, %v, %v)", i, rand.Intn(20), rand.Intn(20), rand.Intn(20)))
}

queries := []string{
"select * from t where a<10",
"select * from t where a<13 and b<15",
"select * from t where b=13",
"select * from t where c<8",
"select * from t where d>8",
"select * from t where c=8 and d>10",
"select * from t where a<12 and b<13 and c<12 and d>2",
}

for _, query := range queries {
tk.MustExec(`set tidb_enable_non_prepared_plan_cache=0`)
resultNormal := tk.MustQuery(query).Sort()
tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("0"))

tk.MustExec(`set tidb_enable_non_prepared_plan_cache=1`)
tk.MustQuery(query) // first process
tk.MustQuery(query).Sort().Check(resultNormal.Rows()) // equal to the result without plan-cache
tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("1")) // this plan is from plan-cache
}
}

func TestIssue38269(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec(`set @@tidb_enable_prepared_plan_cache=1`)
tk.MustExec("set @@tidb_enable_collect_execution_info=0")
tk.MustExec("use test")
tk.MustExec("create table t1(a int)")
tk.MustExec("create table t2(a int, b int, c int, index idx(a, b))")
tk.MustExec("prepare stmt1 from 'select /*+ inl_join(t2) */ * from t1 join t2 on t1.a = t2.a where t2.b in (?, ?, ?)'")
tk.MustExec("set @a = 10, @b = 20, @c = 30, @d = 40, @e = 50, @f = 60")
tk.MustExec("execute stmt1 using @a, @b, @c")
tk.MustExec("execute stmt1 using @d, @e, @f")
tkProcess := tk.Session().ShowProcess()
ps := []*util.ProcessInfo{tkProcess}
tk.Session().SetSessionManager(&testkit.MockSessionManager{PS: ps})
rows := tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)).Rows()
require.Contains(t, rows[6][4], "range: decided by [eq(test.t2.a, test.t1.a) in(test.t2.b, 40, 50, 60)]")
}

func TestIssue38533(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
tk.MustExec("create table t (a int, key (a))")
tk.MustExec(`prepare st from "select /*+ use_index(t, a) */ a from t where a=? and a=?"`)
tk.MustExec(`set @a=1`)
tk.MustExec(`execute st using @a, @a`)
tkProcess := tk.Session().ShowProcess()
ps := []*util.ProcessInfo{tkProcess}
tk.Session().SetSessionManager(&testkit.MockSessionManager{PS: ps})
plan := tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)).Rows()
require.True(t, strings.Contains(plan[1][0].(string), "RangeScan")) // range-scan instead of full-scan

tk.MustExec(`execute st using @a, @a`)
tk.MustExec(`execute st using @a, @a`)
tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0"))
}

func TestIgnoreInsertStmt(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
tk.MustExec("create table t (a int)")

// do not cache native insert-stmt
tk.MustExec("prepare st from 'insert into t values (1)'")
tk.MustExec("execute st")
tk.MustExec("execute st")
tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0"))

// ignore-hint in insert-stmt can work
tk.MustExec("prepare st from 'insert into t select * from t'")
tk.MustExec("execute st")
tk.MustExec("execute st")
tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1"))
tk.MustExec("prepare st from 'insert /*+ ignore_plan_cache() */ into t select * from t'")
tk.MustExec("execute st")
tk.MustExec("execute st")
tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0"))
}

func TestIssue38710(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
tk.MustExec("drop table if exists UK_NO_PRECISION_19392;")
tk.MustExec("CREATE TABLE `UK_NO_PRECISION_19392` (\n `COL1` bit(1) DEFAULT NULL,\n `COL2` varchar(20) COLLATE utf8mb4_bin DEFAULT NULL,\n `COL4` datetime DEFAULT NULL,\n `COL3` bigint DEFAULT NULL,\n `COL5` float DEFAULT NULL,\n UNIQUE KEY `UK_COL1` (`COL1`)\n) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin;")
tk.MustExec("INSERT INTO `UK_NO_PRECISION_19392` VALUES (0x00,'缎馗惫砲兣肬憵急鳸嫅稩邏蠧鄂艘腯灩專妴粈','9294-12-26 06:50:40',-3088380202191555887,-3.33294e38),(NULL,'仲膩蕦圓猴洠飌镂喵疎偌嫺荂踖Ƕ藨蜿諪軁笞','1746-08-30 18:04:04',-4016793239832666288,-2.52633e38),(0x01,'冑溜畁脊乤纊繳蟥哅稐奺躁悼貘飗昹槐速玃沮','1272-01-19 23:03:27',-8014797887128775012,1.48868e38);\n")
tk.MustExec(`prepare stmt from 'select * from UK_NO_PRECISION_19392 where col1 between ? and ? or col3 = ? or col2 in (?, ?, ?);';`)
tk.MustExec("set @a=0x01, @b=0x01, @c=-3088380202191555887, @d=\"缎馗惫砲兣肬憵急鳸嫅稩邏蠧鄂艘腯灩專妴粈\", @e=\"缎馗惫砲兣肬憵急鳸嫅稩邏蠧鄂艘腯灩專妴粈\", @f=\"缎馗惫砲兣肬憵急鳸嫅稩邏蠧鄂艘腯灩專妴粈\";")
rows := tk.MustQuery(`execute stmt using @a,@b,@c,@d,@e,@f;`) // can not be cached because @a = @b
require.Equal(t, 2, len(rows.Rows()))

tk.MustExec(`set @a=NULL, @b=NULL, @c=-4016793239832666288, @d="缎馗惫砲兣肬憵急鳸嫅稩邏蠧鄂艘腯灩專妴粈", @e="仲膩蕦圓猴洠飌镂喵疎偌嫺荂踖Ƕ藨蜿諪軁笞", @f="缎馗惫砲兣肬憵急鳸嫅稩邏蠧鄂艘腯灩專妴粈";`)
rows = tk.MustQuery(`execute stmt using @a,@b,@c,@d,@e,@f;`)
require.Equal(t, 2, len(rows.Rows()))
tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0"))

rows = tk.MustQuery(`execute stmt using @a,@b,@c,@d,@e,@f;`)
require.Equal(t, 2, len(rows.Rows()))
tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1"))

tk.MustExec(`set @a=0x01, @b=0x01, @c=-3088380202191555887, @d="缎馗惫砲兣肬憵急鳸嫅稩邏蠧鄂艘腯灩專妴粈", @e="缎馗惫砲兣肬憵急鳸嫅稩邏蠧鄂艘腯灩專妴粈", @f="缎馗惫砲兣肬憵急鳸嫅稩邏蠧鄂艘腯灩專妴粈";`)
rows = tk.MustQuery(`execute stmt using @a,@b,@c,@d,@e,@f;`)
require.Equal(t, 2, len(rows.Rows()))
tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) // can not use the cache because the types for @a and @b are not equal to the cached plan
}

func TestPlanCacheDiagInfo(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
tk.MustExec("create table t (a int, b int, key(a), key(b))")

tk.MustExec("prepare stmt from 'select * from t where a in (select a from t)'")
tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 skip plan-cache: query has sub-queries is un-cacheable"))

tk.MustExec("prepare stmt from 'select /*+ ignore_plan_cache() */ * from t'")
tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 skip plan-cache: ignore plan cache by hint"))

tk.MustExec("prepare stmt from 'select * from t limit ?'")
tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 skip plan-cache: query has 'limit ?' is un-cacheable"))

tk.MustExec("prepare stmt from 'select * from t limit ?, 1'")
tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 skip plan-cache: query has 'limit ?, 10' is un-cacheable"))

tk.MustExec("prepare stmt from 'select * from t order by ?'")
tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 skip plan-cache: query has 'order by ?' is un-cacheable"))

tk.MustExec("prepare stmt from 'select * from t where a=?'")
tk.MustExec("set @a='123'")
tk.MustExec("execute stmt using @a") // '123' -> 123
tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 skip plan-cache: '123' may be converted to INT"))

tk.MustExec("prepare stmt from 'select * from t where a=? and a=?'")
tk.MustExec("set @a=1, @b=1")
tk.MustExec("execute stmt using @a, @b") // a=1 and a=1 -> a=1
tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 skip plan-cache: some parameters may be overwritten"))
}
Loading

0 comments on commit b1adc61

Please sign in to comment.