Skip to content

Commit

Permalink
This is an automated cherry-pick of #38537
Browse files Browse the repository at this point in the history
Signed-off-by: ti-chi-bot <[email protected]>
  • Loading branch information
qw4990 authored and ti-chi-bot committed Dec 28, 2022
1 parent daf2b17 commit 137f641
Show file tree
Hide file tree
Showing 4 changed files with 161 additions and 13 deletions.
2 changes: 1 addition & 1 deletion executor/explainfor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -952,7 +952,7 @@ func TestIndexMerge4PlanCache(t *testing.T) {
tk.MustExec("set @a=9, @b=10, @c=11;")
tk.MustQuery("execute stmt using @a, @a;").Check(testkit.Rows("10 10 10"))
tk.MustQuery("execute stmt using @a, @c;").Check(testkit.Rows("10 10 10", "11 11 11"))
tk.MustQuery("select @@last_plan_from_cache;").Check(testkit.Rows("1"))
tk.MustQuery("select @@last_plan_from_cache;").Check(testkit.Rows("0")) // a>=9 and a<=9 --> a=9
tk.MustQuery("execute stmt using @c, @a;").Check(testkit.Rows("10 10 10"))
tk.MustQuery("select @@last_plan_from_cache;").Check(testkit.Rows("1"))

Expand Down
151 changes: 151 additions & 0 deletions planner/core/plan_cache_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,151 @@
// 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 (
"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)
require.NotNil(t, lru)
}

func TestGeneralPlanCacheBasically(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_general_plan_cache=0`)
resultNormal := tk.MustQuery(query).Sort()
tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("0"))

tk.MustExec(`set tidb_enable_general_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"))
}
17 changes: 7 additions & 10 deletions planner/core/prepare_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1583,7 +1583,7 @@ func TestIssue29303(t *testing.T) {
tk.MustQuery(`execute stmt using @a,@b,@c,@d`).Check(testkit.Rows())
tk.MustExec(`set @a="龂", @b="龂", @c="龂", @d="龂"`)
tk.MustQuery(`execute stmt using @a,@b,@c,@d`).Check(testkit.Rows("� 龂 � 龂"))
tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("0"))
tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("1"))
}

func TestIssue34725(t *testing.T) {
Expand Down Expand Up @@ -2218,11 +2218,10 @@ func TestPlanCachePointGetAndTableDual(t *testing.T) {
tk.MustExec("insert into t1 values('0000','7777',1)")
tk.MustExec("prepare s1 from 'select * from t1 where c1=? and c2>=? and c2<=?'")
tk.MustExec("set @a1='0000', @b1='9999'")
// IndexLookup plan would be built, we should cache it.
tk.MustQuery("execute s1 using @a1, @b1, @b1").Check(testkit.Rows())
tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0"))
tk.MustQuery("execute s1 using @a1, @a1, @b1").Check(testkit.Rows("0000 7777 1"))
tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1"))
tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) // c2>=9999 and c2<=9999 --> c2=9999

tk.MustExec("create table t2(c1 bigint(20) primary key, c2 varchar(20))")
tk.MustExec("insert into t2 values(1,'7777')")
Expand All @@ -2238,17 +2237,15 @@ func TestPlanCachePointGetAndTableDual(t *testing.T) {
tk.MustExec("insert into t3 values(2,1,1)")
tk.MustExec("prepare s3 from 'select /*+ use_index_merge(t3) */ * from t3 where (c1 >= ? and c1 <= ?) or c2 > 1'")
tk.MustExec("set @a3=1,@b3=3")
// TableReader plan would be built, we should cache it.
tk.MustQuery("execute s3 using @a3,@a3").Check(testkit.Rows())
tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0"))
tk.MustQuery("execute s3 using @a3,@b3").Check(testkit.Rows("2 1 1"))
tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1"))
tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) // c1>=1 and c1<=1 --> c1==1

tk.MustExec("prepare s3 from 'select /*+ use_index_merge(t3) */ * from t3 where (c1 >= ? and c1 <= ?) or c2 > 1'")
tk.MustExec("set @a3=1,@b3=3")
// TableReader plan would be built, we should cache it.
tk.MustQuery("execute s3 using @b3,@a3").Check(testkit.Rows())
tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1"))
tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0"))
tk.MustQuery("execute s3 using @a3,@b3").Check(testkit.Rows("2 1 1"))
tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1"))

Expand All @@ -2259,7 +2256,7 @@ func TestPlanCachePointGetAndTableDual(t *testing.T) {
tk.MustQuery("execute s4 using @a4,@a4").Check(testkit.Rows())
tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0"))
tk.MustQuery("execute s4 using @a4,@b4").Check(testkit.Rows("2 1 1"))
tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1"))
tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) // c1>=3 and c1<=3 --> c1=3

tk.MustExec("prepare s4 from 'select /*+ use_index_merge(t4) */ * from t4 where (c1 >= ? and c1 <= ?) or c2 > 1'")
tk.MustExec("set @a4=1,@b4=3")
Expand Down Expand Up @@ -2337,7 +2334,7 @@ func TestIssue23671(t *testing.T) {
tk.MustQuery("execute s1 using @a, @b, @c").Check(testkit.Rows("1 1"))
tk.MustExec("set @a=1, @b=1, @c=10")
tk.MustQuery("execute s1 using @a, @b, @c").Check(testkit.Rows("1 1", "2 2"))
tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1"))
tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) // b>=1 and b<=1 --> b=1
}

func TestIssue29296(t *testing.T) {
Expand Down Expand Up @@ -2877,7 +2874,7 @@ func TestCachedTable(t *testing.T) {
// IndexLookup
tk.MustQuery("execute indexLookup using @a, @b").Check(testkit.Rows("2"))
require.True(t, lastReadFromCache(tk))
tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1"))
tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) // b>1 and b<3 --> b=2

// PointGet
tk.MustQuery("execute pointGet using @a").Check(testkit.Rows("1"))
Expand Down
4 changes: 2 additions & 2 deletions util/ranger/detacher.go
Original file line number Diff line number Diff line change
Expand Up @@ -578,8 +578,8 @@ func ExtractEqAndInCondition(sctx sessionctx.Context, conditions []expression.Ex
columnValues[i] = &valueInfo{mutable: true}
}
if expression.MaybeOverOptimized4PlanCache(sctx, conditions) {
// TODO: optimize it more elaborately, e.g. return [2 3, 2 3] as accesses for 'where a = 2 and b = 3 and c >= ? and c <= ?'
return nil, conditions, nil, nil, false
// `a=@x and a=@y` --> `a=@x if @x==@y`
sctx.GetSessionVars().StmtCtx.SkipPlanCache = true
}
}
}
Expand Down

0 comments on commit 137f641

Please sign in to comment.