Skip to content

Commit

Permalink
util: add more log for expensive and global memory control (#39397)
Browse files Browse the repository at this point in the history
  • Loading branch information
wshwsh12 authored Nov 28, 2022
1 parent 50bcb52 commit b3bbe96
Show file tree
Hide file tree
Showing 5 changed files with 45 additions and 9 deletions.
4 changes: 2 additions & 2 deletions util/expensivequery/expensivequery.go
Original file line number Diff line number Diff line change
Expand Up @@ -62,9 +62,9 @@ func (eqh *Handle) Run() {
}

costTime := time.Since(info.Time)
if !info.ExceedExpensiveTimeThresh && costTime >= time.Second*time.Duration(threshold) && log.GetLevel() <= zapcore.WarnLevel {
if time.Since(info.ExpensiveLogTime) > 60*time.Second && costTime >= time.Second*time.Duration(threshold) && log.GetLevel() <= zapcore.WarnLevel {
logExpensiveQuery(costTime, info, "expensive_query")
info.ExceedExpensiveTimeThresh = true
info.ExpensiveLogTime = time.Now()
}
if info.MaxExecutionTime > 0 && costTime > time.Duration(info.MaxExecutionTime)*time.Millisecond {
logutil.BgLogger().Warn("execution timeout, kill it", zap.Duration("costTime", costTime),
Expand Down
13 changes: 11 additions & 2 deletions util/memory/tracker.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,9 @@ import (
"time"

"github.com/pingcap/tidb/metrics"
"github.com/pingcap/tidb/util/logutil"
atomicutil "go.uber.org/atomic"
"go.uber.org/zap"
"golang.org/x/exp/slices"
)

Expand Down Expand Up @@ -91,8 +93,9 @@ type Tracker struct {
maxConsumed atomicutil.Int64 // max number of bytes consumed during execution.
SessionID uint64 // SessionID indicates the sessionID the tracker is bound.
NeedKill atomic.Bool // NeedKill indicates whether this session need kill because OOM
IsRootTrackerOfSess bool // IsRootTrackerOfSess indicates whether this tracker is bound for session
isGlobal bool // isGlobal indicates whether this tracker is global tracker
NeedKillReceived sync.Once
IsRootTrackerOfSess bool // IsRootTrackerOfSess indicates whether this tracker is bound for session
isGlobal bool // isGlobal indicates whether this tracker is global tracker
}

type actionMu struct {
Expand Down Expand Up @@ -316,6 +319,7 @@ func (t *Tracker) Detach() {
parent.actionMuForSoftLimit.actionOnExceed = nil
parent.actionMuForSoftLimit.Unlock()
parent.NeedKill.Store(false)
parent.NeedKillReceived = sync.Once{}
}
parent.remove(t)
t.mu.Lock()
Expand Down Expand Up @@ -449,6 +453,11 @@ func (t *Tracker) Consume(bs int64) {
if bs > 0 && sessionRootTracker != nil {
// Kill the Top1 session
if sessionRootTracker.NeedKill.Load() {
sessionRootTracker.NeedKillReceived.Do(
func() {
logutil.BgLogger().Warn("global memory controller, NeedKill signal is received successfully",
zap.Uint64("connID", sessionRootTracker.SessionID))
})
tryActionLastOne(&sessionRootTracker.actionMuForHardLimit, sessionRootTracker)
}
// Update the Top1 session
Expand Down
10 changes: 5 additions & 5 deletions util/processinfo.go
Original file line number Diff line number Diff line change
Expand Up @@ -41,6 +41,7 @@ type OOMAlarmVariablesInfo struct {
// ProcessInfo is a struct used for show processlist statement.
type ProcessInfo struct {
Time time.Time
ExpensiveLogTime time.Time
Plan interface{}
StmtCtx *stmtctx.StatementContext
RefCountOfStmtCtx *stmtctx.ReferenceCount
Expand All @@ -60,11 +61,10 @@ type ProcessInfo struct {
CurTxnStartTS uint64
// MaxExecutionTime is the timeout for select statement, in milliseconds.
// If the query takes too long, kill it.
MaxExecutionTime uint64
State uint16
Command byte
ExceedExpensiveTimeThresh bool
RedactSQL bool
MaxExecutionTime uint64
State uint16
Command byte
RedactSQL bool
}

// ToRowForShow returns []interface{} for the row data of "SHOW [FULL] PROCESSLIST".
Expand Down
2 changes: 2 additions & 0 deletions util/servermemorylimit/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -9,8 +9,10 @@ go_library(
"//parser/mysql",
"//types",
"//util",
"//util/logutil",
"//util/memory",
"@org_uber_go_atomic//:atomic",
"@org_uber_go_zap//:zap",
],
)

Expand Down
25 changes: 25 additions & 0 deletions util/servermemorylimit/servermemorylimit.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,8 +24,10 @@ import (
"github.com/pingcap/tidb/parser/mysql"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util"
"github.com/pingcap/tidb/util/logutil"
"github.com/pingcap/tidb/util/memory"
atomicutil "go.uber.org/atomic"
"go.uber.org/zap"
)

// Process global Observation indicators for memory limit.
Expand Down Expand Up @@ -81,12 +83,24 @@ type sessionToBeKilled struct {
sqlStartTime time.Time
sessionID uint64
sessionTracker *memory.Tracker

killStartTime time.Time
lastLogTime time.Time
}

func killSessIfNeeded(s *sessionToBeKilled, bt uint64, sm util.SessionManager) {
if s.isKilling {
if info, ok := sm.GetProcessInfo(s.sessionID); ok {
if info.Time == s.sqlStartTime {
if time.Since(s.lastLogTime) > 5*time.Second {
logutil.BgLogger().Warn(fmt.Sprintf("global memory controller failed to kill the top-consumer in %ds",
time.Since(s.killStartTime)/time.Second),
zap.Uint64("connID", info.ID),
zap.String("sql digest", info.Digest),
zap.String("sql text", fmt.Sprintf("%.100v", info.Info)),
zap.Int64("sql memory usage", info.MemTracker.BytesConsumed()))
s.lastLogTime = time.Now()
}
return
}
}
Expand All @@ -95,6 +109,7 @@ func killSessIfNeeded(s *sessionToBeKilled, bt uint64, sm util.SessionManager) {
memory.MemUsageTop1Tracker.CompareAndSwap(s.sessionTracker, nil)
//nolint: all_revive,revive
runtime.GC()
logutil.BgLogger().Warn("global memory controller killed the top1 memory consumer successfully")
}

if bt == 0 {
Expand All @@ -108,6 +123,14 @@ func killSessIfNeeded(s *sessionToBeKilled, bt uint64, sm util.SessionManager) {
t := memory.MemUsageTop1Tracker.Load()
if t != nil {
if info, ok := sm.GetProcessInfo(t.SessionID); ok {
logutil.BgLogger().Warn("global memory controller tries to kill the top1 memory consumer",
zap.Uint64("connID", info.ID),
zap.String("sql digest", info.Digest),
zap.String("sql text", fmt.Sprintf("%.100v", info.Info)),
zap.Uint64("tidb_server_memory_limit", bt),
zap.Uint64("heap inuse", instanceStats.HeapInuse),
zap.Int64("sql memory usage", info.MemTracker.BytesConsumed()),
)
s.sessionID = t.SessionID
s.sqlStartTime = info.Time
s.isKilling = true
Expand All @@ -119,6 +142,8 @@ func killSessIfNeeded(s *sessionToBeKilled, bt uint64, sm util.SessionManager) {
SessionKillLast.Store(killTime)
IsKilling.Store(true)
GlobalMemoryOpsHistoryManager.recordOne(info, killTime, bt, instanceStats.HeapInuse)
s.lastLogTime = time.Now()
s.killStartTime = time.Now()
}
}
}
Expand Down

0 comments on commit b3bbe96

Please sign in to comment.