Skip to content

Commit

Permalink
Merge branch 'master' into enable_race_2
Browse files Browse the repository at this point in the history
  • Loading branch information
hawkingrei authored Sep 5, 2022
2 parents b82f70d + b2723bf commit 85cd17f
Show file tree
Hide file tree
Showing 12 changed files with 79 additions and 69 deletions.
5 changes: 2 additions & 3 deletions executor/seqtest/prepared_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,6 @@ import (
"github.com/pingcap/tidb/server"
"github.com/pingcap/tidb/session"
"github.com/pingcap/tidb/testkit"
"github.com/pingcap/tidb/util/kvcache"
dto "github.com/prometheus/client_model/go"
"github.com/stretchr/testify/require"
)
Expand Down Expand Up @@ -344,7 +343,7 @@ func TestPrepareWithAggregation(t *testing.T) {
tk.MustExec(fmt.Sprintf(`set @@tidb_enable_prepared_plan_cache=%v`, flag))

se, err := session.CreateSession4TestWithOpt(store, &session.Opt{
PreparedPlanCache: kvcache.NewSimpleLRUCache(100, 0.1, math.MaxUint64),
PreparedPlanCache: plannercore.NewLRUPlanCache(100, 0.1, math.MaxUint64, plannercore.PickPlanFromBucket),
})
require.NoError(t, err)
tk.SetSession(se)
Expand Down Expand Up @@ -609,7 +608,7 @@ func TestPrepareDealloc(t *testing.T) {
tk.MustExec(`set @@tidb_enable_prepared_plan_cache=true`)

se, err := session.CreateSession4TestWithOpt(store, &session.Opt{
PreparedPlanCache: kvcache.NewSimpleLRUCache(3, 0.1, math.MaxUint64),
PreparedPlanCache: plannercore.NewLRUPlanCache(3, 0.1, math.MaxUint64, plannercore.PickPlanFromBucket),
})
require.NoError(t, err)
tk.SetSession(se)
Expand Down
4 changes: 2 additions & 2 deletions expression/integration_serial_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,11 +26,11 @@ import (
"github.com/pingcap/failpoint"
"github.com/pingcap/tidb/parser/mysql"
"github.com/pingcap/tidb/parser/terror"
plannercore "github.com/pingcap/tidb/planner/core"
"github.com/pingcap/tidb/session"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/testkit"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/kvcache"
"github.com/stretchr/testify/require"
"github.com/tikv/client-go/v2/oracle"
)
Expand Down Expand Up @@ -3797,7 +3797,7 @@ func TestPreparePlanCacheOnCachedTable(t *testing.T) {

var err error
se, err := session.CreateSession4TestWithOpt(store, &session.Opt{
PreparedPlanCache: kvcache.NewSimpleLRUCache(100, 0.1, math.MaxUint64),
PreparedPlanCache: plannercore.NewLRUPlanCache(100, 0.1, math.MaxUint64, plannercore.PickPlanFromBucket),
})
require.NoError(t, err)
tk.SetSession(se)
Expand Down
5 changes: 3 additions & 2 deletions infoschema/tables_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -34,13 +34,13 @@ import (
"github.com/pingcap/tidb/parser/model"
"github.com/pingcap/tidb/parser/mysql"
"github.com/pingcap/tidb/parser/terror"
plannercore "github.com/pingcap/tidb/planner/core"
"github.com/pingcap/tidb/session"
"github.com/pingcap/tidb/session/txninfo"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/testkit"
"github.com/pingcap/tidb/testkit/testutil"
"github.com/pingcap/tidb/util"
"github.com/pingcap/tidb/util/kvcache"
"github.com/stretchr/testify/require"
)

Expand All @@ -53,7 +53,8 @@ func newTestKitWithRoot(t *testing.T, store kv.Storage) *testkit.TestKit {

func newTestKitWithPlanCache(t *testing.T, store kv.Storage) *testkit.TestKit {
tk := testkit.NewTestKit(t, store)
se, err := session.CreateSession4TestWithOpt(store, &session.Opt{PreparedPlanCache: kvcache.NewSimpleLRUCache(100, 0.1, math.MaxUint64)})
se, err := session.CreateSession4TestWithOpt(store, &session.Opt{PreparedPlanCache: plannercore.NewLRUPlanCache(100,
0.1, math.MaxUint64, plannercore.PickPlanFromBucket)})
require.NoError(t, err)
tk.SetSession(se)
tk.RefreshConnectionID()
Expand Down
5 changes: 3 additions & 2 deletions planner/core/plan_cache.go
Original file line number Diff line number Diff line change
Expand Up @@ -216,10 +216,11 @@ func getGeneralPlan(sctx sessionctx.Context, isGeneralPlanCache bool, cacheKey k
sessVars := sctx.GetSessionVars()
stmtCtx := sessVars.StmtCtx

cachedVal, exist := getValidPlanFromCache(sctx, isGeneralPlanCache, cacheKey, paramTypes)
candidate, exist := sctx.GetPlanCache(isGeneralPlanCache).Get(cacheKey, paramTypes)
if !exist {
return nil, nil, false, nil
}
cachedVal := candidate.(*PlanCacheValue)
if err := CheckPreparedPriv(sctx, stmt, is); err != nil {
return nil, nil, false, err
}
Expand Down Expand Up @@ -288,7 +289,7 @@ func generateNewPlan(ctx context.Context, sctx sessionctx.Context, isGeneralPlan
stmt.NormalizedPlan, stmt.PlanDigest = NormalizePlan(p)
stmtCtx.SetPlan(p)
stmtCtx.SetPlanDigest(stmt.NormalizedPlan, stmt.PlanDigest)
putPlanIntoCache(sctx, isGeneralPlanCache, cacheKey, cached)
sctx.GetPlanCache(isGeneralPlanCache).Put(cacheKey, cached, paramTypes)
}
sessVars.FoundInPlanCache = false
return p, names, err
Expand Down
23 changes: 18 additions & 5 deletions planner/core/plan_cache_lru.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@ import (
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/hack"
"github.com/pingcap/tidb/util/kvcache"
"github.com/pingcap/tidb/util/logutil"
"github.com/pingcap/tidb/util/memory"
)

Expand All @@ -30,7 +31,7 @@ type planCacheEntry struct {
PlanValue kvcache.Value
}

// LRUPlanCache is a dedicated least recently used cache, JUST use for plan cache.
// LRUPlanCache is a dedicated least recently used cache, Only used for plan cache.
type LRUPlanCache struct {
capacity uint
size uint
Expand All @@ -53,9 +54,10 @@ type LRUPlanCache struct {
// NewLRUPlanCache creates a PCLRUCache object, whose capacity is "capacity".
// NOTE: "capacity" should be a positive value.
func NewLRUPlanCache(capacity uint, guard float64, quota uint64,
pickFromBucket func(map[*list.Element]struct{}, []*types.FieldType) (*list.Element, bool)) (*LRUPlanCache, error) {
pickFromBucket func(map[*list.Element]struct{}, []*types.FieldType) (*list.Element, bool)) *LRUPlanCache {
if capacity < 1 {
return nil, errors.New("capacity of LRU Cache should be at least 1")
capacity = 100
logutil.BgLogger().Info("capacity of LRU cache is less than 1, will use default value(100) init cache")
}
return &LRUPlanCache{
capacity: capacity,
Expand All @@ -65,7 +67,7 @@ func NewLRUPlanCache(capacity uint, guard float64, quota uint64,
pickFromBucket: pickFromBucket,
quota: quota,
guard: guard,
}, nil
}
}

// Get tries to find the corresponding value according to the given key.
Expand Down Expand Up @@ -155,7 +157,7 @@ func (l *LRUPlanCache) SetCapacity(capacity uint) error {
defer l.lock.Unlock()

if capacity < 1 {
return errors.New("capacity of lru cache should be at least 1")
return errors.New("capacity of LRU cache should be at least 1")
}
l.capacity = capacity
for l.size > l.capacity {
Expand Down Expand Up @@ -194,3 +196,14 @@ func (l *LRUPlanCache) memoryControl() {
memUsed, _ = memory.InstanceMemUsed()
}
}

// PickPlanFromBucket pick one plan from bucket
func PickPlanFromBucket(bucket map[*list.Element]struct{}, paramTypes []*types.FieldType) (*list.Element, bool) {
for k := range bucket {
plan := k.Value.(*planCacheEntry).PlanValue.(*PlanCacheValue)
if plan.ParamTypes.CheckTypesCompatibility4PC(paramTypes) {
return k, true
}
}
return nil, false
}
24 changes: 9 additions & 15 deletions planner/core/plan_cache_lru_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -70,16 +70,14 @@ func pickFromBucket(bucket map[*list.Element]struct{}, ptypes []*types.FieldType

func TestLRUPCPut(t *testing.T) {
// test initialize
lruA, errA := NewLRUPlanCache(0, 0, 0, pickFromBucket)
require.Nil(t, lruA)
require.Error(t, errA, "capacity of LRU Cache should be at least 1")
lruA := NewLRUPlanCache(0, 0, 0, pickFromBucket)
require.Equal(t, lruA.capacity, uint(100))

maxMemDroppedKv := make(map[kvcache.Key]kvcache.Value)
lru, err := NewLRUPlanCache(3, 0, 0, pickFromBucket)
lru := NewLRUPlanCache(3, 0, 0, pickFromBucket)
lru.onEvict = func(key kvcache.Key, value kvcache.Value) {
maxMemDroppedKv[key] = value
}
require.NoError(t, err)
require.Equal(t, uint(3), lru.capacity)

keys := make([]*mockCacheKey, 5)
Expand Down Expand Up @@ -147,8 +145,7 @@ func TestLRUPCPut(t *testing.T) {
}

func TestLRUPCGet(t *testing.T) {
lru, err := NewLRUPlanCache(3, 0, 0, pickFromBucket)
require.NoError(t, err)
lru := NewLRUPlanCache(3, 0, 0, pickFromBucket)

keys := make([]*mockCacheKey, 5)
vals := make([]*fakePlan, 5)
Expand Down Expand Up @@ -197,8 +194,7 @@ func TestLRUPCGet(t *testing.T) {
}

func TestLRUPCDelete(t *testing.T) {
lru, err := NewLRUPlanCache(3, 0, 0, pickFromBucket)
require.NoError(t, err)
lru := NewLRUPlanCache(3, 0, 0, pickFromBucket)

keys := make([]*mockCacheKey, 3)
vals := make([]*fakePlan, 3)
Expand Down Expand Up @@ -230,8 +226,7 @@ func TestLRUPCDelete(t *testing.T) {
}

func TestLRUPCDeleteAll(t *testing.T) {
lru, err := NewLRUPlanCache(3, 0, 0, pickFromBucket)
require.NoError(t, err)
lru := NewLRUPlanCache(3, 0, 0, pickFromBucket)

keys := make([]*mockCacheKey, 3)
vals := make([]*fakePlan, 3)
Expand Down Expand Up @@ -261,11 +256,10 @@ func TestLRUPCDeleteAll(t *testing.T) {

func TestLRUPCSetCapacity(t *testing.T) {
maxMemDroppedKv := make(map[kvcache.Key]kvcache.Value)
lru, err := NewLRUPlanCache(5, 0, 0, pickFromBucket)
lru := NewLRUPlanCache(5, 0, 0, pickFromBucket)
lru.onEvict = func(key kvcache.Key, value kvcache.Value) {
maxMemDroppedKv[key] = value
}
require.NoError(t, err)
require.Equal(t, uint(5), lru.capacity)

keys := make([]*mockCacheKey, 5)
Expand All @@ -289,7 +283,7 @@ func TestLRUPCSetCapacity(t *testing.T) {
require.Equal(t, lru.size, lru.capacity)
require.Equal(t, uint(5), lru.size)

err = lru.SetCapacity(3)
err := lru.SetCapacity(3)
require.NoError(t, err)

// test for non-existent elements
Expand Down Expand Up @@ -323,5 +317,5 @@ func TestLRUPCSetCapacity(t *testing.T) {
require.Nil(t, root)

err = lru.SetCapacity(0)
require.Error(t, err, "capacity of lru cache should be at least 1")
require.Error(t, err, "capacity of LRU cache should be at least 1")
}
12 changes: 12 additions & 0 deletions planner/core/plan_cache_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@ import (
plannercore "github.com/pingcap/tidb/planner/core"
"github.com/pingcap/tidb/testkit"
"github.com/pingcap/tidb/types"
"github.com/stretchr/testify/require"
)

type mockParameterizer struct {
Expand Down Expand Up @@ -95,3 +96,14 @@ func TestGeneralPlanCacheParameterizer(t *testing.T) {
tk.MustQuery("select * from t where a > 2 and a < 5").Sort().Check(testkit.Rows("3", "4"))
tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1"))
}

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)
}
31 changes: 5 additions & 26 deletions planner/core/plan_cache_utils.go
Original file line number Diff line number Diff line change
Expand Up @@ -159,38 +159,17 @@ func GeneratePlanCacheStmtWithAST(ctx context.Context, sctx sessionctx.Context,

func getValidPlanFromCache(sctx sessionctx.Context, isGeneralPlanCache bool, key kvcache.Key, paramTypes []*types.FieldType) (*PlanCacheValue, bool) {
cache := sctx.GetPlanCache(isGeneralPlanCache)
val, exist := cache.Get(key)
val, exist := cache.Get(key, paramTypes)
if !exist {
return nil, exist
}
candidates := val.([]*PlanCacheValue)
for _, candidate := range candidates {
if candidate.varTypesUnchanged(paramTypes) {
return candidate, true
}
}
return nil, false
candidate := val.(*PlanCacheValue)
return candidate, true
}

func putPlanIntoCache(sctx sessionctx.Context, isGeneralPlanCache bool, key kvcache.Key, plan *PlanCacheValue) {
func putPlanIntoCache(sctx sessionctx.Context, isGeneralPlanCache bool, key kvcache.Key, plan *PlanCacheValue, paramTypes []*types.FieldType) {
cache := sctx.GetPlanCache(isGeneralPlanCache)
val, exist := cache.Get(key)
if !exist {
cache.Put(key, []*PlanCacheValue{plan})
return
}
candidates := val.([]*PlanCacheValue)
for i, candidate := range candidates {
if candidate.varTypesUnchanged(plan.ParamTypes) {
// hit an existing cached plan
candidates[i] = plan
return
}
}
// add to current candidate list
// TODO: limit the candidate list length
candidates = append(candidates, plan)
cache.Put(key, candidates)
cache.Put(key, plan, paramTypes)
}

// planCacheKey is used to access Plan Cache. We put some variables that do not affect the plan into planCacheKey, such as the sql text.
Expand Down
3 changes: 1 addition & 2 deletions planner/core/prepare_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -37,7 +37,6 @@ import (
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/testkit"
"github.com/pingcap/tidb/util/hint"
"github.com/pingcap/tidb/util/kvcache"
"github.com/prometheus/client_golang/prometheus"
dto "github.com/prometheus/client_model/go"
"github.com/stretchr/testify/require"
Expand Down Expand Up @@ -1340,7 +1339,7 @@ func TestPlanCacheSwitchDB(t *testing.T) {

// DB is not specified
se2, err := session.CreateSession4TestWithOpt(store, &session.Opt{
PreparedPlanCache: kvcache.NewSimpleLRUCache(100, 0.1, math.MaxUint64),
PreparedPlanCache: core.NewLRUPlanCache(100, 0.1, math.MaxUint64, core.PickPlanFromBucket),
})
require.NoError(t, err)
tk2 := testkit.NewTestKitWithSession(t, store, se2)
Expand Down
18 changes: 10 additions & 8 deletions session/session.go
Original file line number Diff line number Diff line change
Expand Up @@ -234,8 +234,8 @@ type session struct {

store kv.Storage

preparedPlanCache *kvcache.SimpleLRUCache
generalPlanCache *kvcache.SimpleLRUCache
preparedPlanCache sessionctx.PlanCache
generalPlanCache sessionctx.PlanCache

sessionVars *variable.SessionVars
sessionManager util.SessionManager
Expand Down Expand Up @@ -434,14 +434,15 @@ func (s *session) SetCollation(coID int) error {
return s.sessionVars.SetSystemVarWithoutValidation(variable.CollationConnection, co)
}

func (s *session) GetPlanCache(isGeneralPlanCache bool) *kvcache.SimpleLRUCache {
func (s *session) GetPlanCache(isGeneralPlanCache bool) sessionctx.PlanCache {
if isGeneralPlanCache { // use the general plan cache
if !s.GetSessionVars().EnableGeneralPlanCache {
return nil
}
if s.generalPlanCache == nil { // lazy construction
s.generalPlanCache = kvcache.NewSimpleLRUCache(uint(s.GetSessionVars().GeneralPlanCacheSize),
variable.PreparedPlanCacheMemoryGuardRatio.Load(), plannercore.PreparedPlanCacheMaxMemory.Load())
s.generalPlanCache = plannercore.NewLRUPlanCache(uint(s.GetSessionVars().GeneralPlanCacheSize),
variable.PreparedPlanCacheMemoryGuardRatio.Load(), plannercore.PreparedPlanCacheMaxMemory.Load(),
plannercore.PickPlanFromBucket)
}
return s.generalPlanCache
}
Expand All @@ -451,8 +452,9 @@ func (s *session) GetPlanCache(isGeneralPlanCache bool) *kvcache.SimpleLRUCache
return nil
}
if s.preparedPlanCache == nil { // lazy construction
s.preparedPlanCache = kvcache.NewSimpleLRUCache(uint(s.GetSessionVars().PreparedPlanCacheSize),
variable.PreparedPlanCacheMemoryGuardRatio.Load(), plannercore.PreparedPlanCacheMaxMemory.Load())
s.preparedPlanCache = plannercore.NewLRUPlanCache(uint(s.GetSessionVars().PreparedPlanCacheSize),
variable.PreparedPlanCacheMemoryGuardRatio.Load(), plannercore.PreparedPlanCacheMaxMemory.Load(),
plannercore.PickPlanFromBucket)
}
return s.preparedPlanCache
}
Expand Down Expand Up @@ -2545,7 +2547,7 @@ func CreateSession4Test(store kv.Storage) (Session, error) {

// Opt describes the option for creating session
type Opt struct {
PreparedPlanCache *kvcache.SimpleLRUCache
PreparedPlanCache sessionctx.PlanCache
}

// CreateSession4TestWithOpt creates a new session environment for test.
Expand Down
Loading

0 comments on commit 85cd17f

Please sign in to comment.