Skip to content

Commit

Permalink
handle: add global status handler (#46974)
Browse files Browse the repository at this point in the history
ref #46905
  • Loading branch information
hawkingrei authored Sep 14, 2023
1 parent 42723fa commit e890aed
Show file tree
Hide file tree
Showing 16 changed files with 517 additions and 365 deletions.
1 change: 1 addition & 0 deletions executor/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -410,6 +410,7 @@ go_test(
"//sessiontxn/staleread",
"//statistics",
"//statistics/handle",
"//statistics/handle/globalstats",
"//store/copr",
"//store/driver/error",
"//store/helper",
Expand Down
5 changes: 3 additions & 2 deletions executor/historical_stats_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@ import (
"github.com/pingcap/tidb/parser/model"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/statistics/handle"
"github.com/pingcap/tidb/statistics/handle/globalstats"
"github.com/pingcap/tidb/testkit"
"github.com/stretchr/testify/require"
"github.com/tikv/client-go/v2/oracle"
Expand Down Expand Up @@ -343,7 +344,7 @@ PARTITION p0 VALUES LESS THAN (6)
require.NotNil(t, jsTable)
// only has p0 stats
require.NotNil(t, jsTable.Partitions["p0"])
require.Nil(t, jsTable.Partitions[handle.TiDBGlobalStats])
require.Nil(t, jsTable.Partitions[globalstats.TiDBGlobalStats])

// change static to dynamic then assert
tk.MustExec("set @@tidb_partition_prune_mode='dynamic'")
Expand All @@ -365,7 +366,7 @@ PARTITION p0 VALUES LESS THAN (6)
require.NotNil(t, jsTable)
// has both global and p0 stats
require.NotNil(t, jsTable.Partitions["p0"])
require.NotNil(t, jsTable.Partitions[handle.TiDBGlobalStats])
require.NotNil(t, jsTable.Partitions[globalstats.TiDBGlobalStats])
}

func TestDumpHistoricalStatsFallback(t *testing.T) {
Expand Down
1 change: 1 addition & 0 deletions server/handler/optimizor/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -56,6 +56,7 @@ go_test(
"//server/internal/util",
"//session",
"//statistics/handle",
"//statistics/handle/globalstats",
"//store/mockstore/unistore",
"//testkit",
"//testkit/testsetup",
Expand Down
3 changes: 2 additions & 1 deletion server/handler/optimizor/statistics_handler_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,7 @@ import (
"github.com/pingcap/tidb/server/internal/util"
"github.com/pingcap/tidb/session"
"github.com/pingcap/tidb/statistics/handle"
"github.com/pingcap/tidb/statistics/handle/globalstats"
"github.com/pingcap/tidb/testkit"
"github.com/stretchr/testify/require"
)
Expand Down Expand Up @@ -180,7 +181,7 @@ func testDumpPartitionTableStats(t *testing.T, client *testserverclient.TestServ
jsonTable := &handle.JSONTable{}
err = json.Unmarshal(b, jsonTable)
require.NoError(t, err)
require.NotNil(t, jsonTable.Partitions[handle.TiDBGlobalStats])
require.NotNil(t, jsonTable.Partitions[globalstats.TiDBGlobalStats])
require.Len(t, jsonTable.Partitions, expectedLen)
}
check(false)
Expand Down
3 changes: 0 additions & 3 deletions statistics/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,6 @@ go_library(
"histogram.go",
"index.go",
"interact_with_storage.go",
"merge_worker.go",
"row_sampler.go",
"sample.go",
"scalar.go",
Expand Down Expand Up @@ -63,7 +62,6 @@ go_test(
name = "statistics_test",
timeout = "short",
srcs = [
"cmsketch_bench_test.go",
"cmsketch_test.go",
"fmsketch_test.go",
"histogram_bench_test.go",
Expand Down Expand Up @@ -100,7 +98,6 @@ go_test(
"@com_github_pingcap_errors//:errors",
"@com_github_pingcap_failpoint//:failpoint",
"@com_github_stretchr_testify//require",
"@com_github_tiancaiamao_gp//:gp",
"@org_uber_go_goleak//:goleak",
],
)
20 changes: 11 additions & 9 deletions statistics/cmsketch.go
Original file line number Diff line number Diff line change
Expand Up @@ -624,7 +624,7 @@ func (c *TopN) QueryTopN(sctx sessionctx.Context, d []byte) (result uint64, foun
if c == nil {
return 0, false
}
idx := c.findTopN(d)
idx := c.FindTopN(d)
if sctx != nil && sctx.GetSessionVars().StmtCtx.EnableOptimizerDebugTrace {
debugtrace.RecordAnyValuesWithNames(sctx, "FindTopN idx", idx)
}
Expand All @@ -634,7 +634,8 @@ func (c *TopN) QueryTopN(sctx sessionctx.Context, d []byte) (result uint64, foun
return c.TopN[idx].Count, true
}

func (c *TopN) findTopN(d []byte) int {
// FindTopN finds the index of the given value in the TopN.
func (c *TopN) FindTopN(d []byte) int {
if c == nil {
return -1
}
Expand Down Expand Up @@ -741,7 +742,7 @@ func (c *TopN) RemoveVal(val []byte) {
if c == nil {
return
}
pos := c.findTopN(val)
pos := c.FindTopN(val)
if pos == -1 {
return
}
Expand All @@ -766,7 +767,7 @@ func (c *TopN) updateTopNWithDelta(d []byte, delta uint64, increase bool) bool {
if c == nil || c.TopN == nil {
return false
}
idx := c.findTopN(d)
idx := c.FindTopN(d)
if idx >= 0 {
if increase {
c.TopN[idx].Count += delta
Expand Down Expand Up @@ -795,15 +796,15 @@ func NewTopN(n int) *TopN {
// 3. `[]*Histogram` are the partition-level histograms which just delete some values when we merge the global-level topN.
func MergePartTopN2GlobalTopN(loc *time.Location, version int, topNs []*TopN, n uint32, hists []*Histogram,
isIndex bool, killed *uint32) (*TopN, []TopNMeta, []*Histogram, error) {
if checkEmptyTopNs(topNs) {
if CheckEmptyTopNs(topNs) {
return nil, nil, hists, nil
}
partNum := len(topNs)
// Different TopN structures may hold the same value, we have to merge them.
counter := make(map[hack.MutableString]float64)
// datumMap is used to store the mapping from the string type to datum type.
// The datum is used to find the value in the histogram.
datumMap := newDatumMapCache()
datumMap := NewDatumMapCache()
for i, topN := range topNs {
if atomic.LoadUint32(killed) == 1 {
return nil, nil, nil, errors.Trace(ErrQueryInterrupted)
Expand All @@ -826,7 +827,7 @@ func MergePartTopN2GlobalTopN(loc *time.Location, version int, topNs []*TopN, n
if atomic.LoadUint32(killed) == 1 {
return nil, nil, nil, errors.Trace(ErrQueryInterrupted)
}
if (j == i && version >= 2) || topNs[j].findTopN(val.Encoded) != -1 {
if (j == i && version >= 2) || topNs[j].FindTopN(val.Encoded) != -1 {
continue
}
// Get the encodedVal from the hists[j]
Expand Down Expand Up @@ -866,7 +867,7 @@ func MergePartTopN2GlobalTopN(loc *time.Location, version int, topNs []*TopN, n
// The output parameters are the newly generated TopN structure and the remaining numbers.
// Notice: The n can be 0. So n has no default value, we must explicitly specify this value.
func MergeTopN(topNs []*TopN, n uint32) (*TopN, []TopNMeta) {
if checkEmptyTopNs(topNs) {
if CheckEmptyTopNs(topNs) {
return nil, nil
}
// Different TopN structures may hold the same value, we have to merge them.
Expand All @@ -891,7 +892,8 @@ func MergeTopN(topNs []*TopN, n uint32) (*TopN, []TopNMeta) {
return GetMergedTopNFromSortedSlice(sorted, n)
}

func checkEmptyTopNs(topNs []*TopN) bool {
// CheckEmptyTopNs checks whether all TopNs are empty.
func CheckEmptyTopNs(topNs []*TopN) bool {
count := uint64(0)
for _, topN := range topNs {
count += topN.TotalCount()
Expand Down
15 changes: 10 additions & 5 deletions statistics/cmsketch_util.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,22 +22,27 @@ import (
"github.com/pingcap/tidb/util/hack"
)

type datumMapCache struct {
// DatumMapCache is used to store the mapping from the string type to datum type.
// The datum is used to find the value in the histogram.
type DatumMapCache struct {
datumMap map[hack.MutableString]types.Datum
}

func newDatumMapCache() *datumMapCache {
return &datumMapCache{
// NewDatumMapCache creates a new DatumMapCache.
func NewDatumMapCache() *DatumMapCache {
return &DatumMapCache{
datumMap: make(map[hack.MutableString]types.Datum),
}
}

func (d *datumMapCache) Get(key hack.MutableString) (val types.Datum, ok bool) {
// Get gets the datum from the cache.
func (d *DatumMapCache) Get(key hack.MutableString) (val types.Datum, ok bool) {
val, ok = d.datumMap[key]
return
}

func (d *datumMapCache) Put(val TopNMeta, encodedVal hack.MutableString,
// Put puts the datum into the cache.
func (d *DatumMapCache) Put(val TopNMeta, encodedVal hack.MutableString,
tp byte, isIndex bool, loc *time.Location) (dat types.Datum, err error) {
dat, err = topNMetaToDatum(val, tp, isIndex, loc)
if err != nil {
Expand Down
2 changes: 2 additions & 0 deletions statistics/handle/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@ go_library(
"//sessiontxn",
"//statistics",
"//statistics/handle/cache",
"//statistics/handle/globalstats",
"//statistics/handle/lockstats",
"//statistics/handle/metrics",
"//table",
Expand Down Expand Up @@ -76,6 +77,7 @@ go_test(
"//sessionctx/stmtctx",
"//sessionctx/variable",
"//statistics",
"//statistics/handle/globalstats",
"//statistics/handle/internal",
"//testkit",
"//testkit/testsetup",
Expand Down
7 changes: 4 additions & 3 deletions statistics/handle/dump.go
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@ import (
"github.com/pingcap/tidb/sessionctx/stmtctx"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/statistics"
"github.com/pingcap/tidb/statistics/handle/globalstats"
handle_metrics "github.com/pingcap/tidb/statistics/handle/metrics"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/logutil"
Expand Down Expand Up @@ -195,7 +196,7 @@ func (h *Handle) DumpHistoricalStatsBySnapshot(
}
// dump its global-stats if existed
if tbl != nil {
jsonTbl.Partitions[TiDBGlobalStats] = tbl
jsonTbl.Partitions[globalstats.TiDBGlobalStats] = tbl
}
return jsonTbl, fallbackTbls, nil
}
Expand Down Expand Up @@ -233,7 +234,7 @@ func (h *Handle) DumpStatsToJSONBySnapshot(dbName string, tableInfo *model.Table
return nil, errors.Trace(err)
}
if tbl != nil {
jsonTbl.Partitions[TiDBGlobalStats] = tbl
jsonTbl.Partitions[globalstats.TiDBGlobalStats] = tbl
}
return jsonTbl, nil
}
Expand Down Expand Up @@ -396,7 +397,7 @@ func (h *Handle) LoadStatsFromJSON(is infoschema.InfoSchema, jsonTbl *JSONTable)
}
}
// load global-stats if existed
if globalStats, ok := jsonTbl.Partitions[TiDBGlobalStats]; ok {
if globalStats, ok := jsonTbl.Partitions[globalstats.TiDBGlobalStats]; ok {
if err := h.loadStatsFromJSON(tableInfo, tableInfo.ID, globalStats); err != nil {
return errors.Trace(err)
}
Expand Down
5 changes: 3 additions & 2 deletions statistics/handle/dump_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ import (
"github.com/pingcap/tidb/parser/model"
"github.com/pingcap/tidb/statistics"
"github.com/pingcap/tidb/statistics/handle"
"github.com/pingcap/tidb/statistics/handle/globalstats"
"github.com/pingcap/tidb/statistics/handle/internal"
"github.com/pingcap/tidb/testkit"
"github.com/pingcap/tidb/util"
Expand Down Expand Up @@ -136,15 +137,15 @@ func TestDumpGlobalStats(t *testing.T) {
stats := getStatsJSON(t, dom, "test", "t")
require.NotNil(t, stats.Partitions["p0"])
require.NotNil(t, stats.Partitions["p1"])
require.Nil(t, stats.Partitions[handle.TiDBGlobalStats])
require.Nil(t, stats.Partitions[globalstats.TiDBGlobalStats])

// global-stats is existed
tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'")
tk.MustExec("analyze table t")
stats = getStatsJSON(t, dom, "test", "t")
require.NotNil(t, stats.Partitions["p0"])
require.NotNil(t, stats.Partitions["p1"])
require.NotNil(t, stats.Partitions[handle.TiDBGlobalStats])
require.NotNil(t, stats.Partitions[globalstats.TiDBGlobalStats])
}

func TestLoadGlobalStats(t *testing.T) {
Expand Down
45 changes: 45 additions & 0 deletions statistics/handle/globalstats/BUILD.bazel
Original file line number Diff line number Diff line change
@@ -0,0 +1,45 @@
load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test")

go_library(
name = "globalstats",
srcs = [
"global_stats.go",
"merge_worker.go",
"topn.go",
],
importpath = "github.com/pingcap/tidb/statistics/handle/globalstats",
visibility = ["//visibility:public"],
deps = [
"//infoschema",
"//parser/ast",
"//parser/model",
"//sessionctx",
"//sessionctx/variable",
"//statistics",
"//table",
"//types",
"//util/hack",
"//util/logutil",
"@com_github_pingcap_errors//:errors",
"@com_github_tiancaiamao_gp//:gp",
"@org_uber_go_zap//:zap",
],
)

go_test(
name = "globalstats_test",
timeout = "short",
srcs = ["topn_bench_test.go"],
embed = [":globalstats"],
flaky = True,
deps = [
"//parser/mysql",
"//sessionctx/stmtctx",
"//statistics",
"//types",
"//util/chunk",
"//util/codec",
"@com_github_stretchr_testify//require",
"@com_github_tiancaiamao_gp//:gp",
],
)
Loading

0 comments on commit e890aed

Please sign in to comment.