diff --git a/build/nogo_config.json b/build/nogo_config.json index 3a8b2ba60ead0..2a5fe64ba3e49 100644 --- a/build/nogo_config.json +++ b/build/nogo_config.json @@ -181,6 +181,7 @@ "util/tracing/": "util/tracing/ code", "util/trxevents/": "util/trxevents/ code", "util/watcher/": "util/watcher/ code", + "util/gctuner": "util/gctuner", "store/mockstore/unistore/util": "store/mockstore/unistore/util code", "ddl/util/": "ddl/util code" } @@ -197,6 +198,7 @@ ".*_generated\\.go$": "ignore generated code" }, "only_files": { + "util/gctuner": "util/gctuner", "br/pkg/lightning/mydump/": "br/pkg/lightning/mydump/", "br/pkg/lightning/restore/opts": "br/pkg/lightning/restore/opts", "executor/aggregate.go": "executor/aggregate.go", @@ -743,6 +745,7 @@ ".*_generated\\.go$": "ignore generated code" }, "only_files": { + "util/gctuner": "util/gctuner", "br/pkg/lightning/mydump/": "br/pkg/lightning/mydump/", "br/pkg/lightning/restore/opts": "br/pkg/lightning/restore/opts", "executor/aggregate.go": "executor/aggregate.go", diff --git a/sessionctx/variable/BUILD.bazel b/sessionctx/variable/BUILD.bazel index eb006fda89e30..3a9eb1439ce2c 100644 --- a/sessionctx/variable/BUILD.bazel +++ b/sessionctx/variable/BUILD.bazel @@ -41,6 +41,7 @@ go_library( "//util/collate", "//util/dbterror", "//util/execdetails", + "//util/gctuner", "//util/kvcache", "//util/logutil", "//util/mathutil", diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index c2ffebab89398..9221a99862393 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -36,6 +36,7 @@ import ( "github.com/pingcap/tidb/types" _ "github.com/pingcap/tidb/types/parser_driver" // for parser driver "github.com/pingcap/tidb/util/collate" + "github.com/pingcap/tidb/util/gctuner" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/mathutil" "github.com/pingcap/tidb/util/memory" @@ -619,6 +620,14 @@ var defaultSysVars = []*SysVar{ VarTiDBSuperReadOnly.Store(TiDBOptOn(val)) return nil }}, + {Scope: ScopeGlobal, Name: TiDBEnableGOGCTuner, Value: BoolToOnOff(DefTiDBEnableGOGCTuner), Type: TypeBool, SetGlobal: func(s *SessionVars, val string) error { + on := TiDBOptOn(val) + gctuner.EnableGOGCTuner.Store(on) + if !on { + gctuner.SetDefaultGOGC() + } + return nil + }}, {Scope: ScopeGlobal, Name: TiDBEnableTelemetry, Value: BoolToOnOff(DefTiDBEnableTelemetry), Type: TypeBool}, {Scope: ScopeGlobal, Name: TiDBEnableHistoricalStats, Value: Off, Type: TypeBool}, /* tikv gc metrics */ diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index 19c630852afdc..51181a48bebb3 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -822,6 +822,8 @@ const ( TiDBServerMemoryLimit = "tidb_server_memory_limit" // TiDBServerMemoryLimitSessMinSize indicates the minimal memory used of a session, that becomes a candidate for session kill. TiDBServerMemoryLimitSessMinSize = "tidb_server_memory_limit_sess_min_size" + // TiDBEnableGOGCTuner is to enable GOGC tuner. it can tuner GOGC + TiDBEnableGOGCTuner = "tidb_enable_gogc_tuner" ) // TiDB intentional limits @@ -1048,6 +1050,7 @@ const ( DefTiDBOptRangeMaxSize = 0 DefTiDBCostModelVer = 1 DefTiDBServerMemoryLimitSessMinSize = 128 << 20 + DefTiDBEnableGOGCTuner = true ) // Process global variables. diff --git a/tidb-server/BUILD.bazel b/tidb-server/BUILD.bazel index b455a2276c655..b5594261d3ba0 100644 --- a/tidb-server/BUILD.bazel +++ b/tidb-server/BUILD.bazel @@ -36,6 +36,7 @@ go_library( "//util/deadlockhistory", "//util/disk", "//util/domainutil", + "//util/gctuner", "//util/kvcache", "//util/logutil", "//util/memory", diff --git a/tidb-server/main.go b/tidb-server/main.go index b44a313d6203f..032b4cae19111 100644 --- a/tidb-server/main.go +++ b/tidb-server/main.go @@ -61,6 +61,7 @@ import ( "github.com/pingcap/tidb/util/deadlockhistory" "github.com/pingcap/tidb/util/disk" "github.com/pingcap/tidb/util/domainutil" + "github.com/pingcap/tidb/util/gctuner" "github.com/pingcap/tidb/util/kvcache" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/memory" @@ -204,7 +205,7 @@ func main() { printInfo() setupBinlogClient() setupMetrics() - + setupGCTuner() storage, dom := createStoreAndDomain() svr := createServer(storage, dom) @@ -783,6 +784,15 @@ func setupTracing() { opentracing.SetGlobalTracer(tracer) } +func setupGCTuner() { + limit, err := memory.MemTotal() + if err != nil { + log.Fatal("setupGCTuner failed", zap.Error(err)) + } + threshold := limit * 7 / 10 + gctuner.Tuning(threshold) +} + func closeDomainAndStorage(storage kv.Storage, dom *domain.Domain) { tikv.StoreShuttingDown(1) dom.Close() diff --git a/util/gctuner/BUILD.bazel b/util/gctuner/BUILD.bazel new file mode 100644 index 0000000000000..262a280204f3e --- /dev/null +++ b/util/gctuner/BUILD.bazel @@ -0,0 +1,31 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") + +go_library( + name = "gctuner", + srcs = [ + "finalizer.go", + "mem.go", + "tuner.go", + ], + importpath = "github.com/pingcap/tidb/util/gctuner", + visibility = ["//visibility:public"], + deps = [ + "//util", + "@org_uber_go_atomic//:atomic", + ], +) + +go_test( + name = "gctuner_test", + srcs = [ + "finalizer_test.go", + "mem_test.go", + "tuner_test.go", + ], + embed = [":gctuner"], + flaky = True, + deps = [ + "@com_github_stretchr_testify//assert", + "@com_github_stretchr_testify//require", + ], +) diff --git a/util/gctuner/finalizer.go b/util/gctuner/finalizer.go new file mode 100644 index 0000000000000..3a08a5d3a9057 --- /dev/null +++ b/util/gctuner/finalizer.go @@ -0,0 +1,58 @@ +// Copyright 2022 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, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package gctuner + +import ( + "runtime" + + "go.uber.org/atomic" +) + +type finalizerCallback func() + +type finalizer struct { + ref *finalizerRef + callback finalizerCallback + stopped atomic.Int32 +} + +type finalizerRef struct { + parent *finalizer +} + +func finalizerHandler(f *finalizerRef) { + // stop calling callback + if f.parent.stopped.Load() > 0 { + return + } + f.parent.callback() + runtime.SetFinalizer(f, finalizerHandler) +} + +// newFinalizer return a finalizer object and caller should save it to make sure it will not be gc. +// the go runtime promise the callback function should be called every gc time. +func newFinalizer(callback finalizerCallback) *finalizer { + f := &finalizer{ + callback: callback, + } + f.ref = &finalizerRef{parent: f} + runtime.SetFinalizer(f.ref, finalizerHandler) + f.ref = nil // trigger gc + return f +} + +func (f *finalizer) stop() { + f.stopped.Store(1) +} diff --git a/util/gctuner/finalizer_test.go b/util/gctuner/finalizer_test.go new file mode 100644 index 0000000000000..1e11b39627fb3 --- /dev/null +++ b/util/gctuner/finalizer_test.go @@ -0,0 +1,50 @@ +// Copyright 2022 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, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package gctuner + +import ( + "runtime" + "sync/atomic" + "testing" + + "github.com/stretchr/testify/require" +) + +type testState struct { + count int32 +} + +func TestFinalizer(t *testing.T) { + maxCount := int32(16) + state := &testState{} + f := newFinalizer(func() { + n := atomic.AddInt32(&state.count, 1) + if n > maxCount { + t.Fatalf("cannot exec finalizer callback after f has been gc") + } + }) + for i := int32(1); i <= maxCount; i++ { + runtime.GC() + require.Equal(t, i, atomic.LoadInt32(&state.count)) + } + require.Nil(t, f.ref) + + f.stop() + require.Equal(t, maxCount, atomic.LoadInt32(&state.count)) + runtime.GC() + require.Equal(t, maxCount, atomic.LoadInt32(&state.count)) + runtime.GC() + require.Equal(t, maxCount, atomic.LoadInt32(&state.count)) +} diff --git a/util/gctuner/mem.go b/util/gctuner/mem.go new file mode 100644 index 0000000000000..68c02db8a7c66 --- /dev/null +++ b/util/gctuner/mem.go @@ -0,0 +1,26 @@ +// Copyright 2022 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, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package gctuner + +import ( + "runtime" +) + +var memStats runtime.MemStats + +func readMemoryInuse() uint64 { + runtime.ReadMemStats(&memStats) + return memStats.HeapInuse +} diff --git a/util/gctuner/mem_test.go b/util/gctuner/mem_test.go new file mode 100644 index 0000000000000..4cf0b7504ae08 --- /dev/null +++ b/util/gctuner/mem_test.go @@ -0,0 +1,31 @@ +// Copyright 2022 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, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package gctuner + +import ( + "testing" + + "github.com/stretchr/testify/require" +) + +func TestMem(t *testing.T) { + const mb = 1024 * 1024 + + heap := make([]byte, 100*mb+1) + inuse := readMemoryInuse() + t.Logf("mem inuse: %d MB", inuse/mb) + require.GreaterOrEqual(t, inuse, uint64(100*mb)) + heap[0] = 0 +} diff --git a/util/gctuner/tuner.go b/util/gctuner/tuner.go new file mode 100644 index 0000000000000..2ee3679ef2363 --- /dev/null +++ b/util/gctuner/tuner.go @@ -0,0 +1,163 @@ +// Copyright 2022 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, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package gctuner + +import ( + "math" + "os" + "strconv" + "sync/atomic" + + "github.com/pingcap/tidb/util" +) + +const ( + // MaxGCPercent is the default max cost of memory. + MaxGCPercent uint32 = 500 + // MinGCPercent is the default min cost of memory. + MinGCPercent uint32 = 20 +) + +var defaultGCPercent uint32 = 100 + +// EnableGOGCTuner is to control whether enable the GOGC tuner. +var EnableGOGCTuner atomic.Bool + +func init() { + if val, err := strconv.Atoi(os.Getenv("GOGC")); err == nil { + defaultGCPercent = uint32(val) + } +} + +// SetDefaultGOGC is to set the default GOGC value. +func SetDefaultGOGC() { + util.SetGOGC(int(defaultGCPercent)) +} + +// Tuning sets the threshold of heap which will be respect by gc tuner. +// When Tuning, the env GOGC will not be take effect. +// threshold: disable tuning if threshold == 0 +func Tuning(threshold uint64) { + // disable gc tuner if percent is zero + if threshold <= 0 && globalTuner != nil { + globalTuner.stop() + globalTuner = nil + return + } + + if globalTuner == nil { + globalTuner = newTuner(threshold) + return + } + globalTuner.setThreshold(threshold) +} + +// GetGOGC returns the current GCPercent. +func GetGOGC() uint32 { + if globalTuner == nil { + return defaultGCPercent + } + return globalTuner.getGCPercent() +} + +// only allow one gc tuner in one process +// It is not thread-safe. so it is a private, singleton pattern to avoid misuse. +var globalTuner *tuner + +/* +// Heap +// _______________ => limit: host/cgroup memory hard limit +// | | +// |---------------| => threshold: increase GCPercent when gc_trigger < threshold +// | | +// |---------------| => gc_trigger: heap_live + heap_live * GCPercent / 100 +// | | +// |---------------| +// | heap_live | +// |_______________| +*/ +// Go runtime only trigger GC when hit gc_trigger which affected by GCPercent and heap_live. +// So we can change GCPercent dynamically to tuning GC performance. +type tuner struct { + finalizer *finalizer + gcPercent atomic.Uint32 + threshold atomic.Uint64 // high water level, in bytes +} + +func newTuner(threshold uint64) *tuner { + t := &tuner{} + t.gcPercent.Store(defaultGCPercent) + t.threshold.Store(threshold) + t.finalizer = newFinalizer(t.tuning) // start tuning + return t +} + +func (t *tuner) stop() { + t.finalizer.stop() +} + +func (t *tuner) setThreshold(threshold uint64) { + t.threshold.Store(threshold) +} + +func (t *tuner) getThreshold() uint64 { + return t.threshold.Load() +} + +func (t *tuner) setGCPercent(percent uint32) uint32 { + result := uint32(util.SetGOGC(int(percent))) + t.gcPercent.Store(result) + return result +} + +func (t *tuner) getGCPercent() uint32 { + return t.gcPercent.Load() +} + +// tuning check the memory inuse and tune GC percent dynamically. +// Go runtime ensure that it will be called serially. +func (t *tuner) tuning() { + inuse := readMemoryInuse() + threshold := t.getThreshold() + // stop gc tuning + if threshold <= 0 { + return + } + if EnableGOGCTuner.Load() { + t.setGCPercent(calcGCPercent(inuse, threshold)) + } +} + +// threshold = inuse + inuse * (gcPercent / 100) +// => gcPercent = (threshold - inuse) / inuse * 100 +// if threshold < inuse*2, so gcPercent < 100, and GC positively to avoid OOM +// if threshold > inuse*2, so gcPercent > 100, and GC negatively to reduce GC times +func calcGCPercent(inuse, threshold uint64) uint32 { + // invalid params + if inuse == 0 || threshold == 0 { + return defaultGCPercent + } + // inuse heap larger than threshold, use min percent + if threshold <= inuse { + return MinGCPercent + } + gcPercent := uint32(math.Floor(float64(threshold-inuse) / float64(inuse) * 100)) + if gcPercent < MinGCPercent { + return MinGCPercent + } else if gcPercent > MaxGCPercent { + return MaxGCPercent + } + return gcPercent +} diff --git a/util/gctuner/tuner_test.go b/util/gctuner/tuner_test.go new file mode 100644 index 0000000000000..153bf7368ab14 --- /dev/null +++ b/util/gctuner/tuner_test.go @@ -0,0 +1,93 @@ +// Copyright 2022 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, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package gctuner + +import ( + "runtime" + "testing" + + "github.com/stretchr/testify/require" +) + +var testHeap []byte + +func TestTuner(t *testing.T) { + EnableGOGCTuner.Store(true) + memLimit := uint64(100 * 1024 * 1024) //100 MB + threshold := memLimit / 2 + tn := newTuner(threshold) + require.Equal(t, threshold, tn.threshold.Load()) + require.Equal(t, defaultGCPercent, tn.getGCPercent()) + + // no heap + testHeap = make([]byte, 1) + runtime.GC() + runtime.GC() + for i := 0; i < 100; i++ { + runtime.GC() + require.Equal(t, MaxGCPercent, tn.getGCPercent()) + } + + // 1/4 threshold + testHeap = make([]byte, threshold/4) + for i := 0; i < 100; i++ { + runtime.GC() + require.GreaterOrEqual(t, tn.getGCPercent(), uint32(100)) + require.LessOrEqual(t, tn.getGCPercent(), uint32(500)) + } + + // 1/2 threshold + testHeap = make([]byte, threshold/2) + runtime.GC() + for i := 0; i < 100; i++ { + runtime.GC() + require.GreaterOrEqual(t, tn.getGCPercent(), uint32(50)) + require.LessOrEqual(t, tn.getGCPercent(), uint32(100)) + } + + // 3/4 threshold + testHeap = make([]byte, threshold/4*3) + runtime.GC() + for i := 0; i < 100; i++ { + runtime.GC() + require.Equal(t, MinGCPercent, tn.getGCPercent()) + } + + // out of threshold + testHeap = make([]byte, threshold+1024) + runtime.GC() + for i := 0; i < 100; i++ { + runtime.GC() + require.Equal(t, MinGCPercent, tn.getGCPercent()) + } +} + +func TestCalcGCPercent(t *testing.T) { + const gb = 1024 * 1024 * 1024 + // use default value when invalid params + require.Equal(t, defaultGCPercent, calcGCPercent(0, 0)) + require.Equal(t, defaultGCPercent, calcGCPercent(0, 1)) + require.Equal(t, defaultGCPercent, calcGCPercent(1, 0)) + + require.Equal(t, MaxGCPercent, calcGCPercent(1, 3*gb)) + require.Equal(t, MaxGCPercent, calcGCPercent(gb/10, 4*gb)) + require.Equal(t, MaxGCPercent, calcGCPercent(gb/2, 4*gb)) + require.Equal(t, uint32(300), calcGCPercent(1*gb, 4*gb)) + require.Equal(t, uint32(166), calcGCPercent(1.5*gb, 4*gb)) + require.Equal(t, uint32(100), calcGCPercent(2*gb, 4*gb)) + require.Equal(t, uint32(33), calcGCPercent(3*gb, 4*gb)) + require.Equal(t, MinGCPercent, calcGCPercent(4*gb, 4*gb)) + require.Equal(t, MinGCPercent, calcGCPercent(5*gb, 4*gb)) +} diff --git a/util/gogc.go b/util/gogc.go index 044fb45131573..f5aedf9530300 100644 --- a/util/gogc.go +++ b/util/gogc.go @@ -34,13 +34,14 @@ func init() { } // SetGOGC update GOGC and related metrics. -func SetGOGC(val int) { +func SetGOGC(val int) int { if val <= 0 { val = 100 } - debug.SetGCPercent(val) + result := debug.SetGCPercent(val) metrics.GOGC.Set(float64(val)) atomic.StoreInt64(&gogcValue, int64(val)) + return result } // GetGOGC returns the current value of GOGC.