diff --git a/executor/historical_stats_test.go b/executor/historical_stats_test.go index becb1e82212f8..9ddd34655369f 100644 --- a/executor/historical_stats_test.go +++ b/executor/historical_stats_test.go @@ -365,3 +365,33 @@ PARTITION p0 VALUES LESS THAN (6) require.NotNil(t, jsTable.Partitions["p0"]) require.NotNil(t, jsTable.Partitions["global"]) } + +func TestDumpHistoricalStatsFallback(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("set global tidb_enable_historical_stats = 0") + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec(`CREATE TABLE t (a int, b int, index idx(b)) +PARTITION BY RANGE ( a ) ( +PARTITION p0 VALUES LESS THAN (6) +)`) + // dump historical stats + tk.MustExec("analyze table t") + is := dom.InfoSchema() + tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + require.NotNil(t, tbl) + + // dump historical stats + hsWorker := dom.GetHistoricalStatsWorker() + tblID := hsWorker.GetOneHistoricalStatsTable() + // assert no historical stats task generated + require.Equal(t, tblID, int64(-1)) + tk.MustExec("set global tidb_enable_historical_stats = 1") + h := dom.StatsHandle() + jt, err := h.DumpHistoricalStatsBySnapshot("test", tbl.Meta(), oracle.GoTimeToTS(time.Now())) + require.NoError(t, err) + require.NotNil(t, jt) + require.False(t, jt.IsHistoricalStats) +} diff --git a/statistics/handle/dump.go b/statistics/handle/dump.go index da8603ea90573..016fc858857ca 100644 --- a/statistics/handle/dump.go +++ b/statistics/handle/dump.go @@ -18,7 +18,6 @@ import ( "bytes" "compress/gzip" "encoding/json" - "fmt" "io" "time" @@ -40,15 +39,16 @@ import ( // JSONTable is used for dumping statistics. type JSONTable struct { - DatabaseName string `json:"database_name"` - TableName string `json:"table_name"` - Columns map[string]*jsonColumn `json:"columns"` - Indices map[string]*jsonColumn `json:"indices"` - ExtStats []*jsonExtendedStats `json:"ext_stats"` - Count int64 `json:"count"` - ModifyCount int64 `json:"modify_count"` - Partitions map[string]*JSONTable `json:"partitions"` - Version uint64 `json:"version"` + IsHistoricalStats bool `json:"is_historical_stats"` + DatabaseName string `json:"database_name"` + TableName string `json:"table_name"` + Columns map[string]*jsonColumn `json:"columns"` + Indices map[string]*jsonColumn `json:"indices"` + ExtStats []*jsonExtendedStats `json:"ext_stats"` + Count int64 `json:"count"` + ModifyCount int64 `json:"modify_count"` + Partitions map[string]*JSONTable `json:"partitions"` + Version uint64 `json:"version"` } type jsonExtendedStats struct { @@ -142,6 +142,14 @@ var ( // DumpHistoricalStatsBySnapshot dumped json tables from mysql.stats_meta_history and mysql.stats_history func (h *Handle) DumpHistoricalStatsBySnapshot(dbName string, tableInfo *model.TableInfo, snapshot uint64) (jt *JSONTable, err error) { + historicalStatsEnabled, err := h.CheckHistoricalStatsEnable() + if err != nil { + return nil, errors.Errorf("check %v failed: %v", variable.TiDBEnableHistoricalStats, err) + } + if !historicalStatsEnabled { + return nil, errors.Errorf("%v should be enabled", variable.TiDBEnableHistoricalStats) + } + defer func() { if err == nil { dumpHistoricalStatsSuccessCounter.Inc() @@ -149,10 +157,9 @@ func (h *Handle) DumpHistoricalStatsBySnapshot(dbName string, tableInfo *model.T dumpHistoricalStatsFailedCounter.Inc() } }() - pi := tableInfo.GetPartitionInfo() if pi == nil { - return h.tableHistoricalStatsToJSON(tableInfo.ID, snapshot) + return h.getTableHistoricalStatsToJSONWithFallback(dbName, tableInfo, tableInfo.ID, snapshot) } jsonTbl := &JSONTable{ DatabaseName: dbName, @@ -160,27 +167,19 @@ func (h *Handle) DumpHistoricalStatsBySnapshot(dbName string, tableInfo *model.T Partitions: make(map[string]*JSONTable, len(pi.Definitions)), } for _, def := range pi.Definitions { - tbl, err := h.tableHistoricalStatsToJSON(def.ID, snapshot) + tbl, err := h.getTableHistoricalStatsToJSONWithFallback(dbName, tableInfo, def.ID, snapshot) if err != nil { return nil, errors.Trace(err) } - if tbl == nil { - continue - } jsonTbl.Partitions[def.Name.L] = tbl } - h.mu.Lock() - isDynamicMode := variable.PartitionPruneMode(h.mu.ctx.GetSessionVars().PartitionPruneMode.Load()) == variable.Dynamic - h.mu.Unlock() - if isDynamicMode { - tbl, err := h.tableHistoricalStatsToJSON(tableInfo.ID, snapshot) - if err != nil { - logutil.BgLogger().Warn("dump global historical stats failed", - zap.Int64("table-id", tableInfo.ID), - zap.String("table-name", tableInfo.Name.String())) - } else if tbl != nil { - jsonTbl.Partitions["global"] = tbl - } + tbl, err := h.getTableHistoricalStatsToJSONWithFallback(dbName, tableInfo, tableInfo.ID, snapshot) + if err != nil { + return nil, err + } + // dump its global-stats if existed + if tbl != nil { + jsonTbl.Partitions["global"] = tbl } return jsonTbl, nil } @@ -250,11 +249,23 @@ func GenJSONTableFromStats(dbName string, tableInfo *model.TableInfo, tbl *stati return jsonTbl, nil } -func (h *Handle) tableHistoricalStatsToJSON(physicalID int64, snapshot uint64) (*JSONTable, error) { - reader, err := h.getGlobalStatsReader(0) +// getTableHistoricalStatsToJSONWithFallback try to get table historical stats, if not exit, directly fallback to the latest stats +func (h *Handle) getTableHistoricalStatsToJSONWithFallback(dbName string, tableInfo *model.TableInfo, physicalID int64, snapshot uint64) (*JSONTable, error) { + jt, exist, err := h.tableHistoricalStatsToJSON(physicalID, snapshot) if err != nil { return nil, err } + if !exist { + return h.tableStatsToJSON(dbName, tableInfo, physicalID, 0) + } + return jt, nil +} + +func (h *Handle) tableHistoricalStatsToJSON(physicalID int64, snapshot uint64) (*JSONTable, bool, error) { + reader, err := h.getGlobalStatsReader(0) + if err != nil { + return nil, false, err + } defer func() { err1 := h.releaseGlobalStatsReader(reader) if err == nil && err1 != nil { @@ -265,33 +276,39 @@ func (h *Handle) tableHistoricalStatsToJSON(physicalID int64, snapshot uint64) ( // get meta version rows, _, err := reader.Read("select distinct version from mysql.stats_meta_history where table_id = %? and version <= %? order by version desc limit 1", physicalID, snapshot) if err != nil { - return nil, errors.AddStack(err) + return nil, false, errors.AddStack(err) } if len(rows) < 1 { - return nil, fmt.Errorf("failed to get records of stats_meta_history for table_id = %v, snapshot = %v", physicalID, snapshot) + logutil.BgLogger().Warn("failed to get records of stats_meta_history", + zap.Int64("table-id", physicalID), + zap.Uint64("snapshotTS", snapshot)) + return nil, false, nil } statsMetaVersion := rows[0].GetInt64(0) // get stats meta rows, _, err = reader.Read("select modify_count, count from mysql.stats_meta_history where table_id = %? and version = %?", physicalID, statsMetaVersion) if err != nil { - return nil, errors.AddStack(err) + return nil, false, errors.AddStack(err) } modifyCount, count := rows[0].GetInt64(0), rows[0].GetInt64(1) // get stats version rows, _, err = reader.Read("select distinct version from mysql.stats_history where table_id = %? and version <= %? order by version desc limit 1", physicalID, snapshot) if err != nil { - return nil, errors.AddStack(err) + return nil, false, errors.AddStack(err) } if len(rows) < 1 { - return nil, fmt.Errorf("failed to get record of stats_history for table_id = %v, snapshot = %v", physicalID, snapshot) + logutil.BgLogger().Warn("failed to get record of stats_history", + zap.Int64("table-id", physicalID), + zap.Uint64("snapshotTS", snapshot)) + return nil, false, nil } statsVersion := rows[0].GetInt64(0) // get stats rows, _, err = reader.Read("select stats_data from mysql.stats_history where table_id = %? and version = %? order by seq_no", physicalID, statsVersion) if err != nil { - return nil, errors.AddStack(err) + return nil, false, errors.AddStack(err) } blocks := make([][]byte, 0) for _, row := range rows { @@ -299,11 +316,12 @@ func (h *Handle) tableHistoricalStatsToJSON(physicalID int64, snapshot uint64) ( } jsonTbl, err := BlocksToJSONTable(blocks) if err != nil { - return nil, errors.AddStack(err) + return nil, false, errors.AddStack(err) } jsonTbl.Count = count jsonTbl.ModifyCount = modifyCount - return jsonTbl, nil + jsonTbl.IsHistoricalStats = true + return jsonTbl, true, nil } func (h *Handle) tableStatsToJSON(dbName string, tableInfo *model.TableInfo, physicalID int64, snapshot uint64) (*JSONTable, error) {