Skip to content

Commit

Permalink
planner: move more methods from StatsHandle to its sub-packages (ping…
Browse files Browse the repository at this point in the history
  • Loading branch information
qw4990 authored and wuhuizuo committed Apr 2, 2024
1 parent 5709ccd commit fc51cc7
Show file tree
Hide file tree
Showing 10 changed files with 182 additions and 112 deletions.
2 changes: 1 addition & 1 deletion pkg/domain/domain.go
Original file line number Diff line number Diff line change
Expand Up @@ -2268,7 +2268,7 @@ func quitStatsOwner(do *Domain, mgr owner.Manager) {
func (do *Domain) StartLoadStatsSubWorkers(ctxList []sessionctx.Context) {
statsHandle := do.StatsHandle()
for i, ctx := range ctxList {
statsHandle.StatsLoad.SubCtxs[i] = ctx
statsHandle.SetSubCtxs(i, ctx)
do.wg.Add(1)
go statsHandle.SubLoadWorker(ctx, do.exit, do.wg)
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/planner/core/casetest/planstats/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,7 @@ go_test(
"//pkg/sessionctx",
"//pkg/sessionctx/stmtctx",
"//pkg/statistics",
"//pkg/statistics/handle",
"//pkg/statistics/handle/util",
"//pkg/table",
"//pkg/testkit",
"//pkg/testkit/testdata",
Expand Down
4 changes: 2 additions & 2 deletions pkg/planner/core/casetest/planstats/plan_stats_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -32,7 +32,7 @@ import (
"github.com/pingcap/tidb/pkg/sessionctx"
"github.com/pingcap/tidb/pkg/sessionctx/stmtctx"
"github.com/pingcap/tidb/pkg/statistics"
"github.com/pingcap/tidb/pkg/statistics/handle"
utilstats "github.com/pingcap/tidb/pkg/statistics/handle/util"
"github.com/pingcap/tidb/pkg/table"
"github.com/pingcap/tidb/pkg/testkit"
"github.com/pingcap/tidb/pkg/testkit/testdata"
Expand Down Expand Up @@ -268,7 +268,7 @@ func TestPlanStatsLoadTimeout(t *testing.T) {
neededColumn := model.TableItemID{TableID: tableInfo.ID, ID: tableInfo.Columns[0].ID, IsIndex: false}
resultCh := make(chan stmtctx.StatsLoadResult, 1)
timeout := time.Duration(1<<63 - 1)
task := &handle.NeededItemTask{
task := &utilstats.NeededItemTask{
TableItemID: neededColumn,
ResultCh: resultCh,
ToTimeout: time.Now().Local().Add(timeout),
Expand Down
13 changes: 2 additions & 11 deletions pkg/statistics/handle/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -6,7 +6,6 @@ go_library(
"bootstrap.go",
"ddl.go",
"handle.go",
"handle_hist.go",
],
importpath = "github.com/pingcap/tidb/pkg/statistics/handle",
visibility = ["//visibility:public"],
Expand All @@ -15,7 +14,6 @@ go_library(
"//pkg/ddl/util",
"//pkg/infoschema",
"//pkg/kv",
"//pkg/metrics",
"//pkg/parser/model",
"//pkg/parser/mysql",
"//pkg/parser/terror",
Expand All @@ -29,14 +27,13 @@ go_library(
"//pkg/statistics/handle/history",
"//pkg/statistics/handle/lockstats",
"//pkg/statistics/handle/storage",
"//pkg/statistics/handle/syncload",
"//pkg/statistics/handle/usage",
"//pkg/statistics/handle/util",
"//pkg/types",
"//pkg/util",
"//pkg/util/chunk",
"//pkg/util/logutil",
"@com_github_pingcap_errors//:errors",
"@com_github_pingcap_failpoint//:failpoint",
"@com_github_tiancaiamao_gp//:gp",
"@org_uber_go_atomic//:atomic",
"@org_uber_go_zap//:zap",
Expand All @@ -48,25 +45,19 @@ go_test(
timeout = "short",
srcs = [
"ddl_test.go",
"handle_hist_test.go",
"main_test.go",
],
embed = [":handle"],
flaky = True,
race = "on",
shard_count = 8,
shard_count = 4,
deps = [
"//pkg/config",
"//pkg/parser/model",
"//pkg/planner/cardinality",
"//pkg/sessionctx",
"//pkg/sessionctx/stmtctx",
"//pkg/testkit",
"//pkg/testkit/testsetup",
"//pkg/types",
"//pkg/util/mathutil",
"//pkg/util/mock",
"@com_github_pingcap_failpoint//:failpoint",
"@com_github_stretchr_testify//require",
"@org_uber_go_goleak//:goleak",
],
Expand Down
15 changes: 5 additions & 10 deletions pkg/statistics/handle/handle.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,18 +18,17 @@ import (
"math"
"time"

"github.com/pingcap/tidb/pkg/config"
ddlUtil "github.com/pingcap/tidb/pkg/ddl/util"
"github.com/pingcap/tidb/pkg/parser/model"
"github.com/pingcap/tidb/pkg/sessionctx"
"github.com/pingcap/tidb/pkg/sessionctx/stmtctx"
"github.com/pingcap/tidb/pkg/statistics"
"github.com/pingcap/tidb/pkg/statistics/handle/autoanalyze"
"github.com/pingcap/tidb/pkg/statistics/handle/cache"
"github.com/pingcap/tidb/pkg/statistics/handle/globalstats"
"github.com/pingcap/tidb/pkg/statistics/handle/history"
"github.com/pingcap/tidb/pkg/statistics/handle/lockstats"
"github.com/pingcap/tidb/pkg/statistics/handle/storage"
"github.com/pingcap/tidb/pkg/statistics/handle/syncload"
"github.com/pingcap/tidb/pkg/statistics/handle/usage"
"github.com/pingcap/tidb/pkg/statistics/handle/util"
"github.com/pingcap/tidb/pkg/util/logutil"
Expand Down Expand Up @@ -70,6 +69,9 @@ type Handle struct {
// StatsAnalyze is used to handle auto-analyze and manage analyze jobs.
util.StatsAnalyze

// StatsSyncLoad is used to load stats syncly.
util.StatsSyncLoad

// StatsReadWriter is used to read/write stats from/to storage.
util.StatsReadWriter

Expand All @@ -94,9 +96,6 @@ type Handle struct {
// StatsCache ...
util.StatsCache

// StatsLoad is used to load stats concurrently
StatsLoad StatsLoad

lease atomic2.Duration
}

Expand All @@ -111,7 +110,6 @@ func (h *Handle) Clear() {

// NewHandle creates a Handle for update stats.
func NewHandle(_, initStatsCtx sessionctx.Context, lease time.Duration, pool util.SessionPool, tracker sessionctx.SysProcTracker, autoAnalyzeProcIDGetter func() uint64) (*Handle, error) {
cfg := config.GetGlobalConfig()
handle := &Handle{
gpool: gp.New(math.MaxInt16, time.Minute),
ddlEventCh: make(chan *ddlUtil.Event, 1000),
Expand All @@ -135,11 +133,8 @@ func NewHandle(_, initStatsCtx sessionctx.Context, lease time.Duration, pool uti
handle.StatsHistory = history.NewStatsHistory(handle)
handle.StatsUsage = usage.NewStatsUsageImpl(handle)
handle.StatsAnalyze = autoanalyze.NewStatsAnalyze(handle)
handle.StatsSyncLoad = syncload.NewStatsSyncLoad(handle)
handle.StatsGlobal = globalstats.NewStatsGlobal(handle)
handle.StatsLoad.SubCtxs = make([]sessionctx.Context, cfg.Performance.StatsLoadConcurrency)
handle.StatsLoad.NeededItemsCh = make(chan *NeededItemTask, cfg.Performance.StatsLoadQueueSize)
handle.StatsLoad.TimeoutItemsCh = make(chan *NeededItemTask, cfg.Performance.StatsLoadQueueSize)
handle.StatsLoad.WorkingColMap = map[model.TableItemID][]chan stmtctx.StatsLoadResult{}
return handle, nil
}

Expand Down
44 changes: 44 additions & 0 deletions pkg/statistics/handle/syncload/BUILD.bazel
Original file line number Diff line number Diff line change
@@ -0,0 +1,44 @@
load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test")

go_library(
name = "syncload",
srcs = ["stats_syncload.go"],
importpath = "github.com/pingcap/tidb/pkg/statistics/handle/syncload",
visibility = ["//visibility:public"],
deps = [
"//pkg/config",
"//pkg/metrics",
"//pkg/parser/model",
"//pkg/parser/mysql",
"//pkg/sessionctx",
"//pkg/sessionctx/stmtctx",
"//pkg/statistics",
"//pkg/statistics/handle/storage",
"//pkg/statistics/handle/util",
"//pkg/types",
"//pkg/util",
"//pkg/util/logutil",
"@com_github_pingcap_errors//:errors",
"@com_github_pingcap_failpoint//:failpoint",
"@org_uber_go_zap//:zap",
],
)

go_test(
name = "syncload_test",
timeout = "short",
srcs = ["stats_syncload_test.go"],
flaky = True,
race = "on",
shard_count = 4,
deps = [
"//pkg/config",
"//pkg/parser/model",
"//pkg/sessionctx",
"//pkg/sessionctx/stmtctx",
"//pkg/testkit",
"//pkg/util/mathutil",
"@com_github_pingcap_failpoint//:failpoint",
"@com_github_stretchr_testify//require",
],
)
Loading

0 comments on commit fc51cc7

Please sign in to comment.