You can not select more than 25 topics
Topics must start with a letter or number, can include dashes ('-') and can be up to 35 characters long.
143 lines
3.8 KiB
143 lines
3.8 KiB
// Copyright 2018 PingCAP, Inc.
|
|
//
|
|
// Licensed under the Apache License, Version 2.0 (the "License");
|
|
// you may not use this file except in compliance with the License.
|
|
// You may obtain a copy of the License at
|
|
//
|
|
// http://www.apache.org/licenses/LICENSE-2.0
|
|
//
|
|
// Unless required by applicable law or agreed to in writing, software
|
|
// distributed under the License is distributed on an "AS IS" BASIS,
|
|
// See the License for the specific language governing permissions and
|
|
// limitations under the License.
|
|
|
|
package memory
|
|
|
|
import (
|
|
"fmt"
|
|
"sync"
|
|
|
|
"github.com/pingcap/tidb/errno"
|
|
"github.com/pingcap/tidb/util/dbterror"
|
|
"github.com/pingcap/tidb/util/logutil"
|
|
"go.uber.org/zap"
|
|
)
|
|
|
|
// ActionOnExceed is the action taken when memory usage exceeds memory quota.
|
|
// NOTE: All the implementors should be thread-safe.
|
|
type ActionOnExceed interface {
|
|
// Action will be called when memory usage exceeds memory quota by the
|
|
// corresponding Tracker.
|
|
Action(t *Tracker)
|
|
// SetLogHook binds a log hook which will be triggered and log an detailed
|
|
// message for the out-of-memory sql.
|
|
SetLogHook(hook func(uint64))
|
|
// SetFallback sets a fallback action which will be triggered if itself has
|
|
// already been triggered.
|
|
SetFallback(a ActionOnExceed)
|
|
// GetFallback get the fallback action of the Action.
|
|
GetFallback() ActionOnExceed
|
|
// GetPriority get the priority of the Action.
|
|
GetPriority() int64
|
|
}
|
|
|
|
// BaseOOMAction manages the fallback action for all Action.
|
|
type BaseOOMAction struct {
|
|
fallbackAction ActionOnExceed
|
|
}
|
|
|
|
// SetFallback sets a fallback action which will be triggered if itself has
|
|
// already been triggered.
|
|
func (b *BaseOOMAction) SetFallback(a ActionOnExceed) {
|
|
b.fallbackAction = a
|
|
}
|
|
|
|
// GetFallback get the fallback action of the Action.
|
|
func (b *BaseOOMAction) GetFallback() ActionOnExceed {
|
|
return b.fallbackAction
|
|
}
|
|
|
|
// Default OOM Action priority.
|
|
const (
|
|
DefPanicPriority = iota
|
|
DefLogPriority
|
|
DefSpillPriority
|
|
DefRateLimitPriority
|
|
)
|
|
|
|
// LogOnExceed logs a warning only once when memory usage exceeds memory quota.
|
|
type LogOnExceed struct {
|
|
BaseOOMAction
|
|
mutex sync.Mutex // For synchronization.
|
|
acted bool
|
|
ConnID uint64
|
|
logHook func(uint64)
|
|
}
|
|
|
|
// SetLogHook sets a hook for LogOnExceed.
|
|
func (a *LogOnExceed) SetLogHook(hook func(uint64)) {
|
|
a.logHook = hook
|
|
}
|
|
|
|
// Action logs a warning only once when memory usage exceeds memory quota.
|
|
func (a *LogOnExceed) Action(t *Tracker) {
|
|
a.mutex.Lock()
|
|
defer a.mutex.Unlock()
|
|
if !a.acted {
|
|
a.acted = true
|
|
if a.logHook == nil {
|
|
logutil.BgLogger().Warn("memory exceeds quota",
|
|
zap.Error(errMemExceedThreshold.GenWithStackByArgs(t.label, t.BytesConsumed(), t.bytesLimit, t.String())))
|
|
return
|
|
}
|
|
a.logHook(a.ConnID)
|
|
}
|
|
}
|
|
|
|
// GetPriority get the priority of the Action
|
|
func (a *LogOnExceed) GetPriority() int64 {
|
|
return DefLogPriority
|
|
}
|
|
|
|
// PanicOnExceed panics when memory usage exceeds memory quota.
|
|
type PanicOnExceed struct {
|
|
BaseOOMAction
|
|
mutex sync.Mutex // For synchronization.
|
|
acted bool
|
|
ConnID uint64
|
|
logHook func(uint64)
|
|
}
|
|
|
|
// SetLogHook sets a hook for PanicOnExceed.
|
|
func (a *PanicOnExceed) SetLogHook(hook func(uint64)) {
|
|
a.logHook = hook
|
|
}
|
|
|
|
// Action panics when memory usage exceeds memory quota.
|
|
func (a *PanicOnExceed) Action(t *Tracker) {
|
|
a.mutex.Lock()
|
|
if a.acted {
|
|
a.mutex.Unlock()
|
|
return
|
|
}
|
|
a.acted = true
|
|
a.mutex.Unlock()
|
|
if a.logHook != nil {
|
|
a.logHook(a.ConnID)
|
|
}
|
|
panic(PanicMemoryExceed + fmt.Sprintf("[conn_id=%d]", a.ConnID))
|
|
}
|
|
|
|
// GetPriority get the priority of the Action
|
|
func (a *PanicOnExceed) GetPriority() int64 {
|
|
return DefPanicPriority
|
|
}
|
|
|
|
var (
|
|
errMemExceedThreshold = dbterror.ClassUtil.NewStd(errno.ErrMemExceedThreshold)
|
|
)
|
|
|
|
const (
|
|
// PanicMemoryExceed represents the panic message when out of memory quota.
|
|
PanicMemoryExceed string = "Out Of Memory Quota!"
|
|
)
|
|
|