Skip to content

Commit

Permalink
br: mem dump when about to OOM (#59234)
Browse files Browse the repository at this point in the history
close #56971
  • Loading branch information
Tristan1900 authored Feb 25, 2025
1 parent fed0251 commit 15f3c28
Show file tree
Hide file tree
Showing 9 changed files with 302 additions and 53 deletions.
75 changes: 56 additions & 19 deletions br/cmd/br/cmd.go
Original file line number Diff line number Diff line change
Expand Up @@ -79,6 +79,17 @@ const (

flagVersion = "version"
flagVersionShort = "V"

// Memory management related constants
quarterGiB uint64 = 256 * size.MB
halfGiB uint64 = 512 * size.MB
fourGiB uint64 = 4 * size.GB

// Environment variables
envBRHeapDumpDir = "BR_HEAP_DUMP_DIR"

// Default heap dump paths
defaultHeapDumpDir = "/tmp/br_heap_dumps"
)

func timestampLogFileName() string {
Expand Down Expand Up @@ -113,10 +124,6 @@ func DefineCommonFlags(cmd *cobra.Command) {
_ = cmd.PersistentFlags().MarkHidden(FlagRedactLog)
}

const quarterGiB uint64 = 256 * size.MB
const halfGiB uint64 = 512 * size.MB
const fourGiB uint64 = 4 * size.GB

func calculateMemoryLimit(memleft uint64) uint64 {
// memreserved = f(memleft) = 512MB * memleft / (memleft + 4GB)
// * f(0) = 0
Expand All @@ -132,6 +139,47 @@ func calculateMemoryLimit(memleft uint64) uint64 {
return memlimit
}

// setupMemoryMonitoring configures memory limits and starts the memory monitor.
// It returns an error if the setup fails.
func setupMemoryMonitoring(ctx context.Context, memTotal, memUsed uint64) error {
if memUsed >= memTotal {
log.Warn("failed to obtain memory size, skip setting memory limit",
zap.Uint64("memused", memUsed), zap.Uint64("memtotal", memTotal))
return nil
}

memleft := memTotal - memUsed
memlimit := calculateMemoryLimit(memleft)
// BR command needs 256 MiB at least, if the left memory is less than 256 MiB,
// the memory limit cannot limit anyway and then finally OOM.
memlimit = max(memlimit, quarterGiB)

log.Info("calculate the rest memory",
zap.Uint64("memtotal", memTotal),
zap.Uint64("memused", memUsed),
zap.Uint64("memlimit", memlimit))

// No need to set memory limit because the left memory is sufficient.
if memlimit >= uint64(math.MaxInt64) {
return nil
}

debug.SetMemoryLimit(int64(memlimit))

// Configure and start memory monitoring
dumpDir := os.Getenv(envBRHeapDumpDir)
if dumpDir == "" {
dumpDir = defaultHeapDumpDir
}

if err := utils.RunMemoryMonitor(ctx, dumpDir, memlimit); err != nil {
log.Warn("Failed to start memory monitor", zap.Error(err))
return err
}

return nil
}

// Init initializes BR cli.
func Init(cmd *cobra.Command) (err error) {
initOnce.Do(func() {
Expand Down Expand Up @@ -198,21 +246,10 @@ func Init(cmd *cobra.Command) (err error) {
err = e
return
}
if memused >= memtotal {
log.Warn("failed to obtain memory size, skip setting memory limit",
zap.Uint64("memused", memused), zap.Uint64("memtotal", memtotal))
} else {
memleft := memtotal - memused
memlimit := calculateMemoryLimit(memleft)
// BR command needs 256 MiB at least, if the left memory is less than 256 MiB,
// the memory limit cannot limit anyway and then finally OOM.
memlimit = max(memlimit, quarterGiB)
log.Info("calculate the rest memory",
zap.Uint64("memtotal", memtotal), zap.Uint64("memused", memused), zap.Uint64("memlimit", memlimit))
// No need to set memory limit because the left memory is sufficient.
if memlimit < uint64(math.MaxInt64) {
debug.SetMemoryLimit(int64(memlimit))
}

if e := setupMemoryMonitoring(GetDefaultContext(), memtotal, memused); e != nil {
// only log the error, don't fail initialization
log.Error("Failed to setup memory monitoring", zap.Error(e))
}
}

Expand Down
2 changes: 1 addition & 1 deletion br/cmd/br/main.go
Original file line number Diff line number Diff line change
Expand Up @@ -13,6 +13,7 @@ import (
func main() {
gCtx := context.Background()
ctx, cancel := utils.StartExitSingleListener(gCtx)
defer cancel()

rootCmd := &cobra.Command{
Use: "br",
Expand All @@ -34,7 +35,6 @@ func main() {

rootCmd.SetArgs(os.Args[1:])
if err := rootCmd.Execute(); err != nil {
cancel()
log.Error("br failed", zap.Error(err))
os.Exit(1) // nolint:gocritic
}
Expand Down
7 changes: 6 additions & 1 deletion br/pkg/utils/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,7 @@ go_library(
"filter.go",
"json.go",
"key.go",
"memory_monitor.go",
"misc.go",
"pointer.go",
"pprof.go",
Expand Down Expand Up @@ -42,6 +43,8 @@ go_library(
"//pkg/util/codec",
"//pkg/util/encrypt",
"//pkg/util/logutil",
"//pkg/util/memory",
"//pkg/util/memoryusagealarm",
"//pkg/util/sqlexec",
"//pkg/util/table-filter",
"@com_github_cheggaaa_pb_v3//:pb",
Expand Down Expand Up @@ -83,6 +86,7 @@ go_test(
"json_test.go",
"key_test.go",
"main_test.go",
"memory_monitor_test.go",
"misc_test.go",
"progress_test.go",
"register_test.go",
Expand All @@ -91,7 +95,7 @@ go_test(
],
embed = [":utils"],
flaky = True,
shard_count = 37,
shard_count = 38,
deps = [
"//br/pkg/errors",
"//pkg/kv",
Expand All @@ -115,6 +119,7 @@ go_test(
"@io_etcd_go_etcd_tests_v3//integration",
"@org_golang_google_grpc//codes",
"@org_golang_google_grpc//status",
"@org_uber_go_atomic//:atomic",
"@org_uber_go_goleak//:goleak",
"@org_uber_go_multierr//:multierr",
],
Expand Down
87 changes: 87 additions & 0 deletions br/pkg/utils/memory_monitor.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,87 @@
// Copyright 2024 PingCAP, Inc. Licensed under Apache-2.0.

package utils

import (
"context"
"os"

"github.com/pingcap/log"
"github.com/pingcap/tidb/pkg/util/memory"
"github.com/pingcap/tidb/pkg/util/memoryusagealarm"
"go.uber.org/atomic"
"go.uber.org/zap"
)

const (
DefaultProfilesDir = "/tmp/profiles"
// default memory usage alarm ratio (80%)
defaultMemoryUsageAlarmRatio = 0.8
// default number of alarm records to keep
defaultMemoryUsageAlarmKeepRecordNum = 3
)

// BRConfigProvider implements memoryusagealarm.ConfigProvider for BR
type BRConfigProvider struct {
ratio *atomic.Float64
keepNum *atomic.Int64
logDir string
}

func (p *BRConfigProvider) GetMemoryUsageAlarmRatio() float64 {
return p.ratio.Load()
}

func (p *BRConfigProvider) GetMemoryUsageAlarmKeepRecordNum() int64 {
return p.keepNum.Load()
}

func (p *BRConfigProvider) GetLogDir() string {
if p.logDir == "" {
return DefaultProfilesDir
}
return p.logDir
}

func (p *BRConfigProvider) GetComponentName() string {
return "br"
}

// RunMemoryMonitor starts monitoring memory usage and dumps profiles when thresholds are exceeded
func RunMemoryMonitor(ctx context.Context, dumpDir string, memoryLimit uint64) error {
// just in case
if dumpDir == "" {
dumpDir = DefaultProfilesDir
}

// Set memory limit if specified
if memoryLimit > 0 {
memory.ServerMemoryLimit.Store(memoryLimit)
}

log.Info("Memory monitor starting",
zap.String("dump_dir", dumpDir),
zap.Bool("using_temp_dir", dumpDir == os.TempDir()),
zap.Float64("memory_usage_alarm_ratio", defaultMemoryUsageAlarmRatio),
zap.Uint64("memory_limit_mb", memoryLimit/1024/1024))

// Initialize BR config provider with default values
provider := &BRConfigProvider{
ratio: atomic.NewFloat64(defaultMemoryUsageAlarmRatio),
keepNum: atomic.NewInt64(defaultMemoryUsageAlarmKeepRecordNum),
logDir: dumpDir,
}

exitCh := make(chan struct{})
handle := memoryusagealarm.NewMemoryUsageAlarmHandle(exitCh, provider)
// BR doesn't need session manager so setting to nil
handle.SetSessionManager(nil)

go func() {
go handle.Run()
<-ctx.Done()
close(exitCh)
}()

return nil
}
34 changes: 34 additions & 0 deletions br/pkg/utils/memory_monitor_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,34 @@
// Copyright 2024 PingCAP, Inc. Licensed under Apache-2.0.

package utils

import (
"testing"

"github.com/stretchr/testify/require"
"go.uber.org/atomic"
)

func TestBRConfigProvider(t *testing.T) {
provider := &BRConfigProvider{
ratio: atomic.NewFloat64(0.8),
keepNum: atomic.NewInt64(3),
logDir: "/custom/dir",
}

// Test GetMemoryUsageAlarmRatio
require.Equal(t, 0.8, provider.GetMemoryUsageAlarmRatio())

// Test GetMemoryUsageAlarmKeepRecordNum
require.Equal(t, int64(3), provider.GetMemoryUsageAlarmKeepRecordNum())

// Test GetLogDir
require.Equal(t, "/custom/dir", provider.GetLogDir())

// Test GetLogDir with default
provider.logDir = ""
require.Equal(t, DefaultProfilesDir, provider.GetLogDir())

// Test GetComponentName
require.Equal(t, "br", provider.GetComponentName())
}
3 changes: 2 additions & 1 deletion pkg/domain/domain.go
Original file line number Diff line number Diff line change
Expand Up @@ -1374,7 +1374,8 @@ func NewDomainWithEtcdClient(store kv.Storage, schemaLease time.Duration, statsL
do.wg = util.NewWaitGroupEnhancedWrapper("domain", do.exit, config.GetGlobalConfig().TiDBEnableExitCheck)
do.SchemaValidator = NewSchemaValidator(schemaLease, do)
do.expensiveQueryHandle = expensivequery.NewExpensiveQueryHandle(do.exit)
do.memoryUsageAlarmHandle = memoryusagealarm.NewMemoryUsageAlarmHandle(do.exit)
do.memoryUsageAlarmHandle = memoryusagealarm.NewMemoryUsageAlarmHandle(do.exit,
&memoryusagealarm.TiDBConfigProvider{})
do.serverMemoryLimitHandle = servermemorylimit.NewServerMemoryLimitHandle(do.exit)
do.sysProcesses = SysProcesses{mu: &sync.RWMutex{}, procMap: make(map[uint64]sysproctrack.TrackProc)}
do.initDomainSysVars()
Expand Down
1 change: 0 additions & 1 deletion pkg/util/memoryusagealarm/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,6 @@ go_test(
race = "on",
deps = [
"//pkg/sessionctx/stmtctx",
"//pkg/sessionctx/vardef",
"//pkg/util",
"//pkg/util/memory",
"@com_github_stretchr_testify//assert",
Expand Down
Loading

0 comments on commit 15f3c28

Please sign in to comment.