Skip to content

Commit

Permalink
statistics: support gc outdated historical stats (#39788)
Browse files Browse the repository at this point in the history
  • Loading branch information
Yisaer authored Dec 13, 2022
1 parent cb435fe commit 760dbe1
Show file tree
Hide file tree
Showing 4 changed files with 75 additions and 0 deletions.
37 changes: 37 additions & 0 deletions executor/historical_stats_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,8 +19,10 @@ import (
"fmt"
"strconv"
"testing"
"time"

"github.com/pingcap/tidb/parser/model"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/statistics/handle"
"github.com/pingcap/tidb/testkit"
"github.com/stretchr/testify/require"
Expand Down Expand Up @@ -178,3 +180,38 @@ func TestGCHistoryStatsAfterDropTable(t *testing.T) {
tk.MustQuery(fmt.Sprintf("select count(*) from mysql.stats_history where table_id = '%d'",
tableInfo.Meta().ID)).Check(testkit.Rows("0"))
}

func TestGCOutdatedHistoryStats(t *testing.T) {
store, dom := testkit.CreateMockStoreAndDomain(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("set global tidb_enable_historical_stats = 1")
tk.MustExec("use test")
tk.MustExec("drop table if exists t")
tk.MustExec("create table t(a int, b varchar(10))")
tk.MustExec("analyze table test.t")
is := dom.InfoSchema()
tableInfo, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t"))
require.NoError(t, err)
// dump historical stats
h := dom.StatsHandle()
hsWorker := dom.GetHistoricalStatsWorker()
tblID := hsWorker.GetOneHistoricalStatsTable()
err = hsWorker.DumpHistoricalStats(tblID, h)
require.Nil(t, err)

// assert the records of history stats table
tk.MustQuery(fmt.Sprintf("select count(*) from mysql.stats_meta_history where table_id = '%d' order by create_time",
tableInfo.Meta().ID)).Check(testkit.Rows("1"))
tk.MustQuery(fmt.Sprintf("select count(*) from mysql.stats_history where table_id = '%d'",
tableInfo.Meta().ID)).Check(testkit.Rows("1"))

variable.HistoricalStatsDuration.Store(1 * time.Second)
time.Sleep(2 * time.Second)
err = dom.StatsHandle().ClearOutdatedHistoryStats()
require.NoError(t, err)
// assert the records of history stats table
tk.MustQuery(fmt.Sprintf("select count(*) from mysql.stats_meta_history where table_id = '%d' order by create_time",
tableInfo.Meta().ID)).Check(testkit.Rows("0"))
tk.MustQuery(fmt.Sprintf("select count(*) from mysql.stats_history where table_id = '%d'",
tableInfo.Meta().ID)).Check(testkit.Rows("0"))
}
12 changes: 12 additions & 0 deletions sessionctx/variable/sysvar.go
Original file line number Diff line number Diff line change
Expand Up @@ -1169,6 +1169,18 @@ var defaultSysVars = []*SysVar{
return nil
}},

{Scope: ScopeGlobal, Name: TiDBHistoricalStatsDuration, Value: DefTiDBHistoricalStatsDuration.String(), Type: TypeDuration, MinValue: int64(time.Minute * 10), MaxValue: uint64(time.Hour * 24 * 365),
GetGlobal: func(ctx context.Context, vars *SessionVars) (string, error) {
return HistoricalStatsDuration.Load().String(), nil
}, SetGlobal: func(ctx context.Context, vars *SessionVars, s string) error {
d, err := time.ParseDuration(s)
if err != nil {
return err
}
HistoricalStatsDuration.Store(d)
return nil
}},

/* The system variables below have GLOBAL and SESSION scope */
{Scope: ScopeGlobal | ScopeSession, Name: TiDBEnablePlanReplayerCapture, Value: BoolToOnOff(false), Type: TypeBool,
SetSession: func(s *SessionVars, val string) error {
Expand Down
4 changes: 4 additions & 0 deletions sessionctx/variable/tidb_vars.go
Original file line number Diff line number Diff line change
Expand Up @@ -895,6 +895,8 @@ const (
PasswordReuseHistory = "password_history"
// PasswordReuseTime limit how long passwords can be reused.
PasswordReuseTime = "password_reuse_interval"
// TiDBHistoricalStatsDuration indicates the duration to remain tidb historical stats
TiDBHistoricalStatsDuration = "tidb_historical_stats_duration"
)

// TiDB intentional limits
Expand Down Expand Up @@ -1150,6 +1152,7 @@ const (
DefPasswordReuseHistory = 0
DefPasswordReuseTime = 0
DefTiDBStoreBatchSize = 0
DefTiDBHistoricalStatsDuration = 7 * 24 * time.Hour
DefTiDBTTLJobRunInterval = "1h0m0s"
DefTiDBTTLJobScheduleWindowStartTime = "00:00 +0000"
DefTiDBTTLJobScheduleWindowEndTime = "23:59 +0000"
Expand Down Expand Up @@ -1230,6 +1233,7 @@ var (
PasswordHistory = atomic.NewInt64(DefPasswordReuseHistory)
PasswordReuseInterval = atomic.NewInt64(DefPasswordReuseTime)
IsSandBoxModeEnabled = atomic.NewBool(false)
HistoricalStatsDuration = atomic.NewDuration(DefTiDBHistoricalStatsDuration)
)

var (
Expand Down
22 changes: 22 additions & 0 deletions statistics/handle/gc.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ import (
"github.com/pingcap/errors"
"github.com/pingcap/tidb/infoschema"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/util/logutil"
"github.com/pingcap/tidb/util/mathutil"
"github.com/pingcap/tidb/util/sqlexec"
Expand Down Expand Up @@ -54,6 +55,11 @@ func (h *Handle) GCStats(is infoschema.InfoSchema, ddlLease time.Duration) error
return errors.Trace(err)
}
}
if err := h.ClearOutdatedHistoryStats(); err != nil {
logutil.BgLogger().Warn("failed to gc outdated historical stats",
zap.Duration("duration", variable.HistoricalStatsDuration.Load()),
zap.Error(err))
}
return h.removeDeletedExtendedStats(gcVer)
}

Expand Down Expand Up @@ -141,6 +147,22 @@ func (h *Handle) gcTableStats(is infoschema.InfoSchema, physicalID int64) error
return nil
}

// ClearOutdatedHistoryStats clear outdated historical stats
func (h *Handle) ClearOutdatedHistoryStats() error {
ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnStats)
h.mu.Lock()
defer h.mu.Unlock()
exec := h.mu.ctx.(sqlexec.SQLExecutor)
sql := "delete from mysql.stats_meta_history where NOW() - create_time >= %?"
_, err := exec.ExecuteInternal(ctx, sql, variable.HistoricalStatsDuration.Load().Seconds())
if err != nil {
return err
}
sql = "delete from mysql.stats_history where NOW() - create_time >= %? "
_, err = exec.ExecuteInternal(ctx, sql, variable.HistoricalStatsDuration.Load().Seconds())
return err
}

func (h *Handle) gcHistoryStatsFromKV(physicalID int64) error {
h.mu.Lock()
defer h.mu.Unlock()
Expand Down

0 comments on commit 760dbe1

Please sign in to comment.