Skip to content

Commit

Permalink
*: support to modify config server-memory-quota-ratio dynamically (pi…
Browse files Browse the repository at this point in the history
  • Loading branch information
wshwsh12 authored Nov 19, 2020
1 parent 629ef07 commit 2b363fb
Show file tree
Hide file tree
Showing 8 changed files with 50 additions and 14 deletions.
1 change: 1 addition & 0 deletions session/session.go
Original file line number Diff line number Diff line change
Expand Up @@ -2223,6 +2223,7 @@ var builtinGlobalVariable = []string{
variable.TiDBShardAllocateStep,
variable.TiDBEnableChangeColumnType,
variable.TiDBEnableAmendPessimisticTxn,
variable.TiDBMemoryUsageAlarmRatio,
}

var (
Expand Down
17 changes: 17 additions & 0 deletions session/session_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -3517,6 +3517,23 @@ func (s *testBackupRestoreSuite) TestBackupAndRestore(c *C) {
}
}

func (s *testSessionSuite2) TestMemoryUsageAlarmVariable(c *C) {
tk := testkit.NewTestKitWithInit(c, s.store)

tk.MustExec("set @@session.tidb_memory_usage_alarm_ratio=1")
tk.MustQuery("select @@session.tidb_memory_usage_alarm_ratio").Check(testkit.Rows("1"))
tk.MustExec("set @@session.tidb_memory_usage_alarm_ratio=0")
tk.MustQuery("select @@session.tidb_memory_usage_alarm_ratio").Check(testkit.Rows("0"))
tk.MustExec("set @@session.tidb_memory_usage_alarm_ratio=0.7")
tk.MustQuery("select @@session.tidb_memory_usage_alarm_ratio").Check(testkit.Rows("0.7"))
err := tk.ExecToErr("set @@session.tidb_memory_usage_alarm_ratio=1.1")
c.Assert(err.Error(), Equals, "[variable:1231]Variable 'tidb_memory_usage_alarm_ratio' can't be set to the value of '1.1'")
err = tk.ExecToErr("set @@session.tidb_memory_usage_alarm_ratio=-1")
c.Assert(err.Error(), Equals, "[variable:1231]Variable 'tidb_memory_usage_alarm_ratio' can't be set to the value of '-1'")
err = tk.ExecToErr("set @@global.tidb_memory_usage_alarm_ratio=0.8")
c.Assert(err.Error(), Equals, "Variable 'tidb_memory_usage_alarm_ratio' is a SESSION variable and can't be used with SET GLOBAL")
}

func (s *testSessionSuite2) TestSelectLockInShare(c *C) {
tk1 := testkit.NewTestKitWithInit(c, s.store)
tk1.MustExec("DROP TABLE IF EXISTS t_sel_in_share")
Expand Down
2 changes: 2 additions & 0 deletions sessionctx/variable/session.go
Original file line number Diff line number Diff line change
Expand Up @@ -1584,6 +1584,8 @@ func (s *SessionVars) SetSystemVar(name string, val string) error {
s.EnableAmendPessimisticTxn = TiDBOptOn(val)
case TiDBTxnScope:
s.TxnScope = val
case TiDBMemoryUsageAlarmRatio:
MemoryUsageAlarmRatio.Store(tidbOptFloat64(val, 0.8))
}
s.systems[name] = val
return nil
Expand Down
1 change: 1 addition & 0 deletions sessionctx/variable/sysvar.go
Original file line number Diff line number Diff line change
Expand Up @@ -1052,6 +1052,7 @@ var defaultSysVars = []*SysVar{
{Scope: ScopeSession, Name: TiDBWaitSplitRegionTimeout, Value: strconv.Itoa(DefWaitSplitRegionTimeout), Type: TypeUnsigned, MinValue: 1, MaxValue: math.MaxInt64},
{Scope: ScopeSession, Name: TiDBLowResolutionTSO, Value: BoolOff, Type: TypeBool},
{Scope: ScopeSession, Name: TiDBExpensiveQueryTimeThreshold, Value: strconv.Itoa(DefTiDBExpensiveQueryTimeThreshold), Type: TypeUnsigned, MinValue: int64(MinExpensiveQueryTimeThreshold), MaxValue: uint64(math.MaxInt64), AutoConvertOutOfRange: true},
{Scope: ScopeSession, Name: TiDBMemoryUsageAlarmRatio, Value: strconv.FormatFloat(config.GetGlobalConfig().Performance.MemoryUsageAlarmRatio, 'f', -1, 64), Type: TypeFloat, MinValue: 0.0, MaxValue: 1.0},
{Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableNoopFuncs, Value: BoolToOnOff(DefTiDBEnableNoopFuncs), Type: TypeBool},
{Scope: ScopeSession, Name: TiDBReplicaRead, Value: "leader", Type: TypeEnum, PossibleValues: []string{"leader", "follower", "leader-and-follower"}},
{Scope: ScopeSession, Name: TiDBAllowRemoveAutoInc, Value: BoolToOnOff(DefTiDBAllowRemoveAutoInc), Type: TypeBool},
Expand Down
5 changes: 5 additions & 0 deletions sessionctx/variable/tidb_vars.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ import (
"os"

"github.com/pingcap/parser/mysql"
"github.com/pingcap/tidb/config"
"github.com/uber-go/atomic"
)

Expand Down Expand Up @@ -463,6 +464,9 @@ const (

// TiDBEnableAmendPessimisticTxn indicates if amend pessimistic transactions is enabled.
TiDBEnableAmendPessimisticTxn = "tidb_enable_amend_pessimistic_txn"

// TiDBMemoryUsageAlarmRatio indicates the alarm threshold when memory usage of the tidb-server exceeds.
TiDBMemoryUsageAlarmRatio = "tidb_memory_usage_alarm_ratio"
)

// Default TiDB system variable values.
Expand Down Expand Up @@ -596,4 +600,5 @@ var (
MinExpensiveQueryTimeThreshold uint64 = 10 //10s
CapturePlanBaseline = serverGlobalVariable{globalVal: BoolOff}
DefExecutorConcurrency = 5
MemoryUsageAlarmRatio = atomic.NewFloat64(config.GetGlobalConfig().Performance.MemoryUsageAlarmRatio)
)
2 changes: 2 additions & 0 deletions sessionctx/variable/varsutil.go
Original file line number Diff line number Diff line change
Expand Up @@ -126,6 +126,8 @@ func GetSessionOnlySysVars(s *SessionVars, key string) (string, bool, error) {
return val, true, nil
case TiDBExpensiveQueryTimeThreshold:
return fmt.Sprintf("%d", atomic.LoadUint64(&ExpensiveQueryTimeThreshold)), true, nil
case TiDBMemoryUsageAlarmRatio:
return fmt.Sprintf("%g", MemoryUsageAlarmRatio.Load()), true, nil
case TiDBConfig:
conf := config.GetGlobalConfig()
j, err := json.MarshalIndent(conf, "", "\t")
Expand Down
4 changes: 3 additions & 1 deletion util/expensivequery/expensivequery.go
Original file line number Diff line number Diff line change
Expand Up @@ -55,7 +55,7 @@ func (eqh *Handle) Run() {
ticker := time.NewTicker(tickInterval)
defer ticker.Stop()
sm := eqh.sm.Load().(util.SessionManager)
record := initMemoryUsageAlarmRecord()
record := &memoryUsageAlarm{}
for {
select {
case <-ticker.C:
Expand All @@ -75,6 +75,8 @@ func (eqh *Handle) Run() {
}
}
threshold = atomic.LoadUint64(&variable.ExpensiveQueryTimeThreshold)

record.memoryUsageAlarmRatio = variable.MemoryUsageAlarmRatio.Load()
if record.err == nil {
record.alarm4ExcessiveMemUsage(sm)
}
Expand Down
32 changes: 19 additions & 13 deletions util/expensivequery/memory_usage_alarm.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,6 @@ import (
"strings"
"time"

"github.com/pingcap/errors"
"github.com/pingcap/tidb/config"
"github.com/pingcap/tidb/util"
"github.com/pingcap/tidb/util/disk"
Expand All @@ -36,26 +35,22 @@ import (

type memoryUsageAlarm struct {
err error
initialized bool
isServerMemoryQuotaSet bool
serverMemoryQuota uint64
memoryUsageAlarmRatio float64
lastCheckTime time.Time

tmpDir string
lastLogFileName []string
lastProfileFileName [][]string // heap, goroutine
}

func initMemoryUsageAlarmRecord() (record *memoryUsageAlarm) {
record = &memoryUsageAlarm{}
if alert := config.GetGlobalConfig().Performance.MemoryUsageAlarmRatio; alert == 0 || alert == 1 {
record.err = errors.New("close memory usage alarm recorder")
return
}
func (record *memoryUsageAlarm) initMemoryUsageAlarmRecord() {
if quota := config.GetGlobalConfig().Performance.ServerMemoryQuota; quota != 0 {
record.serverMemoryQuota = quota
record.isServerMemoryQuotaSet = true
} else {
// TODO: Get the memory info in container directly.
record.serverMemoryQuota, record.err = memory.MemTotal()
if record.err != nil {
logutil.BgLogger().Error("get system total memory fail", zap.Error(record.err))
Expand All @@ -72,7 +67,8 @@ func initMemoryUsageAlarmRecord() (record *memoryUsageAlarm) {
// Read last records
files, err := ioutil.ReadDir(record.tmpDir)
if err != nil {
return record
record.err = err
return
}
for _, f := range files {
name := filepath.Join(record.tmpDir, f.Name())
Expand All @@ -86,13 +82,23 @@ func initMemoryUsageAlarmRecord() (record *memoryUsageAlarm) {
record.lastProfileFileName[1] = append(record.lastProfileFileName[1], name)
}
}

return record
record.initialized = true
return
}

// If Performance.ServerMemoryQuota is set, use `ServerMemoryQuota * MemoryUsageAlarmRatio` to check oom risk.
// If Performance.ServerMemoryQuota is not set, use `system total memory size * MemoryUsageAlarmRatio` to check oom risk.
func (record *memoryUsageAlarm) alarm4ExcessiveMemUsage(sm util.SessionManager) {
if record.memoryUsageAlarmRatio <= 0.0 || record.memoryUsageAlarmRatio >= 1.0 {
return
}
if !record.initialized {
record.initMemoryUsageAlarmRecord()
if record.err != nil {
return
}
}

var memoryUsage uint64
instanceStats := &runtime.MemStats{}
runtime.ReadMemStats(instanceStats)
Expand All @@ -107,7 +113,7 @@ func (record *memoryUsageAlarm) alarm4ExcessiveMemUsage(sm util.SessionManager)
}

// TODO: Consider NextGC to record SQLs.
if float64(memoryUsage) > float64(record.serverMemoryQuota)*config.GetGlobalConfig().Performance.MemoryUsageAlarmRatio {
if float64(memoryUsage) > float64(record.serverMemoryQuota)*record.memoryUsageAlarmRatio {
// At least ten seconds between two recordings that memory usage is less than threshold (default 80% system memory).
// If the memory is still exceeded, only records once.
interval := time.Since(record.lastCheckTime)
Expand All @@ -129,7 +135,7 @@ func (record *memoryUsageAlarm) doRecord(memUsage uint64, instanceMemoryUsage ui
fields = append(fields, zap.Any("system memory usage", memUsage))
fields = append(fields, zap.Any("tidb-server memory usage", instanceMemoryUsage))
}
fields = append(fields, zap.Any("memory-usage-alarm-ratio", config.GetGlobalConfig().Performance.MemoryUsageAlarmRatio))
fields = append(fields, zap.Any("memory-usage-alarm-ratio", record.memoryUsageAlarmRatio))
fields = append(fields, zap.Any("record path", record.tmpDir))

logutil.BgLogger().Warn("tidb-server has the risk of OOM. Running SQLs and heap profile will be recorded in record path", fields...)
Expand Down

0 comments on commit 2b363fb

Please sign in to comment.