Skip to content

Commit

Permalink
resourcemanger: create cpu/memory monitor
Browse files Browse the repository at this point in the history
Signed-off-by: Weizhen Wang <wangweizhen@pingcap.com>
  • Loading branch information
hawkingrei committed Dec 15, 2022
1 parent acb8d46 commit 3491770
Show file tree
Hide file tree
Showing 12 changed files with 199 additions and 20 deletions.
44 changes: 42 additions & 2 deletions br/pkg/lightning/common/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -34,9 +34,49 @@ go_library(
"@org_golang_google_grpc//codes",
"@org_golang_google_grpc//credentials",
"@org_golang_google_grpc//status",
"@org_golang_x_sys//unix",
"@org_uber_go_zap//:zap",
],
] + select({
"@io_bazel_rules_go//go/platform:aix": [
"@org_golang_x_sys//unix",
],
"@io_bazel_rules_go//go/platform:android": [
"@org_golang_x_sys//unix",
],
"@io_bazel_rules_go//go/platform:darwin": [
"@org_golang_x_sys//unix",
],
"@io_bazel_rules_go//go/platform:dragonfly": [
"@org_golang_x_sys//unix",
],
"@io_bazel_rules_go//go/platform:freebsd": [
"@org_golang_x_sys//unix",
],
"@io_bazel_rules_go//go/platform:illumos": [
"@org_golang_x_sys//unix",
],
"@io_bazel_rules_go//go/platform:ios": [
"@org_golang_x_sys//unix",
],
"@io_bazel_rules_go//go/platform:js": [
"@org_golang_x_sys//unix",
],
"@io_bazel_rules_go//go/platform:linux": [
"@org_golang_x_sys//unix",
],
"@io_bazel_rules_go//go/platform:netbsd": [
"@org_golang_x_sys//unix",
],
"@io_bazel_rules_go//go/platform:openbsd": [
"@org_golang_x_sys//unix",
],
"@io_bazel_rules_go//go/platform:plan9": [
"@org_golang_x_sys//unix",
],
"@io_bazel_rules_go//go/platform:solaris": [
"@org_golang_x_sys//unix",
],
"//conditions:default": [],
}),
)

go_test(
Expand Down
1 change: 0 additions & 1 deletion br/pkg/lightning/web/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -4,7 +4,6 @@ go_library(
name = "web",
srcs = [
"progress.go",
"res.go",
"res_vfsdata.go",
],
importpath = "github.com/pingcap/tidb/br/pkg/lightning/web",
Expand Down
6 changes: 4 additions & 2 deletions build/nogo_config.json
Original file line number Diff line number Diff line change
Expand Up @@ -404,7 +404,8 @@
"util/": "util code",
"parser/": "parser code",
"meta/": "parser code",
"extension/": "extension code"
"extension/": "extension code",
"resourcemanager/": "resourcemanager code"
}
},
"shift": {
Expand Down Expand Up @@ -765,7 +766,8 @@
"server/conn.go": "server/conn.go",
"server/conn_stmt.go": "server/conn_stmt.go",
"server/conn_test.go": "server/conn_test.go",
"extension/": "extension code"
"extension/": "extension code",
"resourcemanager/": "resourcemanager code"
}
},
"SA2000": {
Expand Down
2 changes: 0 additions & 2 deletions domain/domain.go
Original file line number Diff line number Diff line change
Expand Up @@ -2016,15 +2016,13 @@ func (do *Domain) updateStatsWorker(ctx sessionctx.Context, owner owner.Manager)
loadFeedbackTicker := time.NewTicker(5 * lease)
loadLockedTablesTicker := time.NewTicker(5 * lease)
dumpColStatsUsageTicker := time.NewTicker(100 * lease)
readMemTricker := time.NewTicker(memory.ReadMemInterval)
statsHandle := do.StatsHandle()
defer func() {
dumpColStatsUsageTicker.Stop()
loadFeedbackTicker.Stop()
dumpFeedbackTicker.Stop()
gcStatsTicker.Stop()
deltaUpdateTicker.Stop()
readMemTricker.Stop()
do.SetStatsUpdating(false)
logutil.BgLogger().Info("updateStatsWorker exited.")
}()
Expand Down
13 changes: 13 additions & 0 deletions resourcemanager/BUILD.bazel
Original file line number Diff line number Diff line change
@@ -0,0 +1,13 @@
load("@io_bazel_rules_go//go:def.bzl", "go_library")

go_library(
name = "resourcemanage",
srcs = ["rm.go"],
importpath = "github.com/pingcap/tidb/resourcemanager",
visibility = ["//visibility:public"],
deps = [
"//util",
"//util/cpu",
"//util/memory",
],
)
65 changes: 65 additions & 0 deletions resourcemanager/rm.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,65 @@
// 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 resourcemanager

import (
"time"

tidbutil "github.com/pingcap/tidb/util"
"github.com/pingcap/tidb/util/cpu"
"github.com/pingcap/tidb/util/memory"
)

// GlobalResourceManager is a global resource manager
var GlobalResourceManager = NewResourceManger()

// ResourceManager is a resource manager
type ResourceManager struct {
cpuObserver *cpu.Observer
exitCh chan struct{}
wg tidbutil.WaitGroupWrapper
}

// NewResourceManger is to create a new resource manager
func NewResourceManger() *ResourceManager {
return &ResourceManager{
cpuObserver: cpu.NewCPUObserver(),
exitCh: make(chan struct{}),
}
}

// Start is to start resource manager
func (r *ResourceManager) Start() {
r.wg.Run(r.cpuObserver.Start)
r.wg.Run(func() {
readMemTricker := time.NewTicker(memory.ReadMemInterval)
defer readMemTricker.Stop()
for {
select {
case <-readMemTricker.C:
memory.ForceReadMemStats()
case <-r.exitCh:
return
}
}
})
}

// Stop is to stop resource manager
func (r *ResourceManager) Stop() {
r.cpuObserver.Stop()
close(r.exitCh)
r.wg.Done()
}
5 changes: 1 addition & 4 deletions sessionctx/variable/featuretag/concurrencyddl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -2,10 +2,7 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library")

go_library(
name = "concurrencyddl",
srcs = [
"default.go",
"non_default.go",
],
srcs = ["default.go"],
importpath = "github.com/pingcap/tidb/sessionctx/variable/featuretag/concurrencyddl",
visibility = ["//visibility:public"],
)
5 changes: 1 addition & 4 deletions sessionctx/variable/featuretag/distributereorg/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -2,10 +2,7 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library")

go_library(
name = "distributereorg",
srcs = [
"default.go",
"non_default.go",
],
srcs = ["default.go"],
importpath = "github.com/pingcap/tidb/sessionctx/variable/featuretag/distributereorg",
visibility = ["//visibility:public"],
)
1 change: 1 addition & 0 deletions tidb-server/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@ go_library(
"//planner/core",
"//plugin",
"//privilege/privileges",
"//resourcemanager:resourcemanage",
"//server",
"//session",
"//session/txninfo",
Expand Down
3 changes: 3 additions & 0 deletions tidb-server/main.go
Original file line number Diff line number Diff line change
Expand Up @@ -46,6 +46,7 @@ import (
plannercore "github.com/pingcap/tidb/planner/core"
"github.com/pingcap/tidb/plugin"
"github.com/pingcap/tidb/privilege/privileges"
"github.com/pingcap/tidb/resourcemanager"
"github.com/pingcap/tidb/server"
"github.com/pingcap/tidb/session"
"github.com/pingcap/tidb/session/txninfo"
Expand Down Expand Up @@ -210,6 +211,7 @@ func main() {
printInfo()
setupBinlogClient()
setupMetrics()
resourcemanager.GlobalResourceManager.Start()
storage, dom := createStoreAndDomain()
svr := createServer(storage, dom)

Expand All @@ -223,6 +225,7 @@ func main() {
svr.Close()
cleanup(svr, storage, dom, graceful)
cpuprofile.StopCPUProfiler()
resourcemanager.GlobalResourceManager.Stop()
close(exited)
})
topsql.SetupTopSQL()
Expand Down
31 changes: 27 additions & 4 deletions util/signal/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -9,8 +9,31 @@ go_library(
],
importpath = "github.com/pingcap/tidb/util/signal",
visibility = ["//visibility:public"],
deps = [
"//util/logutil",
"@org_uber_go_zap//:zap",
],
deps = select({
"@io_bazel_rules_go//go/platform:android": [
"//util/logutil",
"@org_uber_go_zap//:zap",
],
"@io_bazel_rules_go//go/platform:darwin": [
"//util/logutil",
"@org_uber_go_zap//:zap",
],
"@io_bazel_rules_go//go/platform:freebsd": [
"//util/logutil",
"@org_uber_go_zap//:zap",
],
"@io_bazel_rules_go//go/platform:ios": [
"//util/logutil",
"@org_uber_go_zap//:zap",
],
"@io_bazel_rules_go//go/platform:linux": [
"//util/logutil",
"@org_uber_go_zap//:zap",
],
"@io_bazel_rules_go//go/platform:windows": [
"//util/logutil",
"@org_uber_go_zap//:zap",
],
"//conditions:default": [],
}),
)
43 changes: 42 additions & 1 deletion util/sys/linux/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -9,7 +9,48 @@ go_library(
],
importpath = "github.com/pingcap/tidb/util/sys/linux",
visibility = ["//visibility:public"],
deps = ["@org_golang_x_sys//unix"],
deps = select({
"@io_bazel_rules_go//go/platform:aix": [
"@org_golang_x_sys//unix",
],
"@io_bazel_rules_go//go/platform:android": [
"@org_golang_x_sys//unix",
],
"@io_bazel_rules_go//go/platform:darwin": [
"@org_golang_x_sys//unix",
],
"@io_bazel_rules_go//go/platform:dragonfly": [
"@org_golang_x_sys//unix",
],
"@io_bazel_rules_go//go/platform:freebsd": [
"@org_golang_x_sys//unix",
],
"@io_bazel_rules_go//go/platform:illumos": [
"@org_golang_x_sys//unix",
],
"@io_bazel_rules_go//go/platform:ios": [
"@org_golang_x_sys//unix",
],
"@io_bazel_rules_go//go/platform:js": [
"@org_golang_x_sys//unix",
],
"@io_bazel_rules_go//go/platform:linux": [
"@org_golang_x_sys//unix",
],
"@io_bazel_rules_go//go/platform:netbsd": [
"@org_golang_x_sys//unix",
],
"@io_bazel_rules_go//go/platform:openbsd": [
"@org_golang_x_sys//unix",
],
"@io_bazel_rules_go//go/platform:plan9": [
"@org_golang_x_sys//unix",
],
"@io_bazel_rules_go//go/platform:solaris": [
"@org_golang_x_sys//unix",
],
"//conditions:default": [],
}),
)

go_test(
Expand Down

0 comments on commit 3491770

Please sign in to comment.