Skip to content

Commit

Permalink
Merge branch 'master' into enable_race_in_bazel
Browse files Browse the repository at this point in the history
  • Loading branch information
hawkingrei authored Aug 16, 2022
2 parents ca8f3a0 + 0039dd2 commit 53bb5d7
Show file tree
Hide file tree
Showing 7 changed files with 130 additions and 65 deletions.
30 changes: 30 additions & 0 deletions executor/set_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -755,6 +755,36 @@ func TestSetVar(t *testing.T) {
tk.MustGetErrCode("set global init_connect = '-1'", mysql.ErrWrongTypeForVar)
tk.MustGetErrCode("set global init_connect = 'invalidstring'", mysql.ErrWrongTypeForVar)
tk.MustExec("set global init_connect = 'select now(); select timestamp()'")

// test variable 'tidb_enable_general_plan_cache'
// global scope
tk.MustQuery("select @@global.tidb_enable_general_plan_cache").Check(testkit.Rows("0")) // default value
tk.MustExec("set global tidb_enable_general_plan_cache = 1")
tk.MustQuery("select @@global.tidb_enable_general_plan_cache").Check(testkit.Rows("1"))
tk.MustExec("set global tidb_enable_general_plan_cache = 0")
tk.MustQuery("select @@global.tidb_enable_general_plan_cache").Check(testkit.Rows("0"))
// session scope
tk.MustQuery("select @@session.tidb_enable_general_plan_cache").Check(testkit.Rows("0")) // default value
tk.MustExec("set session tidb_enable_general_plan_cache = 1")
tk.MustQuery("select @@session.tidb_enable_general_plan_cache").Check(testkit.Rows("1"))
tk.MustExec("set session tidb_enable_general_plan_cache = 0")
tk.MustQuery("select @@session.tidb_enable_general_plan_cache").Check(testkit.Rows("0"))

// test variable 'tidb_general_plan_cache-size'
// global scope
tk.MustQuery("select @@global.tidb_general_plan_cache_size").Check(testkit.Rows("100")) // default value
tk.MustExec("set global tidb_general_plan_cache_size = 200")
tk.MustQuery("select @@global.tidb_general_plan_cache_size").Check(testkit.Rows("200"))
tk.MustExec("set global tidb_general_plan_cache_size = 200000000") // overflow
tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1292 Truncated incorrect tidb_general_plan_cache_size value: '200000000'"))
tk.MustQuery("select @@global.tidb_general_plan_cache_size").Check(testkit.Rows("100000"))
// session scope
tk.MustQuery("select @@session.tidb_general_plan_cache_size").Check(testkit.Rows("100")) // default value
tk.MustExec("set session tidb_general_plan_cache_size = 300")
tk.MustQuery("select @@session.tidb_general_plan_cache_size").Check(testkit.Rows("300"))
tk.MustExec("set session tidb_general_plan_cache_size = -1") // underflow
tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1292 Truncated incorrect tidb_general_plan_cache_size value: '-1'"))
tk.MustQuery("select @@session.tidb_general_plan_cache_size").Check(testkit.Rows("0"))
}

func TestGetSetNoopVars(t *testing.T) {
Expand Down
89 changes: 32 additions & 57 deletions planner/core/plan_cache.go
Original file line number Diff line number Diff line change
Expand Up @@ -214,49 +214,39 @@ func getGeneralPlan(sctx sessionctx.Context, cacheKey kvcache.Key, bindSQL strin
sessVars := sctx.GetSessionVars()
stmtCtx := sessVars.StmtCtx

if cacheValue, exists := sctx.PreparedPlanCache().Get(cacheKey); exists {
if err := CheckPreparedPriv(sctx, stmt, is); err != nil {
return nil, nil, false, err
}
cachedVals := cacheValue.([]*PlanCacheValue)
for _, cachedVal := range cachedVals {
if !cachedVal.varTypesUnchanged(paramTypes) {
continue
}
planValid := true
for tblInfo, unionScan := range cachedVal.TblInfo2UnionScan {
if !unionScan && tableHasDirtyContent(sctx, tblInfo) {
planValid = false
// TODO we can inject UnionScan into cached plan to avoid invalidating it, though
// rebuilding the filters in UnionScan is pretty trivial.
sctx.PreparedPlanCache().Delete(cacheKey)
break
}
}
if planValid {
err := RebuildPlan4CachedPlan(cachedVal.Plan)
if err != nil {
logutil.BgLogger().Debug("rebuild range failed", zap.Error(err))
return nil, nil, false, nil
}
sessVars.FoundInPlanCache = true
if len(bindSQL) > 0 {
// When the `len(bindSQL) > 0`, it means we use the binding.
// So we need to record this.
sessVars.FoundInBinding = true
}
if metrics.ResettablePlanCacheCounterFortTest {
metrics.PlanCacheCounter.WithLabelValues("prepare").Inc()
} else {
planCacheCounter.Inc()
}
stmtCtx.SetPlanDigest(stmt.NormalizedPlan, stmt.PlanDigest)
return cachedVal.Plan, cachedVal.OutPutNames, true, nil
}
break
cachedVal, exist := getValidPlanFromCache(sctx, cacheKey, paramTypes)
if !exist {
return nil, nil, false, nil
}
if err := CheckPreparedPriv(sctx, stmt, is); err != nil {
return nil, nil, false, err
}
for tblInfo, unionScan := range cachedVal.TblInfo2UnionScan {
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.PreparedPlanCache().Delete(cacheKey)
return nil, nil, false, nil
}
}
return nil, nil, false, nil
err := RebuildPlan4CachedPlan(cachedVal.Plan)
if err != nil {
logutil.BgLogger().Debug("rebuild range failed", zap.Error(err))
return nil, nil, false, nil
}
sessVars.FoundInPlanCache = true
if len(bindSQL) > 0 {
// When the `len(bindSQL) > 0`, it means we use the binding.
// So we need to record this.
sessVars.FoundInBinding = true
}
if metrics.ResettablePlanCacheCounterFortTest {
metrics.PlanCacheCounter.WithLabelValues("prepare").Inc()
} else {
planCacheCounter.Inc()
}
stmtCtx.SetPlanDigest(stmt.NormalizedPlan, stmt.PlanDigest)
return cachedVal.Plan, cachedVal.OutPutNames, true, nil
}

// generateNewPlan call the optimizer to generate a new plan for current statement
Expand Down Expand Up @@ -296,22 +286,7 @@ func generateNewPlan(ctx context.Context, sctx sessionctx.Context, is infoschema
stmt.NormalizedPlan, stmt.PlanDigest = NormalizePlan(p)
stmtCtx.SetPlan(p)
stmtCtx.SetPlanDigest(stmt.NormalizedPlan, stmt.PlanDigest)
if cacheVals, exists := sctx.PreparedPlanCache().Get(cacheKey); exists {
hitVal := false
for i, cacheVal := range cacheVals.([]*PlanCacheValue) {
if cacheVal.varTypesUnchanged(paramTypes) {
hitVal = true
cacheVals.([]*PlanCacheValue)[i] = cached
break
}
}
if !hitVal {
cacheVals = append(cacheVals.([]*PlanCacheValue), cached)
}
sctx.PreparedPlanCache().Put(cacheKey, cacheVals)
} else {
sctx.PreparedPlanCache().Put(cacheKey, []*PlanCacheValue{cached})
}
putPlanIntoCache(sctx, cacheKey, cached)
}
sessVars.FoundInPlanCache = false
return p, names, err
Expand Down
52 changes: 44 additions & 8 deletions planner/core/cache.go → planner/core/plan_cache_utils.go
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,42 @@ var (
PreparedPlanCacheMaxMemory = *atomic2.NewUint64(math.MaxUint64)
)

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

func putPlanIntoCache(sctx sessionctx.Context, key kvcache.Key, plan *PlanCacheValue) {
cache := sctx.PreparedPlanCache()
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)
}

// planCacheKey is used to access Plan Cache. We put some variables that do not affect the plan into planCacheKey, such as the sql text.
// Put the parameters that may affect the plan in planCacheValue.
// However, due to some compatibility reasons, we will temporarily keep some system variable-related values in planCacheKey.
Expand Down Expand Up @@ -157,7 +193,7 @@ func NewPlanCacheKey(sessionVars *variable.SessionVars, stmtText, stmtDB string,
}

// FieldSlice is the slice of the types.FieldType
type FieldSlice []types.FieldType
type FieldSlice []*types.FieldType

// CheckTypesCompatibility4PC compares FieldSlice with []*types.FieldType
// Currently this is only used in plan cache to check whether the types of parameters are compatible.
Expand Down Expand Up @@ -193,29 +229,29 @@ type PlanCacheValue struct {
Plan Plan
OutPutNames []*types.FieldName
TblInfo2UnionScan map[*model.TableInfo]bool
TxtVarTypes FieldSlice
ParamTypes FieldSlice
}

func (v *PlanCacheValue) varTypesUnchanged(txtVarTps []*types.FieldType) bool {
return v.TxtVarTypes.CheckTypesCompatibility4PC(txtVarTps)
return v.ParamTypes.CheckTypesCompatibility4PC(txtVarTps)
}

// NewPlanCacheValue creates a SQLCacheValue.
func NewPlanCacheValue(plan Plan, names []*types.FieldName, srcMap map[*model.TableInfo]bool,
txtVarTps []*types.FieldType) *PlanCacheValue {
paramTypes []*types.FieldType) *PlanCacheValue {
dstMap := make(map[*model.TableInfo]bool)
for k, v := range srcMap {
dstMap[k] = v
}
userVarTypes := make([]types.FieldType, len(txtVarTps))
for i, tp := range txtVarTps {
userVarTypes[i] = *tp
userParamTypes := make([]*types.FieldType, len(paramTypes))
for i, tp := range paramTypes {
userParamTypes[i] = tp.Clone()
}
return &PlanCacheValue{
Plan: plan,
OutPutNames: names,
TblInfo2UnionScan: dstMap,
TxtVarTypes: userVarTypes,
ParamTypes: userParamTypes,
}
}

Expand Down
File renamed without changes.
6 changes: 6 additions & 0 deletions sessionctx/variable/session.go
Original file line number Diff line number Diff line change
Expand Up @@ -1226,6 +1226,12 @@ type SessionVars struct {

// EnablePreparedPlanCache indicates whether to enable prepared plan cache.
EnablePreparedPlanCache bool

// EnableGeneralPlanCache indicates whether to enable general plan cache.
EnableGeneralPlanCache bool

// GeneralPlanCacheSize controls the size of general plan cache.
GeneralPlanCacheSize uint64
}

// GetPreparedStmtByName returns the prepared statement specified by stmtName.
Expand Down
11 changes: 11 additions & 0 deletions sessionctx/variable/sysvar.go
Original file line number Diff line number Diff line change
Expand Up @@ -796,6 +796,17 @@ var defaultSysVars = []*SysVar{
}, GetGlobal: func(s *SessionVars) (string, error) {
return strconv.FormatFloat(PreparedPlanCacheMemoryGuardRatio.Load(), 'f', -1, 64), nil
}},
{Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableGeneralPlanCache, Value: BoolToOnOff(DefTiDBEnableGeneralPlanCache), Type: TypeBool, SetSession: func(s *SessionVars, val string) error {
s.EnableGeneralPlanCache = TiDBOptOn(val)
return nil
}},
{Scope: ScopeGlobal | ScopeSession, Name: TiDBGeneralPlanCacheSize, Value: strconv.FormatUint(uint64(DefTiDBGeneralPlanCacheSize), 10), Type: TypeUnsigned, MinValue: 0, MaxValue: 100000, SetSession: func(s *SessionVars, val string) error {
uVal, err := strconv.ParseUint(val, 10, 64)
if err == nil {
s.GeneralPlanCacheSize = uVal
}
return err
}},
{Scope: ScopeGlobal, Name: TiDBMemOOMAction, Value: DefTiDBMemOOMAction, PossibleValues: []string{"CANCEL", "LOG"}, Type: TypeEnum,
GetGlobal: func(s *SessionVars) (string, error) {
return OOMAction.Load(), nil
Expand Down
7 changes: 7 additions & 0 deletions sessionctx/variable/tidb_vars.go
Original file line number Diff line number Diff line change
Expand Up @@ -717,6 +717,11 @@ const (
// When set to (0, 1], Selectivity() will use the value of this variable as the default selectivity of those
// functions instead of the selectionFactor (0.8).
TiDBDefaultStrMatchSelectivity = "tidb_default_string_match_selectivity"

// TiDBEnableGeneralPlanCache indicates whether to enable general plan cache.
TiDBEnableGeneralPlanCache = "tidb_enable_general_plan_cache"
// TiDBGeneralPlanCacheSize controls the size of general plan cache.
TiDBGeneralPlanCacheSize = "tidb_general_plan_cache_size"
)

// TiDB vars that have only global scope
Expand Down Expand Up @@ -1007,6 +1012,8 @@ const (
DefTiDBEnableFastReorg = false
DefTiDBDDLDiskQuota = 100 * 1024 * 1024 * 1024 // 100GB
DefExecutorConcurrency = 5
DefTiDBEnableGeneralPlanCache = false
DefTiDBGeneralPlanCacheSize = 100
// MaxDDLReorgBatchSize is exported for testing.
MaxDDLReorgBatchSize int32 = 10240
MinDDLReorgBatchSize int32 = 32
Expand Down

0 comments on commit 53bb5d7

Please sign in to comment.