diff --git a/executor/seqtest/prepared_test.go b/executor/seqtest/prepared_test.go index 009dbb76e6a3a..69f2fcf8d0564 100644 --- a/executor/seqtest/prepared_test.go +++ b/executor/seqtest/prepared_test.go @@ -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" ) @@ -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) @@ -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) diff --git a/expression/integration_serial_test.go b/expression/integration_serial_test.go index d2956744ff862..67cc805ff820c 100644 --- a/expression/integration_serial_test.go +++ b/expression/integration_serial_test.go @@ -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" ) @@ -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) diff --git a/infoschema/tables_test.go b/infoschema/tables_test.go index c75c03f1af71b..e85628f8e6b82 100644 --- a/infoschema/tables_test.go +++ b/infoschema/tables_test.go @@ -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" ) @@ -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() diff --git a/planner/core/plan_cache.go b/planner/core/plan_cache.go index 5a656e1a8ef32..42badb0fcd5dd 100644 --- a/planner/core/plan_cache.go +++ b/planner/core/plan_cache.go @@ -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 } @@ -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 diff --git a/planner/core/plan_cache_lru.go b/planner/core/plan_cache_lru.go index 5c369f7bb97a0..2f3ab11a5c57b 100644 --- a/planner/core/plan_cache_lru.go +++ b/planner/core/plan_cache_lru.go @@ -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" ) @@ -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 @@ -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, @@ -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. @@ -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 { @@ -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 +} diff --git a/planner/core/plan_cache_lru_test.go b/planner/core/plan_cache_lru_test.go index 2cc8a79147916..c1a02c5489b9f 100644 --- a/planner/core/plan_cache_lru_test.go +++ b/planner/core/plan_cache_lru_test.go @@ -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) @@ -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) @@ -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) @@ -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) @@ -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) @@ -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 @@ -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") } diff --git a/planner/core/plan_cache_test.go b/planner/core/plan_cache_test.go index ad481a49adc61..919e2102ebea9 100644 --- a/planner/core/plan_cache_test.go +++ b/planner/core/plan_cache_test.go @@ -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 { @@ -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) +} diff --git a/planner/core/plan_cache_utils.go b/planner/core/plan_cache_utils.go index 2d2bf2385dc71..1dcf00288e710 100644 --- a/planner/core/plan_cache_utils.go +++ b/planner/core/plan_cache_utils.go @@ -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. diff --git a/planner/core/prepare_test.go b/planner/core/prepare_test.go index bb0c8349a5ff5..32ac16e7164f8 100644 --- a/planner/core/prepare_test.go +++ b/planner/core/prepare_test.go @@ -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" @@ -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) diff --git a/session/session.go b/session/session.go index b00ab8c35e336..8d6ef19bc95f7 100644 --- a/session/session.go +++ b/session/session.go @@ -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 @@ -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 } @@ -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 } @@ -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. diff --git a/sessionctx/context.go b/sessionctx/context.go index 7ab28480eb867..7be92f102e56c 100644 --- a/sessionctx/context.go +++ b/sessionctx/context.go @@ -26,6 +26,7 @@ import ( "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/sessionctx/sessionstates" "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/kvcache" "github.com/pingcap/tidb/util/sli" @@ -50,6 +51,16 @@ type SessionStatesHandler interface { DecodeSessionStates(context.Context, Context, *sessionstates.SessionStates) error } +// PlanCache is an interface for prepare and general plan cache +type PlanCache interface { + Get(key kvcache.Key, paramTypes []*types.FieldType) (value kvcache.Value, ok bool) + Put(key kvcache.Key, value kvcache.Value, paramTypes []*types.FieldType) + Delete(key kvcache.Key) + DeleteAll() + Size() int + SetCapacity(capacity uint) error +} + // Context is an interface for transaction and executive args environment. type Context interface { SessionStatesHandler @@ -108,7 +119,7 @@ type Context interface { // GetPlanCache returns the cache of the physical plan. // generalPlanCache indicates to return the general plan cache or the prepared plan cache. - GetPlanCache(isGeneralPlanCache bool) *kvcache.SimpleLRUCache + GetPlanCache(isGeneralPlanCache bool) PlanCache // StoreQueryFeedback stores the query feedback. StoreQueryFeedback(feedback interface{}) diff --git a/util/mock/context.go b/util/mock/context.go index 53b15d47ecfcd..58ad6e22a32d3 100644 --- a/util/mock/context.go +++ b/util/mock/context.go @@ -31,7 +31,6 @@ import ( "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/disk" - "github.com/pingcap/tidb/util/kvcache" "github.com/pingcap/tidb/util/memory" "github.com/pingcap/tidb/util/sli" "github.com/pingcap/tidb/util/sqlexec" @@ -56,7 +55,7 @@ type Context struct { values map[fmt.Stringer]interface{} sessionVars *variable.SessionVars cancel context.CancelFunc - pcache *kvcache.SimpleLRUCache + pcache sessionctx.PlanCache level kvrpcpb.DiskFullOpt } @@ -248,7 +247,7 @@ func (*Context) SetGlobalSysVar(_ sessionctx.Context, name string, value string) } // GetPlanCache implements the sessionctx.Context interface. -func (c *Context) GetPlanCache(_ bool) *kvcache.SimpleLRUCache { +func (c *Context) GetPlanCache(_ bool) sessionctx.PlanCache { return c.pcache }