Skip to content

Commit

Permalink
planner: non-prep and prep statements use the same unified plan cache (
Browse files Browse the repository at this point in the history
  • Loading branch information
qw4990 authored Apr 17, 2023
1 parent 9cf0ed8 commit b1e6fe7
Show file tree
Hide file tree
Showing 10 changed files with 59 additions and 87 deletions.
2 changes: 1 addition & 1 deletion executor/prepared.go
Original file line number Diff line number Diff line change
Expand Up @@ -211,7 +211,7 @@ func (e *DeallocateExec) Next(ctx context.Context, req *chunk.Chunk) error {
return err
}
if !vars.IgnorePreparedCacheCloseStmt { // keep the plan in cache
e.ctx.GetPlanCache(false).Delete(cacheKey)
e.ctx.GetSessionPlanCache().Delete(cacheKey)
}
}
vars.RemovePreparedStmt(id)
Expand Down
12 changes: 6 additions & 6 deletions executor/seqtest/prepared_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -610,7 +610,7 @@ func TestPrepareDealloc(t *testing.T) {
tk.MustExec("drop table if exists prepare_test")
tk.MustExec("create table prepare_test (id int PRIMARY KEY, c1 int)")

require.Equal(t, 0, tk.Session().GetPlanCache(false).Size())
require.Equal(t, 0, tk.Session().GetSessionPlanCache().Size())
tk.MustExec(`prepare stmt1 from 'select id from prepare_test'`)
tk.MustExec("execute stmt1")
tk.MustExec(`prepare stmt2 from 'select c1 from prepare_test'`)
Expand All @@ -619,28 +619,28 @@ func TestPrepareDealloc(t *testing.T) {
tk.MustExec("execute stmt3")
tk.MustExec(`prepare stmt4 from 'select * from prepare_test'`)
tk.MustExec("execute stmt4")
require.Equal(t, 3, tk.Session().GetPlanCache(false).Size())
require.Equal(t, 3, tk.Session().GetSessionPlanCache().Size())

tk.MustExec("deallocate prepare stmt1")
require.Equal(t, 3, tk.Session().GetPlanCache(false).Size())
require.Equal(t, 3, tk.Session().GetSessionPlanCache().Size())
tk.MustExec("deallocate prepare stmt2")
tk.MustExec("deallocate prepare stmt3")
tk.MustExec("deallocate prepare stmt4")
require.Equal(t, 0, tk.Session().GetPlanCache(false).Size())
require.Equal(t, 0, tk.Session().GetSessionPlanCache().Size())

tk.MustExec(`prepare stmt1 from 'select * from prepare_test'`)
tk.MustExec(`execute stmt1`)
tk.MustExec(`prepare stmt2 from 'select * from prepare_test'`)
tk.MustExec(`execute stmt2`)
require.Equal(t, 1, tk.Session().GetPlanCache(false).Size()) // use the same cached plan since they have the same statement
require.Equal(t, 1, tk.Session().GetSessionPlanCache().Size()) // use the same cached plan since they have the same statement

tk.MustExec(`drop database if exists plan_cache`)
tk.MustExec(`create database plan_cache`)
tk.MustExec(`use plan_cache`)
tk.MustExec(`create table prepare_test (id int PRIMARY KEY, c1 int)`)
tk.MustExec(`prepare stmt3 from 'select * from prepare_test'`)
tk.MustExec(`execute stmt3`)
require.Equal(t, 2, tk.Session().GetPlanCache(false).Size()) // stmt3 has different DB
require.Equal(t, 2, tk.Session().GetSessionPlanCache().Size()) // stmt3 has different DB
}

func TestPreparedIssue8153(t *testing.T) {
Expand Down
2 changes: 1 addition & 1 deletion executor/simple.go
Original file line number Diff line number Diff line change
Expand Up @@ -2828,7 +2828,7 @@ func (e *SimpleExec) executeAdminFlushPlanCache(s *ast.AdminStmt) error {
}
now := types.NewTime(types.FromGoTime(time.Now().In(e.ctx.GetSessionVars().StmtCtx.TimeZone)), mysql.TypeTimestamp, 3)
e.ctx.GetSessionVars().LastUpdateTime4PC = now
e.ctx.GetPlanCache(false).DeleteAll()
e.ctx.GetSessionPlanCache().DeleteAll()
if s.StatementScope == ast.StatementScopeInstance {
// Record the timestamp. When other sessions want to use the plan cache,
// it will check the timestamp first to decide whether the plan cache should be flushed.
Expand Down
40 changes: 15 additions & 25 deletions planner/core/metrics/metrics.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,17 +21,15 @@ import (

// planner core metrics vars
var (
PseudoEstimationNotAvailable prometheus.Counter
PseudoEstimationOutdate prometheus.Counter
preparedPlanCacheHitCounter prometheus.Counter
nonPreparedPlanCacheHitCounter prometheus.Counter
preparedPlanCacheMissCounter prometheus.Counter
nonPreparedPlanCacheMissCounter prometheus.Counter
nonPreparedPlanCacheUnsupportedCounter prometheus.Counter
preparedPlanCacheInstancePlanNumCounter prometheus.Gauge
nonPreparedPlanCacheInstancePlanNumCounter prometheus.Gauge
preparedPlanCacheInstanceMemoryUsage prometheus.Gauge
nonPreparedPlanCacheInstanceMemoryUsage prometheus.Gauge
PseudoEstimationNotAvailable prometheus.Counter
PseudoEstimationOutdate prometheus.Counter
preparedPlanCacheHitCounter prometheus.Counter
nonPreparedPlanCacheHitCounter prometheus.Counter
preparedPlanCacheMissCounter prometheus.Counter
nonPreparedPlanCacheMissCounter prometheus.Counter
nonPreparedPlanCacheUnsupportedCounter prometheus.Counter
sessionPlanCacheInstancePlanNumCounter prometheus.Gauge
sessionPlanCacheInstanceMemoryUsage prometheus.Gauge
)

func init() {
Expand All @@ -48,10 +46,8 @@ func InitMetricsVars() {
preparedPlanCacheMissCounter = metrics.PlanCacheMissCounter.WithLabelValues("prepared")
nonPreparedPlanCacheMissCounter = metrics.PlanCacheMissCounter.WithLabelValues("non-prepared")
nonPreparedPlanCacheUnsupportedCounter = metrics.PlanCacheMissCounter.WithLabelValues("non-prepared-unsupported")
preparedPlanCacheInstancePlanNumCounter = metrics.PlanCacheInstancePlanNumCounter.WithLabelValues(" prepared")
nonPreparedPlanCacheInstancePlanNumCounter = metrics.PlanCacheInstancePlanNumCounter.WithLabelValues(" non-prepared")
preparedPlanCacheInstanceMemoryUsage = metrics.PlanCacheInstanceMemoryUsage.WithLabelValues(" prepared")
nonPreparedPlanCacheInstanceMemoryUsage = metrics.PlanCacheInstanceMemoryUsage.WithLabelValues(" non-prepared")
sessionPlanCacheInstancePlanNumCounter = metrics.PlanCacheInstancePlanNumCounter.WithLabelValues(" session-plan-cache")
sessionPlanCacheInstanceMemoryUsage = metrics.PlanCacheInstanceMemoryUsage.WithLabelValues(" session-plan-cache")
}

// GetPlanCacheHitCounter get different plan cache hit counter
Expand All @@ -76,17 +72,11 @@ func GetNonPrepPlanCacheUnsupportedCounter() prometheus.Counter {
}

// GetPlanCacheInstanceNumCounter get different plan counter of plan cache
func GetPlanCacheInstanceNumCounter(isNonPrepared bool) prometheus.Gauge {
if isNonPrepared {
return nonPreparedPlanCacheInstancePlanNumCounter
}
return preparedPlanCacheInstancePlanNumCounter
func GetPlanCacheInstanceNumCounter() prometheus.Gauge {
return sessionPlanCacheInstancePlanNumCounter
}

// GetPlanCacheInstanceMemoryUsage get different plan memory usage counter of plan cache
func GetPlanCacheInstanceMemoryUsage(isNonPrepared bool) prometheus.Gauge {
if isNonPrepared {
return nonPreparedPlanCacheInstanceMemoryUsage
}
return preparedPlanCacheInstanceMemoryUsage
func GetPlanCacheInstanceMemoryUsage() prometheus.Gauge {
return sessionPlanCacheInstanceMemoryUsage
}
8 changes: 4 additions & 4 deletions planner/core/plan_cache.go
Original file line number Diff line number Diff line change
Expand Up @@ -114,7 +114,7 @@ func planCachePreprocess(ctx context.Context, sctx sessionctx.Context, isNonPrep
// And update lastUpdateTime to the newest one.
expiredTimeStamp4PC := domain.GetDomain(sctx).ExpiredTimeStamp4PC()
if stmt.StmtCacheable && expiredTimeStamp4PC.Compare(vars.LastUpdateTime4PC) > 0 {
sctx.GetPlanCache(isNonPrepared).DeleteAll()
sctx.GetSessionPlanCache().DeleteAll()
stmtAst.CachedPlan = nil
vars.LastUpdateTime4PC = expiredTimeStamp4PC
}
Expand Down Expand Up @@ -239,7 +239,7 @@ func getCachedPlan(sctx sessionctx.Context, isNonPrepared bool, cacheKey kvcache
sessVars := sctx.GetSessionVars()
stmtCtx := sessVars.StmtCtx

candidate, exist := sctx.GetPlanCache(isNonPrepared).Get(cacheKey, matchOpts)
candidate, exist := sctx.GetSessionPlanCache().Get(cacheKey, matchOpts)
if !exist {
return nil, nil, false, nil
}
Expand All @@ -251,7 +251,7 @@ func getCachedPlan(sctx sessionctx.Context, isNonPrepared bool, cacheKey kvcache
if !unionScan && tableHasDirtyContent(sctx, tblInfo) {
// TODO we can inject UnionScan into cached plan to avoid invalidating it, though
// rebuilding the filters in UnionScan is pretty trivial.
sctx.GetPlanCache(isNonPrepared).Delete(cacheKey)
sctx.GetSessionPlanCache().Delete(cacheKey)
return nil, nil, false, nil
}
}
Expand Down Expand Up @@ -316,7 +316,7 @@ func generateNewPlan(ctx context.Context, sctx sessionctx.Context, isNonPrepared
stmt.NormalizedPlan, stmt.PlanDigest = NormalizePlan(p)
stmtCtx.SetPlan(p)
stmtCtx.SetPlanDigest(stmt.NormalizedPlan, stmt.PlanDigest)
sctx.GetPlanCache(isNonPrepared).Put(cacheKey, cached, matchOpts)
sctx.GetSessionPlanCache().Put(cacheKey, cached, matchOpts)
}
sessVars.FoundInPlanCache = false
return p, names, err
Expand Down
36 changes: 17 additions & 19 deletions planner/core/plan_cache_lru.go
Original file line number Diff line number Diff line change
Expand Up @@ -60,7 +60,6 @@ type LRUPlanCache struct {

memoryUsageTotal int64
sctx sessionctx.Context
isNonPrepared bool
}

// NewLRUPlanCache creates a PCLRUCache object, whose capacity is "capacity".
Expand All @@ -71,14 +70,13 @@ func NewLRUPlanCache(capacity uint, guard float64, quota uint64, sctx sessionctx
logutil.BgLogger().Info("capacity of LRU cache is less than 1, will use default value(100) init cache")
}
return &LRUPlanCache{
capacity: capacity,
size: 0,
buckets: make(map[string]map[*list.Element]struct{}, 1), //Generally one query has one plan
lruList: list.New(),
quota: quota,
guard: guard,
sctx: sctx,
isNonPrepared: isNonPrepared,
capacity: capacity,
size: 0,
buckets: make(map[string]map[*list.Element]struct{}, 1), //Generally one query has one plan
lruList: list.New(),
quota: quota,
guard: guard,
sctx: sctx,
}
}

Expand Down Expand Up @@ -204,9 +202,9 @@ func (l *LRUPlanCache) Close() {
return
}
if l.sctx.GetSessionVars().EnablePreparedPlanCacheMemoryMonitor {
core_metrics.GetPlanCacheInstanceMemoryUsage(l.isNonPrepared).Sub(float64(l.memoryUsageTotal))
core_metrics.GetPlanCacheInstanceMemoryUsage().Sub(float64(l.memoryUsageTotal))
}
core_metrics.GetPlanCacheInstanceNumCounter(l.isNonPrepared).Sub(float64(l.size))
core_metrics.GetPlanCacheInstanceNumCounter().Sub(float64(l.size))
}

// removeOldest removes the oldest element from the cache.
Expand Down Expand Up @@ -302,31 +300,31 @@ func checkUint64SliceIfEqual(a, b []uint64) bool {

// updateInstanceMetric update the memory usage and plan num for show in grafana
func (l *LRUPlanCache) updateInstanceMetric(in, out *planCacheEntry) {
updateInstancePlanNum(in, out, l.isNonPrepared)
updateInstancePlanNum(in, out)
if l == nil || !l.sctx.GetSessionVars().EnablePreparedPlanCacheMemoryMonitor {
return
}

if in != nil && out != nil { // replace plan
core_metrics.GetPlanCacheInstanceMemoryUsage(l.isNonPrepared).Sub(float64(out.MemoryUsage()))
core_metrics.GetPlanCacheInstanceMemoryUsage(l.isNonPrepared).Add(float64(in.MemoryUsage()))
core_metrics.GetPlanCacheInstanceMemoryUsage().Sub(float64(out.MemoryUsage()))
core_metrics.GetPlanCacheInstanceMemoryUsage().Add(float64(in.MemoryUsage()))
l.memoryUsageTotal += in.MemoryUsage() - out.MemoryUsage()
} else if in != nil { // put plan
core_metrics.GetPlanCacheInstanceMemoryUsage(l.isNonPrepared).Add(float64(in.MemoryUsage()))
core_metrics.GetPlanCacheInstanceMemoryUsage().Add(float64(in.MemoryUsage()))
l.memoryUsageTotal += in.MemoryUsage()
} else { // delete plan
core_metrics.GetPlanCacheInstanceMemoryUsage(l.isNonPrepared).Sub(float64(out.MemoryUsage()))
core_metrics.GetPlanCacheInstanceMemoryUsage().Sub(float64(out.MemoryUsage()))
l.memoryUsageTotal -= out.MemoryUsage()
}
}

// updateInstancePlanNum update the plan num
func updateInstancePlanNum(in, out *planCacheEntry, isNonPrepared bool) {
func updateInstancePlanNum(in, out *planCacheEntry) {
if in != nil && out != nil { // replace plan
return
} else if in != nil { // put plan
core_metrics.GetPlanCacheInstanceNumCounter(isNonPrepared).Add(1)
core_metrics.GetPlanCacheInstanceNumCounter().Add(1)
} else { // delete plan
core_metrics.GetPlanCacheInstanceNumCounter(isNonPrepared).Sub(1)
core_metrics.GetPlanCacheInstanceNumCounter().Sub(1)
}
}
2 changes: 1 addition & 1 deletion server/driver_tidb.go
Original file line number Diff line number Diff line change
Expand Up @@ -177,7 +177,7 @@ func (ts *TiDBStatement) Close() error {
return err
}
if !ts.ctx.GetSessionVars().IgnorePreparedCacheCloseStmt { // keep the plan in cache
ts.ctx.GetPlanCache(false).Delete(cacheKey)
ts.ctx.GetSessionPlanCache().Delete(cacheKey)
}
}
ts.ctx.GetSessionVars().RemovePreparedStmt(ts.id)
Expand Down
35 changes: 10 additions & 25 deletions session/session.go
Original file line number Diff line number Diff line change
Expand Up @@ -221,8 +221,7 @@ type session struct {

store kv.Storage

preparedPlanCache sessionctx.PlanCache
nonPreparedPlanCache sessionctx.PlanCache
sessionPlanCache sessionctx.PlanCache

sessionVars *variable.SessionVars
sessionManager util.SessionManager
Expand Down Expand Up @@ -366,7 +365,7 @@ func (s *session) cleanRetryInfo() {
plannercore.SetPstmtIDSchemaVersion(cacheKey, stmtText, preparedAst.SchemaVersion, s.sessionVars.IsolationReadEngines)
}
if !s.sessionVars.IgnorePreparedCacheCloseStmt { // keep the plan in cache
s.GetPlanCache(false).Delete(cacheKey)
s.GetSessionPlanCache().Delete(cacheKey)
}
}
s.sessionVars.RemovePreparedStmt(stmtID)
Expand Down Expand Up @@ -425,27 +424,16 @@ func (s *session) SetCollation(coID int) error {
return s.sessionVars.SetSystemVarWithoutValidation(variable.CollationConnection, co)
}

func (s *session) GetPlanCache(isNonPrepared bool) sessionctx.PlanCache {
if isNonPrepared { // use the non-prepared plan cache
if !s.GetSessionVars().EnableNonPreparedPlanCache {
return nil
}
if s.nonPreparedPlanCache == nil { // lazy construction
s.nonPreparedPlanCache = plannercore.NewLRUPlanCache(uint(s.GetSessionVars().NonPreparedPlanCacheSize),
variable.PreparedPlanCacheMemoryGuardRatio.Load(), plannercore.PreparedPlanCacheMaxMemory.Load(), s, true)
}
return s.nonPreparedPlanCache
}

func (s *session) GetSessionPlanCache() sessionctx.PlanCache {
// use the prepared plan cache
if !s.GetSessionVars().EnablePreparedPlanCache {
if !s.GetSessionVars().EnablePreparedPlanCache && !s.GetSessionVars().EnableNonPreparedPlanCache {
return nil
}
if s.preparedPlanCache == nil { // lazy construction
s.preparedPlanCache = plannercore.NewLRUPlanCache(uint(s.GetSessionVars().PreparedPlanCacheSize),
if s.sessionPlanCache == nil { // lazy construction
s.sessionPlanCache = plannercore.NewLRUPlanCache(uint(s.GetSessionVars().PreparedPlanCacheSize),
variable.PreparedPlanCacheMemoryGuardRatio.Load(), plannercore.PreparedPlanCacheMaxMemory.Load(), s, false)
}
return s.preparedPlanCache
return s.sessionPlanCache
}

func (s *session) SetSessionManager(sm util.SessionManager) {
Expand Down Expand Up @@ -2586,11 +2574,8 @@ func (s *session) Close() {
s.stmtStats.SetFinished()
}
s.ClearDiskFullOpt()
if s.preparedPlanCache != nil {
s.preparedPlanCache.Close()
}
if s.nonPreparedPlanCache != nil {
s.nonPreparedPlanCache.Close()
if s.sessionPlanCache != nil {
s.sessionPlanCache.Close()
}
}

Expand Down Expand Up @@ -3557,7 +3542,7 @@ func createSessionWithOpt(store kv.Storage, opt *Opt) (*session, error) {

s.functionUsageMu.builtinFunctionUsage = make(telemetry.BuiltinFunctionsUsage)
if opt != nil && opt.PreparedPlanCache != nil {
s.preparedPlanCache = opt.PreparedPlanCache
s.sessionPlanCache = opt.PreparedPlanCache
}
s.mu.values = make(map[fmt.Stringer]interface{})
s.lockedTables = make(map[int64]model.TableLockTpInfo)
Expand Down
5 changes: 2 additions & 3 deletions sessionctx/context.go
Original file line number Diff line number Diff line change
Expand Up @@ -119,9 +119,8 @@ type Context interface {
// GetStore returns the store of session.
GetStore() kv.Storage

// GetPlanCache returns the cache of the physical plan.
// isNonPrepared indicates to return the non-prepared plan cache or the prepared plan cache.
GetPlanCache(isNonPrepared bool) PlanCache
// GetSessionPlanCache returns the session-level cache of the physical plan.
GetSessionPlanCache() PlanCache

// StoreQueryFeedback stores the query feedback.
StoreQueryFeedback(feedback interface{})
Expand Down
4 changes: 2 additions & 2 deletions util/mock/context.go
Original file line number Diff line number Diff line change
Expand Up @@ -248,8 +248,8 @@ func (*Context) SetGlobalSysVar(_ sessionctx.Context, name string, value string)
return nil
}

// GetPlanCache implements the sessionctx.Context interface.
func (c *Context) GetPlanCache(_ bool) sessionctx.PlanCache {
// GetSessionPlanCache implements the sessionctx.Context interface.
func (c *Context) GetSessionPlanCache() sessionctx.PlanCache {
return c.pcache
}

Expand Down

0 comments on commit b1e6fe7

Please sign in to comment.