Skip to content

Commit

Permalink
planner: remove StatsHandler.mu (pingcap#47082)
Browse files Browse the repository at this point in the history
  • Loading branch information
qw4990 authored Sep 19, 2023
1 parent 4450ae4 commit 8f0b9d5
Show file tree
Hide file tree
Showing 7 changed files with 51 additions and 42 deletions.
4 changes: 0 additions & 4 deletions domain/domain.go
Original file line number Diff line number Diff line change
Expand Up @@ -2329,10 +2329,6 @@ func (do *Domain) loadStatsWorker() {
for {
select {
case <-loadTicker.C:
err = statsHandle.RefreshVars()
if err != nil {
logutil.BgLogger().Debug("refresh variables failed", zap.Error(err))
}
err = statsHandle.Update(do.InfoSchema())
if err != nil {
logutil.BgLogger().Debug("update stats info failed", zap.Error(err))
Expand Down
1 change: 0 additions & 1 deletion executor/infoschema_reader_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -433,7 +433,6 @@ func TestPartitionsTable(t *testing.T) {
tk := testkit.NewTestKit(t, store)
tk.MustExec("USE test;")
testkit.WithPruneMode(tk, variable.Static, func() {
require.NoError(t, h.RefreshVars())
tk.MustExec("DROP TABLE IF EXISTS `test_partitions`;")
tk.MustExec(`CREATE TABLE test_partitions (a int, b int, c varchar(5), primary key(a), index idx(c)) PARTITION BY RANGE (a) (PARTITION p0 VALUES LESS THAN (6), PARTITION p1 VALUES LESS THAN (11), PARTITION p2 VALUES LESS THAN (16));`)
require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh()))
Expand Down
1 change: 0 additions & 1 deletion statistics/handle/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -41,7 +41,6 @@ go_library(
"//util/logutil",
"//util/mathutil",
"//util/sqlexec",
"//util/syncutil",
"//util/timeutil",
"@com_github_ngaut_pools//:pools",
"@com_github_pingcap_errors//:errors",
Expand Down
28 changes: 1 addition & 27 deletions statistics/handle/handle.go
Original file line number Diff line number Diff line change
Expand Up @@ -48,7 +48,6 @@ import (
"github.com/pingcap/tidb/util/logutil"
"github.com/pingcap/tidb/util/mathutil"
"github.com/pingcap/tidb/util/sqlexec"
"github.com/pingcap/tidb/util/syncutil"
"github.com/tiancaiamao/gp"
"github.com/tikv/client-go/v2/oracle"
atomic2 "go.uber.org/atomic"
Expand Down Expand Up @@ -95,11 +94,6 @@ type Handle struct {
// StatsLoad is used to load stats concurrently
StatsLoad StatsLoad

mu struct {
ctx sessionctx.Context
syncutil.RWMutex
}

schemaMu struct {
// pid2tid is the map from partition ID to table ID.
pid2tid map[int64]int64
Expand Down Expand Up @@ -160,26 +154,18 @@ func (h *Handle) execRestrictedSQLWithSnapshot(ctx context.Context, sql string,

// Clear the statsCache, only for test.
func (h *Handle) Clear() {
// TODO: Here h.mu seems to protect all the fields of Handle. Is is reasonable?
h.mu.Lock()
cache, err := cache.NewStatsCache()
if err != nil {
logutil.BgLogger().Warn("create stats cache failed", zap.Error(err))
h.mu.Unlock()
return
}
h.statsCache.Replace(cache)
for len(h.ddlEventCh) > 0 {
<-h.ddlEventCh
}
h.mu.ctx.GetSessionVars().InitChunkSize = 1
h.mu.ctx.GetSessionVars().MaxChunkSize = 1
h.mu.ctx.GetSessionVars().EnableChunkRPC = false
h.mu.ctx.GetSessionVars().SetProjectionConcurrency(0)
h.listHead.ClearForTest()
h.tableDelta.reset()
h.statsUsage.reset()
h.mu.Unlock()
}

type sessionPool interface {
Expand All @@ -188,7 +174,7 @@ type sessionPool interface {
}

// NewHandle creates a Handle for update stats.
func NewHandle(ctx, initStatsCtx sessionctx.Context, lease time.Duration, pool sessionPool, tracker sessionctx.SysProcTracker, autoAnalyzeProcIDGetter func() uint64) (*Handle, error) {
func NewHandle(_, initStatsCtx sessionctx.Context, lease time.Duration, pool sessionPool, tracker sessionctx.SysProcTracker, autoAnalyzeProcIDGetter func() uint64) (*Handle, error) {
cfg := config.GetGlobalConfig()
gpool := gp.New(math.MaxInt16, time.Minute)
handle := &Handle{
Expand All @@ -203,7 +189,6 @@ func NewHandle(ctx, initStatsCtx sessionctx.Context, lease time.Duration, pool s
}
handle.initStatsCtx = initStatsCtx
handle.lease.Store(lease)
handle.mu.ctx = ctx
statsCache, err := cache.NewStatsCachePointer()
if err != nil {
return nil, err
Expand All @@ -215,10 +200,6 @@ func NewHandle(ctx, initStatsCtx sessionctx.Context, lease time.Duration, pool s
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{}
err = handle.RefreshVars()
if err != nil {
return nil, err
}
return handle, nil
}

Expand Down Expand Up @@ -1539,13 +1520,6 @@ func (h *Handle) SaveExtendedStatsToStorage(tableID int64, extStats *statistics.
return nil
}

// RefreshVars uses to pull PartitionPruneMethod vars from kv storage.
func (h *Handle) RefreshVars() error {
h.mu.Lock()
defer h.mu.Unlock()
return UpdateSCtxVarsForStats(h.mu.ctx)
}

// CheckAnalyzeVersion checks whether all the statistics versions of this table's columns and indexes are the same.
func (h *Handle) CheckAnalyzeVersion(tblInfo *model.TableInfo, physicalIDs []int64, version *int) bool {
// We simply choose one physical id to get its stats.
Expand Down
11 changes: 6 additions & 5 deletions statistics/handle/update.go
Original file line number Diff line number Diff line change
Expand Up @@ -888,7 +888,7 @@ func (h *Handle) autoAnalyzeTable(tblInfo *model.TableInfo, statsTbl *statistics
return false
}
logutil.BgLogger().Info("auto analyze triggered", zap.String("category", "stats"), zap.String("sql", escaped), zap.String("reason", reason))
tableStatsVer, err := h.getCurrentAnalyzeVersion()
tableStatsVer, err := h.GetCurrentAnalyzeVersion()
if err != nil {
logutil.BgLogger().Error("fail to get analyze version", zap.String("category", "stats"), zap.Error(err))
return false
Expand All @@ -906,7 +906,7 @@ func (h *Handle) autoAnalyzeTable(tblInfo *model.TableInfo, statsTbl *statistics
return false
}
logutil.BgLogger().Info("auto analyze for unanalyzed", zap.String("category", "stats"), zap.String("sql", escaped))
tableStatsVer, err := h.getCurrentAnalyzeVersion()
tableStatsVer, err := h.GetCurrentAnalyzeVersion()
if err != nil {
logutil.BgLogger().Error("fail to get analyze version", zap.String("category", "stats"), zap.Error(err))
return false
Expand All @@ -919,7 +919,8 @@ func (h *Handle) autoAnalyzeTable(tblInfo *model.TableInfo, statsTbl *statistics
return false
}

func (h *Handle) getCurrentAnalyzeVersion() (int, error) {
// GetCurrentAnalyzeVersion returns the current analyze version.
func (h *Handle) GetCurrentAnalyzeVersion() (int, error) {
se, err := h.pool.Get()
if err != nil {
return 0, err
Expand All @@ -932,7 +933,7 @@ func (h *Handle) getCurrentAnalyzeVersion() (int, error) {
return sctx.GetSessionVars().AnalyzeVersion, nil
}

// GetCurrentPruneMode returns the current latest partitioning talbe prune mode.
// GetCurrentPruneMode returns the current latest partitioning table prune mode.
func (h *Handle) GetCurrentPruneMode() (string, error) {
se, err := h.pool.Get()
if err != nil {
Expand All @@ -947,7 +948,7 @@ func (h *Handle) GetCurrentPruneMode() (string, error) {
}

func (h *Handle) autoAnalyzePartitionTableInDynamicMode(tblInfo *model.TableInfo, partitionDefs []model.PartitionDefinition, db string, ratio float64, analyzeSnapshot bool) bool {
tableStatsVer, err := h.getCurrentAnalyzeVersion()
tableStatsVer, err := h.GetCurrentAnalyzeVersion()
if err != nil {
logutil.BgLogger().Info("fail to get analyze version", zap.String("category", "stats"),
zap.String("table", tblInfo.Name.String()),
Expand Down
3 changes: 2 additions & 1 deletion statistics/handle/updatetest/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -8,11 +8,12 @@ go_test(
"update_test.go",
],
flaky = True,
shard_count = 27,
shard_count = 28,
deps = [
"//parser/model",
"//parser/mysql",
"//planner/cardinality",
"//sessionctx",
"//sessionctx/variable",
"//statistics",
"//statistics/handle",
Expand Down
45 changes: 42 additions & 3 deletions statistics/handle/updatetest/update_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/parser/mysql"
"github.com/pingcap/tidb/planner/cardinality"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/statistics"
"github.com/pingcap/tidb/statistics/handle"
Expand Down Expand Up @@ -299,7 +300,6 @@ func TestUpdatePartition(t *testing.T) {
testKit.MustQuery("select @@tidb_partition_prune_mode").Check(testkit.Rows(pruneMode))
testKit.MustExec("use test")
testkit.WithPruneMode(testKit, variable.Static, func() {
err := dom.StatsHandle().RefreshVars()
require.NoError(t, err)
testKit.MustExec("drop table if exists t")
createTable := `CREATE TABLE t (a int, b char(5)) PARTITION BY RANGE (a) (PARTITION p0 VALUES LESS THAN (6),PARTITION p1 VALUES LESS THAN (11))`
Expand Down Expand Up @@ -481,7 +481,6 @@ func TestAutoUpdatePartition(t *testing.T) {
tableInfo := tbl.Meta()
pi := tableInfo.GetPartitionInfo()
h := do.StatsHandle()
require.NoError(t, h.RefreshVars())

require.NoError(t, h.Update(is))
stats := h.GetPartitionStats(tableInfo, pi.Definitions[0].ID)
Expand Down Expand Up @@ -1004,6 +1003,47 @@ func TestMergeTopN(t *testing.T) {
}
}

func TestStatsVariables(t *testing.T) {
store, dom := testkit.CreateMockStoreAndDomain(t)
tk := testkit.NewTestKit(t, store)
h := dom.StatsHandle()
sctx := tk.Session().(sessionctx.Context)

pruneMode, err := h.GetCurrentPruneMode()
require.NoError(t, err)
require.Equal(t, string(variable.Dynamic), pruneMode)
analyzeVer, err := h.GetCurrentAnalyzeVersion()
require.NoError(t, err)
require.Equal(t, 2, analyzeVer)
err = handle.UpdateSCtxVarsForStats(sctx)
require.NoError(t, err)
require.Equal(t, 2, sctx.GetSessionVars().AnalyzeVersion)
require.Equal(t, true, sctx.GetSessionVars().EnableHistoricalStats)
require.Equal(t, string(variable.Dynamic), sctx.GetSessionVars().PartitionPruneMode.Load())
require.Equal(t, false, sctx.GetSessionVars().EnableAnalyzeSnapshot)
require.Equal(t, true, sctx.GetSessionVars().SkipMissingPartitionStats)

tk.MustExec(`set global tidb_analyze_version=1`)
tk.MustExec(`set global tidb_partition_prune_mode='static'`)
tk.MustExec(`set global tidb_enable_historical_stats=0`)
tk.MustExec(`set global tidb_enable_analyze_snapshot=1`)
tk.MustExec(`set global tidb_skip_missing_partition_stats=0`)

pruneMode, err = h.GetCurrentPruneMode()
require.NoError(t, err)
require.Equal(t, string(variable.Static), pruneMode)
analyzeVer, err = h.GetCurrentAnalyzeVersion()
require.NoError(t, err)
require.Equal(t, 1, analyzeVer)
err = handle.UpdateSCtxVarsForStats(sctx)
require.NoError(t, err)
require.Equal(t, 1, sctx.GetSessionVars().AnalyzeVersion)
require.Equal(t, false, sctx.GetSessionVars().EnableHistoricalStats)
require.Equal(t, string(variable.Static), sctx.GetSessionVars().PartitionPruneMode.Load())
require.Equal(t, true, sctx.GetSessionVars().EnableAnalyzeSnapshot)
require.Equal(t, false, sctx.GetSessionVars().SkipMissingPartitionStats)
}

func TestAutoUpdatePartitionInDynamicOnlyMode(t *testing.T) {
store, dom := testkit.CreateMockStoreAndDomain(t)
testKit := testkit.NewTestKit(t, store)
Expand All @@ -1020,7 +1060,6 @@ func TestAutoUpdatePartitionInDynamicOnlyMode(t *testing.T) {
do := dom
is := do.InfoSchema()
h := do.StatsHandle()
require.NoError(t, h.RefreshVars())
require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh()))

testKit.MustExec("insert into t values (1, 'a'), (2, 'b'), (11, 'c'), (12, 'd'), (21, 'e'), (22, 'f')")
Expand Down

0 comments on commit 8f0b9d5

Please sign in to comment.