Skip to content

Commit

Permalink
planner: create a separate unified function to update variables for s…
Browse files Browse the repository at this point in the history
…tatistics sctx (#47051)

ref #46905
  • Loading branch information
qw4990 authored Sep 18, 2023
1 parent defa2ad commit 4cc6c65
Show file tree
Hide file tree
Showing 9 changed files with 61 additions and 63 deletions.
10 changes: 0 additions & 10 deletions session/session.go
Original file line number Diff line number Diff line change
Expand Up @@ -3005,16 +3005,6 @@ func (s *session) AuthWithoutVerification(user *auth.UserIdentity) bool {
return false
}

// RefreshVars implements the sessionctx.Context interface.
func (s *session) RefreshVars(_ context.Context) error {
pruneMode, err := s.GetSessionVars().GlobalVarsAccessor.GetGlobalSysVar(variable.TiDBPartitionPruneMode)
if err != nil {
return err
}
s.sessionVars.PartitionPruneMode.Store(pruneMode)
return nil
}

// SetSessionStatesHandler implements the Session.SetSessionStatesHandler interface.
func (s *session) SetSessionStatesHandler(stateType sessionstates.SessionStateType, handler sessionctx.SessionStatesHandler) {
s.sessionStatesHandlers[stateType] = handler
Expand Down
4 changes: 0 additions & 4 deletions sessionctx/context.go
Original file line number Diff line number Diff line change
Expand Up @@ -112,10 +112,6 @@ type Context interface {
// now just for load data and batch insert.
RefreshTxnCtx(context.Context) error

// RefreshVars refreshes modified global variable to current session.
// only used to daemon session like `statsHandle` to detect global variable change.
RefreshVars(context.Context) error

// GetStore returns the store of session.
GetStore() kv.Storage

Expand Down
3 changes: 3 additions & 0 deletions sessionctx/variable/session.go
Original file line number Diff line number Diff line change
Expand Up @@ -1244,6 +1244,9 @@ type SessionVars struct {
// AnalyzeVersion indicates how TiDB collect and use analyzed statistics.
AnalyzeVersion int

// EnableHistoricalStats indicates whether to enable historical statistics.
EnableHistoricalStats bool

// EnableIndexMergeJoin indicates whether to enable index merge join.
EnableIndexMergeJoin bool

Expand Down
1 change: 0 additions & 1 deletion statistics/handle/globalstats/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -14,7 +14,6 @@ go_library(
"//parser/ast",
"//parser/model",
"//sessionctx",
"//sessionctx/variable",
"//statistics",
"//table",
"//types",
Expand Down
11 changes: 0 additions & 11 deletions statistics/handle/globalstats/global_stats.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,6 @@ import (
"github.com/pingcap/tidb/parser/ast"
"github.com/pingcap/tidb/parser/model"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/statistics"
"github.com/pingcap/tidb/table"
"github.com/pingcap/tidb/types"
Expand Down Expand Up @@ -110,16 +109,6 @@ func (g *GlobalStatusHandler) MergePartitionStats2GlobalStats(sc sessionctx.Cont
}

skipMissingPartitionStats := sc.GetSessionVars().SkipMissingPartitionStats
if sc.GetSessionVars().InRestrictedSQL {
// For AutoAnalyze and HandleDDLEvent(ActionDropTablePartition), we need to use @@global.tidb_skip_missing_partition_stats
val, err1 := sc.GetSessionVars().GlobalVarsAccessor.GetGlobalSysVar(variable.TiDBAnalyzeSkipColumnTypes)
if err1 != nil {
logutil.BgLogger().Error("loading tidb_skip_missing_partition_stats failed", zap.Error(err1))
err = err1
return
}
skipMissingPartitionStats = variable.TiDBOptOn(val)
}
for _, def := range globalTableInfo.Partition.Definitions {
partitionID := def.ID
partitionTable, ok := getTableByPhysicalIDFn(is, partitionID)
Expand Down
65 changes: 52 additions & 13 deletions statistics/handle/handle.go
Original file line number Diff line number Diff line change
Expand Up @@ -324,16 +324,57 @@ func (h *Handle) Update(is infoschema.InfoSchema) error {
func (h *Handle) UpdateSessionVar() error {
h.mu.Lock()
defer h.mu.Unlock()
verInString, err := h.mu.ctx.GetSessionVars().GlobalVarsAccessor.GetGlobalSysVar(variable.TiDBAnalyzeVersion)
return UpdateSCtxVarsForStats(h.mu.ctx)
}

// UpdateSCtxVarsForStats updates all necessary variables that may affect the behavior of statistics.
func UpdateSCtxVarsForStats(sctx sessionctx.Context) error {
// analyzer version
verInString, err := sctx.GetSessionVars().GlobalVarsAccessor.GetGlobalSysVar(variable.TiDBAnalyzeVersion)
if err != nil {
return err
}
ver, err := strconv.ParseInt(verInString, 10, 64)
if err != nil {
return err
}
h.mu.ctx.GetSessionVars().AnalyzeVersion = int(ver)
return err
sctx.GetSessionVars().AnalyzeVersion = int(ver)

// enable historical stats
val, err := sctx.GetSessionVars().GlobalVarsAccessor.GetGlobalSysVar(variable.TiDBEnableHistoricalStats)
if err != nil {
return err
}
sctx.GetSessionVars().EnableHistoricalStats = variable.TiDBOptOn(val)

// partition mode
pruneMode, err := sctx.GetSessionVars().GlobalVarsAccessor.GetGlobalSysVar(variable.TiDBPartitionPruneMode)
if err != nil {
return err
}
sctx.GetSessionVars().PartitionPruneMode.Store(pruneMode)

// enable analyze snapshot
analyzeSnapshot, err := sctx.GetSessionVars().GlobalVarsAccessor.GetGlobalSysVar(variable.TiDBEnableAnalyzeSnapshot)
if err != nil {
return err
}
sctx.GetSessionVars().EnableAnalyzeSnapshot = variable.TiDBOptOn(analyzeSnapshot)

// enable skip column types
val, err = sctx.GetSessionVars().GlobalVarsAccessor.GetGlobalSysVar(variable.TiDBAnalyzeSkipColumnTypes)
if err != nil {
return err
}
sctx.GetSessionVars().AnalyzeSkipColumnTypes = variable.ParseAnalyzeSkipColumnTypes(val)

// skip missing partition stats
val, err = sctx.GetSessionVars().GlobalVarsAccessor.GetGlobalSysVar(variable.TiDBSkipMissingPartitionStats)
if err != nil {
return err
}
sctx.GetSessionVars().SkipMissingPartitionStats = variable.TiDBOptOn(val)
return nil
}

// MergePartitionStats2GlobalStatsByTableID merge the partition-level stats to global-level stats based on the tableID.
Expand Down Expand Up @@ -364,6 +405,9 @@ func (h *Handle) mergePartitionStats2GlobalStats(sc sessionctx.Context,
opts map[ast.AnalyzeOptionType]uint64, is infoschema.InfoSchema, globalTableInfo *model.TableInfo,
isIndex int, histIDs []int64,
allPartitionStats map[int64]*statistics.Table) (globalStats *globalstats.GlobalStats, err error) {
if err := UpdateSCtxVarsForStats(sc); err != nil {
return nil, err
}
return h.globalstatushandler.MergePartitionStats2GlobalStats(sc, opts, is, globalTableInfo, isIndex, histIDs, allPartitionStats, h.getTableByPhysicalID, h.loadTablePartitionStats)
}

Expand Down Expand Up @@ -1488,7 +1532,7 @@ func (h *Handle) CurrentPruneMode() variable.PartitionPruneMode {
func (h *Handle) RefreshVars() error {
h.mu.Lock()
defer h.mu.Unlock()
return h.mu.ctx.RefreshVars(context.Background())
return UpdateSCtxVarsForStats(h.mu.ctx)
}

// CheckAnalyzeVersion checks whether all the statistics versions of this table's columns and indexes are the same.
Expand Down Expand Up @@ -1691,19 +1735,14 @@ func (h *Handle) RecordHistoricalStatsToStorage(dbName string, tableInfo *model.
return version, nil
}

func checkHistoricalStatsEnable(sctx sessionctx.Context) (enable bool, err error) {
val, err := sctx.GetSessionVars().GlobalVarsAccessor.GetGlobalSysVar(variable.TiDBEnableHistoricalStats)
if err != nil {
return false, errors.Trace(err)
}
return variable.TiDBOptOn(val), nil
}

// CheckHistoricalStatsEnable is used to check whether TiDBEnableHistoricalStats is enabled.
func (h *Handle) CheckHistoricalStatsEnable() (enable bool, err error) {
h.mu.Lock()
defer h.mu.Unlock()
return checkHistoricalStatsEnable(h.mu.ctx)
if err := UpdateSCtxVarsForStats(h.mu.ctx); err != nil {
return false, err
}
return h.mu.ctx.GetSessionVars().EnableHistoricalStats, nil
}

// InsertAnalyzeJob inserts analyze job into mysql.analyze_jobs and gets job ID for further updating job.
Expand Down
5 changes: 2 additions & 3 deletions statistics/handle/historical_stats_handler.go
Original file line number Diff line number Diff line change
Expand Up @@ -43,11 +43,10 @@ func recordHistoricalStatsMeta(sctx sessionctx.Context, tableID int64, version u
if tableID == 0 || version == 0 {
return errors.Errorf("tableID %d, version %d are invalid", tableID, version)
}
historicalStatsEnabled, err := checkHistoricalStatsEnable(sctx)
if err != nil {
if err := UpdateSCtxVarsForStats(sctx); err != nil {
return errors.Errorf("check tidb_enable_historical_stats failed: %v", err)
}
if !historicalStatsEnabled {
if !sctx.GetSessionVars().EnableHistoricalStats {
return nil
}
ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnStats)
Expand Down
20 changes: 4 additions & 16 deletions statistics/handle/update.go
Original file line number Diff line number Diff line change
Expand Up @@ -752,16 +752,6 @@ func parseAnalyzePeriod(start, end string) (time.Time, time.Time, error) {
return s, e, err
}

func (h *Handle) getAnalyzeSnapshot() (bool, error) {
h.mu.Lock()
defer h.mu.Unlock()
analyzeSnapshot, err := h.mu.ctx.GetSessionVars().GlobalVarsAccessor.GetGlobalSysVar(variable.TiDBEnableAnalyzeSnapshot)
if err != nil {
return false, err
}
return variable.TiDBOptOn(analyzeSnapshot), nil
}

// HandleAutoAnalyze analyzes the newly created table or index.
func (h *Handle) HandleAutoAnalyze(is infoschema.InfoSchema) (analyzed bool) {
defer func() {
Expand All @@ -785,12 +775,10 @@ func (h *Handle) HandleAutoAnalyze(is infoschema.InfoSchema) (analyzed bool) {
if !timeutil.WithinDayTimePeriod(start, end, time.Now()) {
return false
}
pruneMode := h.CurrentPruneMode()
analyzeSnapshot, err := h.getAnalyzeSnapshot()
if err != nil {
logutil.BgLogger().Error("load tidb_enable_analyze_snapshot for auto analyze session failed", zap.String("category", "stats"), zap.Error(err))
return false
}
h.mu.Lock()
pruneMode := variable.PartitionPruneMode(h.mu.ctx.GetSessionVars().PartitionPruneMode.Load())
analyzeSnapshot := h.mu.ctx.GetSessionVars().EnableAnalyzeSnapshot
h.mu.Unlock()
rd := rand.New(rand.NewSource(time.Now().UnixNano())) // #nosec G404
rd.Shuffle(len(dbs), func(i, j int) {
dbs[i], dbs[j] = dbs[j], dbs[i]
Expand Down
5 changes: 0 additions & 5 deletions util/mock/context.go
Original file line number Diff line number Diff line change
Expand Up @@ -283,11 +283,6 @@ func (c *Context) RefreshTxnCtx(ctx context.Context) error {
return errors.Trace(c.NewTxn(ctx))
}

// RefreshVars implements the sessionctx.Context interface.
func (*Context) RefreshVars(_ context.Context) error {
return nil
}

// RollbackTxn indicates an expected call of RollbackTxn.
func (c *Context) RollbackTxn(_ context.Context) {
defer c.sessionVars.SetInTxn(false)
Expand Down

0 comments on commit 4cc6c65

Please sign in to comment.