diff --git a/pkg/domain/domain.go b/pkg/domain/domain.go index 8ddfd0ef3d7ce..aa31e7834b639 100644 --- a/pkg/domain/domain.go +++ b/pkg/domain/domain.go @@ -3110,9 +3110,9 @@ func (do *Domain) StopAutoAnalyze() { // InitInstancePlanCache initializes the instance level plan cache for this Domain. func (do *Domain) InitInstancePlanCache() { - softLimit := variable.InstancePlanCacheTargetMemSize.Load() hardLimit := variable.InstancePlanCacheMaxMemSize.Load() - do.instancePlanCache = NewInstancePlanCache(softLimit, hardLimit) + softLimit := float64(hardLimit) * (1 - variable.InstancePlanCacheReservedPercentage.Load()) + do.instancePlanCache = NewInstancePlanCache(int64(softLimit), hardLimit) // use a separate goroutine to avoid the eviction blocking other operations. do.wg.Run(do.planCacheEvictTrigger, "planCacheEvictTrigger") do.wg.Run(do.planCacheMetricsAndVars, "planCacheMetricsAndVars") @@ -3136,8 +3136,8 @@ func (do *Domain) planCacheMetricsAndVars() { select { case <-ticker.C: // update limits - softLimit := variable.InstancePlanCacheTargetMemSize.Load() hardLimit := variable.InstancePlanCacheMaxMemSize.Load() + softLimit := int64(float64(hardLimit) * (1 - variable.InstancePlanCacheReservedPercentage.Load())) curSoft, curHard := do.instancePlanCache.GetLimits() if curSoft != softLimit || curHard != hardLimit { do.instancePlanCache.SetLimits(softLimit, hardLimit) diff --git a/pkg/executor/set_test.go b/pkg/executor/set_test.go index d86c735d0cc2b..4f11f7621230f 100644 --- a/pkg/executor/set_test.go +++ b/pkg/executor/set_test.go @@ -768,20 +768,24 @@ func TestSetVar(t *testing.T) { // test for instance plan cache variables tk.MustQuery("select @@global.tidb_enable_instance_plan_cache").Check(testkit.Rows("0")) // default 0 - tk.MustQuery("select @@global.tidb_instance_plan_cache_target_mem_size").Check(testkit.Rows("104857600")) - tk.MustQuery("select @@global.tidb_instance_plan_cache_max_mem_size").Check(testkit.Rows("125829120")) - tk.MustExecToErr("set global tidb_instance_plan_cache_target_mem_size = 125829121") // target <= max - tk.MustExecToErr("set global tidb_instance_plan_cache_max_mem_size = 104857599") - tk.MustExec("set global tidb_instance_plan_cache_target_mem_size = 114857600") - tk.MustQuery("select @@global.tidb_instance_plan_cache_target_mem_size").Check(testkit.Rows("114857600")) - tk.MustExec("set global tidb_instance_plan_cache_max_mem_size = 135829120") - tk.MustQuery("select @@global.tidb_instance_plan_cache_max_mem_size").Check(testkit.Rows("135829120")) - tk.MustExec("set global tidb_instance_plan_cache_max_mem_size = 1GiB") - tk.MustQuery("select @@global.tidb_instance_plan_cache_max_mem_size").Check(testkit.Rows("1073741824")) - tk.MustExec("set global tidb_instance_plan_cache_target_mem_size = 999MiB") - tk.MustQuery("select @@global.tidb_instance_plan_cache_target_mem_size").Check(testkit.Rows("1047527424")) - tk.MustExec("set global tidb_instance_plan_cache_target_mem_size = 998MiB") - tk.MustQuery("select @@global.tidb_instance_plan_cache_target_mem_size").Check(testkit.Rows("1046478848")) + tk.MustQuery("select @@global.tidb_instance_plan_cache_max_size").Check(testkit.Rows("104857600")) + tk.MustExec("set global tidb_instance_plan_cache_max_size = 135829120") + tk.MustQuery("select @@global.tidb_instance_plan_cache_max_size").Check(testkit.Rows("135829120")) + tk.MustExec("set global tidb_instance_plan_cache_max_size = 999999999") + tk.MustQuery("select @@global.tidb_instance_plan_cache_max_size").Check(testkit.Rows("999999999")) + tk.MustExec("set global tidb_instance_plan_cache_max_size = 1GiB") + tk.MustQuery("select @@global.tidb_instance_plan_cache_max_size").Check(testkit.Rows("1073741824")) + tk.MustExec("set global tidb_instance_plan_cache_max_size = 2GiB") + tk.MustQuery("select @@global.tidb_instance_plan_cache_max_size").Check(testkit.Rows("2147483648")) + tk.MustExecToErr("set global tidb_instance_plan_cache_max_size = 2.5GiB") + tk.MustQuery("select @@global.tidb_instance_plan_cache_max_size").Check(testkit.Rows("2147483648")) + tk.MustQuery("select @@global.tidb_instance_plan_cache_reserved_percentage").Check(testkit.Rows("0.1")) + tk.MustExec(`set global tidb_instance_plan_cache_reserved_percentage=1.1`) + tk.MustQuery("select @@global.tidb_instance_plan_cache_reserved_percentage").Check(testkit.Rows("1")) + tk.MustExec(`set global tidb_instance_plan_cache_reserved_percentage=-0.1`) + tk.MustQuery("select @@global.tidb_instance_plan_cache_reserved_percentage").Check(testkit.Rows("0")) + tk.MustExec(`set global tidb_instance_plan_cache_reserved_percentage=0.5`) + tk.MustQuery("select @@global.tidb_instance_plan_cache_reserved_percentage").Check(testkit.Rows("0.5")) // test variables for cost model ver2 tk.MustQuery("select @@tidb_cost_model_version").Check(testkit.Rows(fmt.Sprintf("%v", variable.DefTiDBCostModelVer))) diff --git a/pkg/sessionctx/variable/sysvar.go b/pkg/sessionctx/variable/sysvar.go index a2b623206579a..a1826409c84ba 100644 --- a/pkg/sessionctx/variable/sysvar.go +++ b/pkg/sessionctx/variable/sysvar.go @@ -1348,19 +1348,15 @@ var defaultSysVars = []*SysVar{ EnableInstancePlanCache.Store(TiDBOptOn(val)) return nil }}, - {Scope: ScopeGlobal, Name: TiDBInstancePlanCacheTargetMemSize, Value: strconv.Itoa(int(DefTiDBInstancePlanCacheTargetMemSize)), Type: TypeStr, + {Scope: ScopeGlobal, Name: TiDBInstancePlanCacheReservedPercentage, + Value: strconv.FormatFloat(DefTiDBInstancePlanCacheReservedPercentage, 'f', -1, 64), + Type: TypeFloat, MinValue: 0, MaxValue: 1, GetGlobal: func(_ context.Context, s *SessionVars) (string, error) { - return strconv.FormatInt(InstancePlanCacheTargetMemSize.Load(), 10), nil + return strconv.FormatFloat(InstancePlanCacheReservedPercentage.Load(), 'f', -1, 64), nil }, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { - v, str := parseByteSize(val) - if str == "" { - v = uint64(TidbOptInt64(val, int64(DefTiDBInstancePlanCacheTargetMemSize))) - } - if v > uint64(InstancePlanCacheMaxMemSize.Load()) { - return errors.New("tidb_instance_plan_cache_target_mem_size must be less than tidb_instance_plan_cache_max_mem_size") - } - InstancePlanCacheTargetMemSize.Store(int64(v)) + v := tidbOptFloat64(val, DefTiDBInstancePlanCacheReservedPercentage) + InstancePlanCacheReservedPercentage.Store(v) return nil }}, {Scope: ScopeGlobal, Name: TiDBInstancePlanCacheMaxMemSize, Value: strconv.Itoa(int(DefTiDBInstancePlanCacheMaxMemSize)), Type: TypeStr, @@ -1370,10 +1366,10 @@ var defaultSysVars = []*SysVar{ SetGlobal: func(_ context.Context, s *SessionVars, val string) error { v, str := parseByteSize(val) if str == "" { - v = uint64(TidbOptInt64(val, int64(DefTiDBInstancePlanCacheTargetMemSize))) + v = uint64(TidbOptInt64(val, int64(DefTiDBInstancePlanCacheMaxMemSize))) } - if v < uint64(InstancePlanCacheTargetMemSize.Load()) { - return errors.New("tidb_instance_plan_cache_max_mem_size must be greater than tidb_instance_plan_cache_target_mem_size") + if v < 0 { + return errors.New("tidb_instance_plan_cache_max_mem_size must be a non-negative integer") } InstancePlanCacheMaxMemSize.Store(int64(v)) return nil diff --git a/pkg/sessionctx/variable/tidb_vars.go b/pkg/sessionctx/variable/tidb_vars.go index 9998376cf3647..cad5fd33bb006 100644 --- a/pkg/sessionctx/variable/tidb_vars.go +++ b/pkg/sessionctx/variable/tidb_vars.go @@ -847,10 +847,10 @@ const ( // TiDBEnableInstancePlanCache indicates whether to enable instance plan cache. // If this variable is false, session-level plan cache will be used. TiDBEnableInstancePlanCache = "tidb_enable_instance_plan_cache" - // TiDBInstancePlanCacheTargetMemSize indicates the target memory size of instance plan cache. - TiDBInstancePlanCacheTargetMemSize = "tidb_instance_plan_cache_target_mem_size" + // TiDBInstancePlanCacheReservedPercentage indicates the percentage memory to evict. + TiDBInstancePlanCacheReservedPercentage = "tidb_instance_plan_cache_reserved_percentage" // TiDBInstancePlanCacheMaxMemSize indicates the maximum memory size of instance plan cache. - TiDBInstancePlanCacheMaxMemSize = "tidb_instance_plan_cache_max_mem_size" + TiDBInstancePlanCacheMaxMemSize = "tidb_instance_plan_cache_max_size" // TiDBConstraintCheckInPlacePessimistic controls whether to skip certain kinds of pessimistic locks. TiDBConstraintCheckInPlacePessimistic = "tidb_constraint_check_in_place_pessimistic" @@ -1352,96 +1352,96 @@ const ( DefTiDBGuaranteeLinearizability = true DefTiDBAnalyzeVersion = 2 // Deprecated: This variable is deprecated, please do not use this variable. - DefTiDBAutoAnalyzePartitionBatchSize = mysql.PartitionCountLimit - DefTiDBEnableIndexMergeJoin = false - DefTiDBTrackAggregateMemoryUsage = true - DefCTEMaxRecursionDepth = 1000 - DefTiDBTmpTableMaxSize = 64 << 20 // 64MB. - DefTiDBEnableLocalTxn = false - DefTiDBTSOClientBatchMaxWaitTime = 0.0 // 0ms - DefTiDBEnableTSOFollowerProxy = false - DefPDEnableFollowerHandleRegion = false - DefTiDBEnableOrderedResultMode = false - DefTiDBEnablePseudoForOutdatedStats = false - DefTiDBRegardNULLAsPoint = true - DefEnablePlacementCheck = true - DefTimestamp = "0" - DefTimestampFloat = 0.0 - DefTiDBEnableStmtSummary = true - DefTiDBStmtSummaryInternalQuery = false - DefTiDBStmtSummaryRefreshInterval = 1800 - DefTiDBStmtSummaryHistorySize = 24 - DefTiDBStmtSummaryMaxStmtCount = 3000 - DefTiDBStmtSummaryMaxSQLLength = 4096 - DefTiDBCapturePlanBaseline = Off - DefTiDBIgnoreInlistPlanDigest = false - DefTiDBEnableIndexMerge = true - DefEnableLegacyInstanceScope = true - DefTiDBTableCacheLease = 3 // 3s - DefTiDBPersistAnalyzeOptions = true - DefTiDBStatsLoadSyncWait = 100 - DefTiDBStatsLoadPseudoTimeout = true - DefSysdateIsNow = false - DefTiDBEnableParallelHashaggSpill = true - DefTiDBEnableMutationChecker = false - DefTiDBTxnAssertionLevel = AssertionOffStr - DefTiDBIgnorePreparedCacheCloseStmt = false - DefTiDBBatchPendingTiFlashCount = 4000 - DefRCReadCheckTS = false - DefTiDBRemoveOrderbyInSubquery = true - DefTiDBSkewDistinctAgg = false - DefTiDB3StageDistinctAgg = true - DefTiDB3StageMultiDistinctAgg = false - DefTiDBOptExplainEvaledSubquery = false - DefTiDBReadStaleness = 0 - DefTiDBGCMaxWaitTime = 24 * 60 * 60 - DefMaxAllowedPacket uint64 = 67108864 - DefTiDBEnableBatchDML = false - DefTiDBMemQuotaQuery = memory.DefMemQuotaQuery // 1GB - DefTiDBStatsCacheMemQuota = 0 - MaxTiDBStatsCacheMemQuota = 1024 * 1024 * 1024 * 1024 // 1TB - DefTiDBQueryLogMaxLen = 4096 - DefRequireSecureTransport = false - DefTiDBCommitterConcurrency = 128 - DefTiDBBatchDMLIgnoreError = false - DefTiDBMemQuotaAnalyze = -1 - DefTiDBEnableAutoAnalyze = true - DefTiDBEnableAutoAnalyzePriorityQueue = true - DefTiDBAnalyzeColumnOptions = "PREDICATE" - DefTiDBMemOOMAction = "CANCEL" - DefTiDBMaxAutoAnalyzeTime = 12 * 60 * 60 - DefTiDBAutoAnalyzeConcurrency = 2 - DefTiDBEnablePrepPlanCache = true - DefTiDBPrepPlanCacheSize = 100 - DefTiDBSessionPlanCacheSize = 100 - DefTiDBEnablePrepPlanCacheMemoryMonitor = true - DefTiDBPrepPlanCacheMemoryGuardRatio = 0.1 - DefTiDBEnableDistTask = true - DefTiDBEnableFastCreateTable = false - DefTiDBSimplifiedMetrics = false - DefTiDBEnablePaging = true - DefTiFlashFineGrainedShuffleStreamCount = 0 - DefStreamCountWhenMaxThreadsNotSet = 8 - DefTiFlashFineGrainedShuffleBatchSize = 8192 - DefAdaptiveClosestReadThreshold = 4096 - DefTiDBEnableAnalyzeSnapshot = false - DefTiDBGenerateBinaryPlan = true - DefEnableTiDBGCAwareMemoryTrack = false - DefTiDBDefaultStrMatchSelectivity = 0.8 - DefTiDBEnableTmpStorageOnOOM = true - DefTiDBEnableMDL = true - DefTiFlashFastScan = false - DefMemoryUsageAlarmRatio = 0.7 - DefMemoryUsageAlarmKeepRecordNum = 5 - DefTiDBEnableFastReorg = true - DefTiDBDDLDiskQuota = 100 * 1024 * 1024 * 1024 // 100GB - DefExecutorConcurrency = 5 - DefTiDBEnableNonPreparedPlanCache = false - DefTiDBEnableNonPreparedPlanCacheForDML = false - DefTiDBNonPreparedPlanCacheSize = 100 - DefTiDBPlanCacheMaxPlanSize = 2 * size.MB - DefTiDBInstancePlanCacheTargetMemSize = 100 * size.MB - DefTiDBInstancePlanCacheMaxMemSize = 120 * size.MB + DefTiDBAutoAnalyzePartitionBatchSize = mysql.PartitionCountLimit + DefTiDBEnableIndexMergeJoin = false + DefTiDBTrackAggregateMemoryUsage = true + DefCTEMaxRecursionDepth = 1000 + DefTiDBTmpTableMaxSize = 64 << 20 // 64MB. + DefTiDBEnableLocalTxn = false + DefTiDBTSOClientBatchMaxWaitTime = 0.0 // 0ms + DefTiDBEnableTSOFollowerProxy = false + DefPDEnableFollowerHandleRegion = false + DefTiDBEnableOrderedResultMode = false + DefTiDBEnablePseudoForOutdatedStats = false + DefTiDBRegardNULLAsPoint = true + DefEnablePlacementCheck = true + DefTimestamp = "0" + DefTimestampFloat = 0.0 + DefTiDBEnableStmtSummary = true + DefTiDBStmtSummaryInternalQuery = false + DefTiDBStmtSummaryRefreshInterval = 1800 + DefTiDBStmtSummaryHistorySize = 24 + DefTiDBStmtSummaryMaxStmtCount = 3000 + DefTiDBStmtSummaryMaxSQLLength = 4096 + DefTiDBCapturePlanBaseline = Off + DefTiDBIgnoreInlistPlanDigest = false + DefTiDBEnableIndexMerge = true + DefEnableLegacyInstanceScope = true + DefTiDBTableCacheLease = 3 // 3s + DefTiDBPersistAnalyzeOptions = true + DefTiDBStatsLoadSyncWait = 100 + DefTiDBStatsLoadPseudoTimeout = true + DefSysdateIsNow = false + DefTiDBEnableParallelHashaggSpill = true + DefTiDBEnableMutationChecker = false + DefTiDBTxnAssertionLevel = AssertionOffStr + DefTiDBIgnorePreparedCacheCloseStmt = false + DefTiDBBatchPendingTiFlashCount = 4000 + DefRCReadCheckTS = false + DefTiDBRemoveOrderbyInSubquery = true + DefTiDBSkewDistinctAgg = false + DefTiDB3StageDistinctAgg = true + DefTiDB3StageMultiDistinctAgg = false + DefTiDBOptExplainEvaledSubquery = false + DefTiDBReadStaleness = 0 + DefTiDBGCMaxWaitTime = 24 * 60 * 60 + DefMaxAllowedPacket uint64 = 67108864 + DefTiDBEnableBatchDML = false + DefTiDBMemQuotaQuery = memory.DefMemQuotaQuery // 1GB + DefTiDBStatsCacheMemQuota = 0 + MaxTiDBStatsCacheMemQuota = 1024 * 1024 * 1024 * 1024 // 1TB + DefTiDBQueryLogMaxLen = 4096 + DefRequireSecureTransport = false + DefTiDBCommitterConcurrency = 128 + DefTiDBBatchDMLIgnoreError = false + DefTiDBMemQuotaAnalyze = -1 + DefTiDBEnableAutoAnalyze = true + DefTiDBEnableAutoAnalyzePriorityQueue = true + DefTiDBAnalyzeColumnOptions = "PREDICATE" + DefTiDBMemOOMAction = "CANCEL" + DefTiDBMaxAutoAnalyzeTime = 12 * 60 * 60 + DefTiDBAutoAnalyzeConcurrency = 2 + DefTiDBEnablePrepPlanCache = true + DefTiDBPrepPlanCacheSize = 100 + DefTiDBSessionPlanCacheSize = 100 + DefTiDBEnablePrepPlanCacheMemoryMonitor = true + DefTiDBPrepPlanCacheMemoryGuardRatio = 0.1 + DefTiDBEnableDistTask = true + DefTiDBEnableFastCreateTable = false + DefTiDBSimplifiedMetrics = false + DefTiDBEnablePaging = true + DefTiFlashFineGrainedShuffleStreamCount = 0 + DefStreamCountWhenMaxThreadsNotSet = 8 + DefTiFlashFineGrainedShuffleBatchSize = 8192 + DefAdaptiveClosestReadThreshold = 4096 + DefTiDBEnableAnalyzeSnapshot = false + DefTiDBGenerateBinaryPlan = true + DefEnableTiDBGCAwareMemoryTrack = false + DefTiDBDefaultStrMatchSelectivity = 0.8 + DefTiDBEnableTmpStorageOnOOM = true + DefTiDBEnableMDL = true + DefTiFlashFastScan = false + DefMemoryUsageAlarmRatio = 0.7 + DefMemoryUsageAlarmKeepRecordNum = 5 + DefTiDBEnableFastReorg = true + DefTiDBDDLDiskQuota = 100 * 1024 * 1024 * 1024 // 100GB + DefExecutorConcurrency = 5 + DefTiDBEnableNonPreparedPlanCache = false + DefTiDBEnableNonPreparedPlanCacheForDML = false + DefTiDBNonPreparedPlanCacheSize = 100 + DefTiDBPlanCacheMaxPlanSize = 2 * size.MB + DefTiDBInstancePlanCacheMaxMemSize = 100 * size.MB + DefTiDBInstancePlanCacheReservedPercentage = 0.1 // MaxDDLReorgBatchSize is exported for testing. MaxDDLReorgBatchSize int32 = 10240 MinDDLReorgBatchSize int32 = 32 @@ -1588,17 +1588,17 @@ var ( OOMAction = atomic.NewString(DefTiDBMemOOMAction) MaxAutoAnalyzeTime = atomic.NewInt64(DefTiDBMaxAutoAnalyzeTime) // variables for plan cache - PreparedPlanCacheMemoryGuardRatio = atomic.NewFloat64(DefTiDBPrepPlanCacheMemoryGuardRatio) - EnableInstancePlanCache = atomic.NewBool(false) - InstancePlanCacheTargetMemSize = atomic.NewInt64(int64(DefTiDBInstancePlanCacheTargetMemSize)) - InstancePlanCacheMaxMemSize = atomic.NewInt64(int64(DefTiDBInstancePlanCacheMaxMemSize)) - EnableDistTask = atomic.NewBool(DefTiDBEnableDistTask) - EnableFastCreateTable = atomic.NewBool(DefTiDBEnableFastCreateTable) - DDLForce2Queue = atomic.NewBool(false) - EnableNoopVariables = atomic.NewBool(DefTiDBEnableNoopVariables) - EnableMDL = atomic.NewBool(false) - AutoAnalyzePartitionBatchSize = atomic.NewInt64(DefTiDBAutoAnalyzePartitionBatchSize) - AutoAnalyzeConcurrency = atomic.NewInt32(DefTiDBAutoAnalyzeConcurrency) + PreparedPlanCacheMemoryGuardRatio = atomic.NewFloat64(DefTiDBPrepPlanCacheMemoryGuardRatio) + EnableInstancePlanCache = atomic.NewBool(false) + InstancePlanCacheReservedPercentage = atomic.NewFloat64(0.1) + InstancePlanCacheMaxMemSize = atomic.NewInt64(int64(DefTiDBInstancePlanCacheMaxMemSize)) + EnableDistTask = atomic.NewBool(DefTiDBEnableDistTask) + EnableFastCreateTable = atomic.NewBool(DefTiDBEnableFastCreateTable) + DDLForce2Queue = atomic.NewBool(false) + EnableNoopVariables = atomic.NewBool(DefTiDBEnableNoopVariables) + EnableMDL = atomic.NewBool(false) + AutoAnalyzePartitionBatchSize = atomic.NewInt64(DefTiDBAutoAnalyzePartitionBatchSize) + AutoAnalyzeConcurrency = atomic.NewInt32(DefTiDBAutoAnalyzeConcurrency) // EnableFastReorg indicates whether to use lightning to enhance DDL reorg performance. EnableFastReorg = atomic.NewBool(DefTiDBEnableFastReorg) // DDLDiskQuota is the temporary variable for set disk quota for lightning