From afd0bc4a3d1c52c8451a2ec86137680b1fd74e17 Mon Sep 17 00:00:00 2001 From: yisaer Date: Mon, 5 Dec 2022 12:08:34 +0800 Subject: [PATCH 1/6] add testcase --- executor/analyzetest/analyze_test.go | 102 --------------- executor/historical_stats_test.go | 180 +++++++++++++++++++++++++++ statistics/handle/gc.go | 25 ++++ 3 files changed, 205 insertions(+), 102 deletions(-) create mode 100644 executor/historical_stats_test.go diff --git a/executor/analyzetest/analyze_test.go b/executor/analyzetest/analyze_test.go index 63df848541013..e3bf9d51d9260 100644 --- a/executor/analyzetest/analyze_test.go +++ b/executor/analyzetest/analyze_test.go @@ -16,7 +16,6 @@ package analyzetest import ( "context" - "encoding/json" "fmt" "strconv" "strings" @@ -2163,107 +2162,6 @@ func TestAnalyzeColumnsErrorAndWarning(t *testing.T) { } } -func TestRecordHistoryStatsAfterAnalyze(t *testing.T) { - store, dom := testkit.CreateMockStoreAndDomain(t) - - tk := testkit.NewTestKit(t, store) - tk.MustExec("set @@tidb_analyze_version = 2") - 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 varchar(10))") - - h := dom.StatsHandle() - is := dom.InfoSchema() - tableInfo, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - require.NoError(t, err) - - // 1. switch off the tidb_enable_historical_stats, and there is no records in table `mysql.stats_history` - rows := tk.MustQuery(fmt.Sprintf("select count(*) from mysql.stats_history where table_id = '%d'", tableInfo.Meta().ID)).Rows() - num, _ := strconv.Atoi(rows[0][0].(string)) - require.Equal(t, num, 0) - - tk.MustExec("analyze table t with 2 topn") - rows = tk.MustQuery(fmt.Sprintf("select count(*) from mysql.stats_history where table_id = '%d'", tableInfo.Meta().ID)).Rows() - num, _ = strconv.Atoi(rows[0][0].(string)) - require.Equal(t, num, 0) - - // 2. switch on the tidb_enable_historical_stats and do analyze - tk.MustExec("set global tidb_enable_historical_stats = 1") - defer tk.MustExec("set global tidb_enable_historical_stats = 0") - tk.MustExec("analyze table t with 2 topn") - // dump historical stats - hsWorker := dom.GetHistoricalStatsWorker() - tblID := hsWorker.GetOneHistoricalStatsTable() - err = hsWorker.DumpHistoricalStats(tblID, h) - require.Nil(t, err) - rows = tk.MustQuery(fmt.Sprintf("select count(*) from mysql.stats_history where table_id = '%d'", tableInfo.Meta().ID)).Rows() - num, _ = strconv.Atoi(rows[0][0].(string)) - require.GreaterOrEqual(t, num, 1) - - // 3. dump current stats json - dumpJSONTable, err := h.DumpStatsToJSON("test", tableInfo.Meta(), nil, true) - require.NoError(t, err) - jsOrigin, _ := json.Marshal(dumpJSONTable) - - // 4. get the historical stats json - rows = tk.MustQuery(fmt.Sprintf("select * from mysql.stats_history where table_id = '%d' and create_time = ("+ - "select create_time from mysql.stats_history where table_id = '%d' order by create_time desc limit 1) "+ - "order by seq_no", tableInfo.Meta().ID, tableInfo.Meta().ID)).Rows() - num = len(rows) - require.GreaterOrEqual(t, num, 1) - data := make([][]byte, num) - for i, row := range rows { - data[i] = []byte(row[1].(string)) - } - jsonTbl, err := handle.BlocksToJSONTable(data) - require.NoError(t, err) - jsCur, err := json.Marshal(jsonTbl) - require.NoError(t, err) - // 5. historical stats must be equal to the current stats - require.JSONEq(t, string(jsOrigin), string(jsCur)) -} - -func TestRecordHistoryStatsMetaAfterAnalyze(t *testing.T) { - store, dom := testkit.CreateMockStoreAndDomain(t) - - tk := testkit.NewTestKit(t, store) - tk.MustExec("set @@tidb_analyze_version = 2") - 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)") - tk.MustExec("analyze table test.t") - - h := dom.StatsHandle() - is := dom.InfoSchema() - tableInfo, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - require.NoError(t, err) - - // 1. switch off the tidb_enable_historical_stats, and there is no record in table `mysql.stats_meta_history` - tk.MustQuery(fmt.Sprintf("select count(*) from mysql.stats_meta_history where table_id = '%d'", tableInfo.Meta().ID)).Check(testkit.Rows("0")) - // insert demo tuples, and there is no record either. - insertNums := 5 - for i := 0; i < insertNums; i++ { - tk.MustExec("insert into test.t (a,b) values (1,1), (2,2), (3,3)") - err := h.DumpStatsDeltaToKV(handle.DumpDelta) - require.NoError(t, err) - } - tk.MustQuery(fmt.Sprintf("select count(*) from mysql.stats_meta_history where table_id = '%d'", tableInfo.Meta().ID)).Check(testkit.Rows("0")) - - // 2. switch on the tidb_enable_historical_stats and insert tuples to produce count/modifyCount delta change. - tk.MustExec("set global tidb_enable_historical_stats = 1") - defer tk.MustExec("set global tidb_enable_historical_stats = 0") - - for i := 0; i < insertNums; i++ { - tk.MustExec("insert into test.t (a,b) values (1,1), (2,2), (3,3)") - err := h.DumpStatsDeltaToKV(handle.DumpDelta) - require.NoError(t, err) - } - tk.MustQuery(fmt.Sprintf("select modify_count, count from mysql.stats_meta_history where table_id = '%d' order by create_time", tableInfo.Meta().ID)).Sort().Check( - testkit.Rows("18 18", "21 21", "24 24", "27 27", "30 30")) -} - func checkAnalyzeStatus(t *testing.T, tk *testkit.TestKit, jobInfo, status, failReason, comment string, timeLimit int64) { rows := tk.MustQuery("show analyze status where table_schema = 'test' and table_name = 't' and partition_name = ''").Rows() require.Equal(t, 1, len(rows), comment) diff --git a/executor/historical_stats_test.go b/executor/historical_stats_test.go new file mode 100644 index 0000000000000..20d11d6e4a88c --- /dev/null +++ b/executor/historical_stats_test.go @@ -0,0 +1,180 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package executor_test + +import ( + "encoding/json" + "fmt" + "strconv" + "testing" + + "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/statistics/handle" + "github.com/pingcap/tidb/testkit" + "github.com/stretchr/testify/require" +) + +func TestRecordHistoryStatsAfterAnalyze(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + + tk := testkit.NewTestKit(t, store) + tk.MustExec("set @@tidb_analyze_version = 2") + 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 varchar(10))") + + h := dom.StatsHandle() + is := dom.InfoSchema() + tableInfo, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + + // 1. switch off the tidb_enable_historical_stats, and there is no records in table `mysql.stats_history` + rows := tk.MustQuery(fmt.Sprintf("select count(*) from mysql.stats_history where table_id = '%d'", tableInfo.Meta().ID)).Rows() + num, _ := strconv.Atoi(rows[0][0].(string)) + require.Equal(t, num, 0) + + tk.MustExec("analyze table t with 2 topn") + rows = tk.MustQuery(fmt.Sprintf("select count(*) from mysql.stats_history where table_id = '%d'", tableInfo.Meta().ID)).Rows() + num, _ = strconv.Atoi(rows[0][0].(string)) + require.Equal(t, num, 0) + + // 2. switch on the tidb_enable_historical_stats and do analyze + tk.MustExec("set global tidb_enable_historical_stats = 1") + defer tk.MustExec("set global tidb_enable_historical_stats = 0") + tk.MustExec("analyze table t with 2 topn") + // dump historical stats + hsWorker := dom.GetHistoricalStatsWorker() + tblID := hsWorker.GetOneHistoricalStatsTable() + err = hsWorker.DumpHistoricalStats(tblID, h) + require.Nil(t, err) + rows = tk.MustQuery(fmt.Sprintf("select count(*) from mysql.stats_history where table_id = '%d'", tableInfo.Meta().ID)).Rows() + num, _ = strconv.Atoi(rows[0][0].(string)) + require.GreaterOrEqual(t, num, 1) + + // 3. dump current stats json + dumpJSONTable, err := h.DumpStatsToJSON("test", tableInfo.Meta(), nil, true) + require.NoError(t, err) + jsOrigin, _ := json.Marshal(dumpJSONTable) + + // 4. get the historical stats json + rows = tk.MustQuery(fmt.Sprintf("select * from mysql.stats_history where table_id = '%d' and create_time = ("+ + "select create_time from mysql.stats_history where table_id = '%d' order by create_time desc limit 1) "+ + "order by seq_no", tableInfo.Meta().ID, tableInfo.Meta().ID)).Rows() + num = len(rows) + require.GreaterOrEqual(t, num, 1) + data := make([][]byte, num) + for i, row := range rows { + data[i] = []byte(row[1].(string)) + } + jsonTbl, err := handle.BlocksToJSONTable(data) + require.NoError(t, err) + jsCur, err := json.Marshal(jsonTbl) + require.NoError(t, err) + // 5. historical stats must be equal to the current stats + require.JSONEq(t, string(jsOrigin), string(jsCur)) +} + +func TestRecordHistoryStatsMetaAfterAnalyze(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + + tk := testkit.NewTestKit(t, store) + tk.MustExec("set @@tidb_analyze_version = 2") + 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)") + tk.MustExec("analyze table test.t") + + h := dom.StatsHandle() + is := dom.InfoSchema() + tableInfo, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + + // 1. switch off the tidb_enable_historical_stats, and there is no record in table `mysql.stats_meta_history` + tk.MustQuery(fmt.Sprintf("select count(*) from mysql.stats_meta_history where table_id = '%d'", tableInfo.Meta().ID)).Check(testkit.Rows("0")) + // insert demo tuples, and there is no record either. + insertNums := 5 + for i := 0; i < insertNums; i++ { + tk.MustExec("insert into test.t (a,b) values (1,1), (2,2), (3,3)") + err := h.DumpStatsDeltaToKV(handle.DumpDelta) + require.NoError(t, err) + } + tk.MustQuery(fmt.Sprintf("select count(*) from mysql.stats_meta_history where table_id = '%d'", tableInfo.Meta().ID)).Check(testkit.Rows("0")) + + // 2. switch on the tidb_enable_historical_stats and insert tuples to produce count/modifyCount delta change. + tk.MustExec("set global tidb_enable_historical_stats = 1") + defer tk.MustExec("set global tidb_enable_historical_stats = 0") + + for i := 0; i < insertNums; i++ { + tk.MustExec("insert into test.t (a,b) values (1,1), (2,2), (3,3)") + err := h.DumpStatsDeltaToKV(handle.DumpDelta) + require.NoError(t, err) + } + tk.MustQuery(fmt.Sprintf("select modify_count, count from mysql.stats_meta_history where table_id = '%d' order by create_time", tableInfo.Meta().ID)).Sort().Check( + testkit.Rows("18 18", "21 21", "24 24", "27 27", "30 30")) + + // assert delete + tk.MustExec("delete from test.t where test.t.a = 1") + err = h.DumpStatsDeltaToKV(handle.DumpAll) + require.NoError(t, err) + tk.MustQuery(fmt.Sprintf("select modify_count, count from mysql.stats_meta where table_id = '%d' order by create_time desc", tableInfo.Meta().ID)).Sort().Check( + testkit.Rows("40 20")) + tk.MustQuery(fmt.Sprintf("select modify_count, count from mysql.stats_meta_history where table_id = '%d' order by create_time desc limit 1", tableInfo.Meta().ID)).Sort().Check( + testkit.Rows("40 20")) + + // assert update + tk.MustExec("update test.t set test.t.b = 4 where test.t.a = 2") + err = h.DumpStatsDeltaToKV(handle.DumpAll) + require.NoError(t, err) + tk.MustQuery(fmt.Sprintf("select modify_count, count from mysql.stats_meta where table_id = '%d' order by create_time desc", tableInfo.Meta().ID)).Sort().Check( + testkit.Rows("50 20")) + tk.MustQuery(fmt.Sprintf("select modify_count, count from mysql.stats_meta_history where table_id = '%d' order by create_time desc limit 1", tableInfo.Meta().ID)).Sort().Check( + testkit.Rows("50 20")) +} + +func TestGCHistoryStatsAfterDropTable(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")) + // drop the table and gc stats + tk.MustExec("drop table t") + h.GCStats(is, 0) + + // assert stats_history tables delete the record of dropped 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")) +} diff --git a/statistics/handle/gc.go b/statistics/handle/gc.go index 1babb4321eb9e..6ef3ee3f7c12c 100644 --- a/statistics/handle/gc.go +++ b/statistics/handle/gc.go @@ -50,6 +50,9 @@ func (h *Handle) GCStats(is infoschema.InfoSchema, ddlLease time.Duration) error if err := h.gcTableStats(is, row.GetInt64(0)); err != nil { return errors.Trace(err) } + if err := h.gcHistoryStatsFromKV(row.GetInt64(0)); err != nil { + return errors.Trace(err) + } } return h.removeDeletedExtendedStats(gcVer) } @@ -138,6 +141,28 @@ func (h *Handle) gcTableStats(is infoschema.InfoSchema, physicalID int64) error return nil } +func (h *Handle) gcHistoryStatsFromKV(physicalID int64) error { + h.mu.Lock() + defer h.mu.Unlock() + exec := h.mu.ctx.(sqlexec.SQLExecutor) + ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnStats) + _, err := exec.ExecuteInternal(ctx, "begin pessimistic") + if err != nil { + return errors.Trace(err) + } + defer func() { + err = finishTransaction(ctx, exec, err) + }() + sql := "delete from mysql.stats_history where table_id = %?" + _, err = exec.ExecuteInternal(ctx, sql, physicalID) + if err != nil { + return errors.Trace(err) + } + sql = "delete from mysql.stats_meta_history where table_id = %?" + _, err = exec.ExecuteInternal(ctx, sql, physicalID) + return err +} + // deleteHistStatsFromKV deletes all records about a column or an index and updates version. func (h *Handle) deleteHistStatsFromKV(physicalID int64, histID int64, isIndex int) (err error) { h.mu.Lock() From d41dfd2773cbad06559dff906f65b4811334f8f6 Mon Sep 17 00:00:00 2001 From: yisaer Date: Mon, 5 Dec 2022 12:12:49 +0800 Subject: [PATCH 2/6] bazel prepare --- executor/BUILD.bazel | 1 + 1 file changed, 1 insertion(+) diff --git a/executor/BUILD.bazel b/executor/BUILD.bazel index 0d0f285ab3710..7b1a4d603f993 100644 --- a/executor/BUILD.bazel +++ b/executor/BUILD.bazel @@ -278,6 +278,7 @@ go_test( "explainfor_test.go", "grant_test.go", "hash_table_test.go", + "historical_stats_test.go", "hot_regions_history_table_test.go", "index_advise_test.go", "index_lookup_join_test.go", From 9275e16f62c1bed0068e6732c08453fb3d510d17 Mon Sep 17 00:00:00 2001 From: yisaer Date: Tue, 6 Dec 2022 12:55:40 +0800 Subject: [PATCH 3/6] add test --- server/statistics_handler.go | 19 +----- server/statistics_handler_test.go | 6 ++ statistics/handle/ddl.go | 4 +- statistics/handle/dump.go | 99 +++++++++++++++++++++++++++++++ statistics/handle/handle.go | 29 ++++++--- statistics/handle/update.go | 2 +- 6 files changed, 129 insertions(+), 30 deletions(-) diff --git a/server/statistics_handler.go b/server/statistics_handler.go index 09de26810a332..d33532f597a1a 100644 --- a/server/statistics_handler.go +++ b/server/statistics_handler.go @@ -25,7 +25,6 @@ import ( "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/gcutil" "github.com/tikv/client-go/v2/oracle" ) @@ -105,16 +104,6 @@ func (sh StatsHistoryHandler) ServeHTTP(w http.ResponseWriter, req *http.Request return } defer se.Close() - - dumpPartitionStats := true - if len(params[pDumpPartitionStats]) > 0 { - dumpPartitionStats, err = strconv.ParseBool(params[pDumpPartitionStats]) - if err != nil { - writeError(w, err) - return - } - } - se.GetSessionVars().StmtCtx.TimeZone = time.Local t, err := types.ParseTime(se.GetSessionVars().StmtCtx, params[pSnapshot], mysql.TypeTimestamp, 6) if err != nil { @@ -127,12 +116,6 @@ func (sh StatsHistoryHandler) ServeHTTP(w http.ResponseWriter, req *http.Request return } snapshot := oracle.GoTimeToTS(t1) - err = gcutil.ValidateSnapshot(se, snapshot) - if err != nil { - writeError(w, err) - return - } - is, err := sh.do.GetSnapshotInfoSchema(snapshot) if err != nil { writeError(w, err) @@ -144,7 +127,7 @@ func (sh StatsHistoryHandler) ServeHTTP(w http.ResponseWriter, req *http.Request writeError(w, err) return } - js, err := h.DumpStatsToJSONBySnapshot(params[pDBName], tbl.Meta(), snapshot, dumpPartitionStats) + js, err := h.DumpHistoricalStatsBySnapshot(params[pDBName], tbl.Meta(), snapshot) if err != nil { writeError(w, err) } else { diff --git a/server/statistics_handler_test.go b/server/statistics_handler_test.go index a4d557f45be88..e0ecc7ba853f0 100644 --- a/server/statistics_handler_test.go +++ b/server/statistics_handler_test.go @@ -25,6 +25,7 @@ import ( "github.com/go-sql-driver/mysql" "github.com/gorilla/mux" + "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/statistics/handle" "github.com/pingcap/tidb/testkit" @@ -59,6 +60,10 @@ func TestDumpStatsAPI(t *testing.T) { statsHandler := &StatsHandler{dom} prepareData(t, client, statsHandler) + tableInfo, err := dom.InfoSchema().TableByName(model.NewCIStr("tidb"), model.NewCIStr("test")) + require.NoError(t, err) + err = dom.GetHistoricalStatsWorker().DumpHistoricalStats(tableInfo.Meta().ID, dom.StatsHandle()) + require.NoError(t, err) router := mux.NewRouter() router.Handle("/stats/dump/{db}/{table}", statsHandler) @@ -168,6 +173,7 @@ func prepareData(t *testing.T, client *testServerClient, statHandle *StatsHandle tk.MustExec("insert test values (1, 's')") require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) tk.MustExec("analyze table test") + tk.MustExec("set global tidb_enable_historical_stats = 1") tk.MustExec("insert into test(a,b) values (1, 'v'),(3, 'vvv'),(5, 'vv')") is := statHandle.do.InfoSchema() require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) diff --git a/statistics/handle/ddl.go b/statistics/handle/ddl.go index 84e0a087a13d3..d5926c84f49c0 100644 --- a/statistics/handle/ddl.go +++ b/statistics/handle/ddl.go @@ -221,7 +221,7 @@ func (h *Handle) insertTableStats2KV(info *model.TableInfo, physicalID int64) (e statsVer := uint64(0) defer func() { if err == nil && statsVer != 0 { - err = h.recordHistoricalStatsMeta(physicalID, statsVer) + h.recordHistoricalStatsMeta(physicalID, statsVer) } }() h.mu.Lock() @@ -263,7 +263,7 @@ func (h *Handle) insertColStats2KV(physicalID int64, colInfos []*model.ColumnInf statsVer := uint64(0) defer func() { if err == nil && statsVer != 0 { - err = h.recordHistoricalStatsMeta(physicalID, statsVer) + h.recordHistoricalStatsMeta(physicalID, statsVer) } }() h.mu.Lock() diff --git a/statistics/handle/dump.go b/statistics/handle/dump.go index 8f9b070ef2f6f..c3bc72a450afb 100644 --- a/statistics/handle/dump.go +++ b/statistics/handle/dump.go @@ -18,6 +18,7 @@ import ( "bytes" "compress/gzip" "encoding/json" + "fmt" "io/ioutil" "time" @@ -130,6 +131,42 @@ func (h *Handle) DumpStatsToJSON(dbName string, tableInfo *model.TableInfo, return h.DumpStatsToJSONBySnapshot(dbName, tableInfo, snapshot, dumpPartitionStats) } +// DumpHistoricalStatsBySnapshot dumped json tables from mysql.stats_meta_history and mysql.stats_history +func (h *Handle) DumpHistoricalStatsBySnapshot(dbName string, tableInfo *model.TableInfo, snapshot uint64) (*JSONTable, error) { + pi := tableInfo.GetPartitionInfo() + if pi == nil { + return h.tableHistoricalStatsToJSON(tableInfo.ID, snapshot) + } + jsonTbl := &JSONTable{ + DatabaseName: dbName, + TableName: tableInfo.Name.L, + Partitions: make(map[string]*JSONTable, len(pi.Definitions)), + } + for _, def := range pi.Definitions { + tbl, err := h.tableHistoricalStatsToJSON(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 { + return nil, errors.Trace(err) + } + if tbl != nil { + jsonTbl.Partitions["global"] = tbl + } + } + return jsonTbl, nil +} + // DumpStatsToJSONBySnapshot dumps statistic to json. func (h *Handle) DumpStatsToJSONBySnapshot(dbName string, tableInfo *model.TableInfo, snapshot uint64, dumpPartitionStats bool) (*JSONTable, error) { h.mu.Lock() @@ -194,6 +231,68 @@ 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) + if err != nil { + return nil, err + } + defer func() { + err1 := h.releaseGlobalStatsReader(reader) + if err == nil && err1 != nil { + err = err1 + } + }() + + // 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) + } + if len(rows) < 1 { + return nil, errors.New(fmt.Sprintf("failed to get version of stats_meta_history for table_id = %v, snapshot = %v", physicalID, snapshot)) + } + 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) + } + if len(rows) < 1 { + return nil, errors.New(fmt.Sprintf("failed to get records of stats_meta_history for table_id = %v, version = %v", physicalID, statsMetaVersion)) + } + 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) + } + if len(rows) < 1 { + return nil, errors.New(fmt.Sprintf("failed to get version of stats_history for table_id = %v, snapshot = %v", physicalID, snapshot)) + } + 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) + } + if len(rows) < 1 { + return nil, errors.New(fmt.Sprintf("failed to get records of stats_history for table_id = %v, version = %v", physicalID, statsVersion)) + } + blocks := make([][]byte, 0) + for _, row := range rows { + blocks = append(blocks, row.GetBytes(0)) + } + jsonTbl, err := BlocksToJSONTable(blocks) + if err != nil { + return nil, errors.AddStack(err) + } + jsonTbl.Count = count + jsonTbl.ModifyCount = modifyCount + return jsonTbl, nil +} + func (h *Handle) tableStatsToJSON(dbName string, tableInfo *model.TableInfo, physicalID int64, snapshot uint64) (*JSONTable, error) { tbl, err := h.TableStatsFromStorage(tableInfo, physicalID, true, snapshot) if err != nil || tbl == nil { diff --git a/statistics/handle/handle.go b/statistics/handle/handle.go index b483d23c1cdfb..1d67a430ba730 100644 --- a/statistics/handle/handle.go +++ b/statistics/handle/handle.go @@ -1619,7 +1619,7 @@ func (h *Handle) SaveTableStatsToStorage(results *statistics.AnalyzeResults, ana statsVer := uint64(0) defer func() { if err == nil && statsVer != 0 { - err = h.recordHistoricalStatsMeta(tableID, statsVer) + h.recordHistoricalStatsMeta(tableID, statsVer) } }() h.mu.Lock() @@ -1634,7 +1634,12 @@ func SaveTableStatsToStorage(sctx sessionctx.Context, results *statistics.Analyz statsVer := uint64(0) defer func() { if err == nil && statsVer != 0 { - err = recordHistoricalStatsMeta(sctx, tableID, statsVer) + if err1 := recordHistoricalStatsMeta(sctx, tableID, statsVer); err1 != nil { + logutil.BgLogger().Error("record historical stats meta failed", + zap.Int64("table-id", tableID), + zap.Uint64("version", statsVer), + zap.Error(err1)) + } } }() ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnStats) @@ -1808,7 +1813,7 @@ func (h *Handle) SaveStatsToStorage(tableID int64, count, modifyCount int64, isI statsVer := uint64(0) defer func() { if err == nil && statsVer != 0 { - err = h.recordHistoricalStatsMeta(tableID, statsVer) + h.recordHistoricalStatsMeta(tableID, statsVer) } }() h.mu.Lock() @@ -1887,7 +1892,7 @@ func (h *Handle) SaveMetaToStorage(tableID, count, modifyCount int64) (err error statsVer := uint64(0) defer func() { if err == nil && statsVer != 0 { - err = h.recordHistoricalStatsMeta(tableID, statsVer) + h.recordHistoricalStatsMeta(tableID, statsVer) } }() h.mu.Lock() @@ -2093,7 +2098,7 @@ func (h *Handle) InsertExtendedStats(statsName string, colIDs []int64, tp int, t statsVer := uint64(0) defer func() { if err == nil && statsVer != 0 { - err = h.recordHistoricalStatsMeta(tableID, statsVer) + h.recordHistoricalStatsMeta(tableID, statsVer) } }() slices.Sort(colIDs) @@ -2164,7 +2169,7 @@ func (h *Handle) MarkExtendedStatsDeleted(statsName string, tableID int64, ifExi statsVer := uint64(0) defer func() { if err == nil && statsVer != 0 { - err = h.recordHistoricalStatsMeta(tableID, statsVer) + h.recordHistoricalStatsMeta(tableID, statsVer) } }() ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnStats) @@ -2377,7 +2382,7 @@ func (h *Handle) SaveExtendedStatsToStorage(tableID int64, extStats *statistics. statsVer := uint64(0) defer func() { if err == nil && statsVer != 0 { - err = h.recordHistoricalStatsMeta(tableID, statsVer) + h.recordHistoricalStatsMeta(tableID, statsVer) } }() if extStats == nil || len(extStats.Stats) == 0 { @@ -2676,10 +2681,16 @@ func recordHistoricalStatsMeta(sctx sessionctx.Context, tableID int64, version u return nil } -func (h *Handle) recordHistoricalStatsMeta(tableID int64, version uint64) error { +func (h *Handle) recordHistoricalStatsMeta(tableID int64, version uint64) { h.mu.Lock() defer h.mu.Unlock() - return recordHistoricalStatsMeta(h.mu.ctx, tableID, version) + err := recordHistoricalStatsMeta(h.mu.ctx, tableID, version) + if err != nil { + logutil.BgLogger().Error("record historical stats meta failed", + zap.Int64("table-id", tableID), + zap.Uint64("version", version), + zap.Error(err)) + } } // InsertAnalyzeJob inserts analyze job into mysql.analyze_jobs and gets job ID for further updating job. diff --git a/statistics/handle/update.go b/statistics/handle/update.go index 1da73f188e3a1..aaebf78b323d3 100644 --- a/statistics/handle/update.go +++ b/statistics/handle/update.go @@ -524,7 +524,7 @@ func (h *Handle) dumpTableStatCountToKV(id int64, delta variable.TableDelta) (up statsVer := uint64(0) defer func() { if err == nil && statsVer != 0 { - err = h.recordHistoricalStatsMeta(id, statsVer) + h.recordHistoricalStatsMeta(id, statsVer) } }() if delta.Count == 0 { From 455b4f8bbd2ee9958a0d4544592e6b13bde785a6 Mon Sep 17 00:00:00 2001 From: yisaer Date: Wed, 7 Dec 2022 13:19:47 +0800 Subject: [PATCH 4/6] check enabled --- server/statistics_handler.go | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/server/statistics_handler.go b/server/statistics_handler.go index d33532f597a1a..1844905e0a782 100644 --- a/server/statistics_handler.go +++ b/server/statistics_handler.go @@ -15,6 +15,7 @@ package server import ( + "fmt" "net/http" "strconv" "time" @@ -24,6 +25,7 @@ import ( "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/session" + "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/types" "github.com/tikv/client-go/v2/oracle" ) @@ -104,6 +106,16 @@ func (sh StatsHistoryHandler) ServeHTTP(w http.ResponseWriter, req *http.Request return } defer se.Close() + enabeld, err := sh.do.StatsHandle().CheckHistoricalStatsEnable() + if err != nil { + writeError(w, err) + return + } + if !enabeld { + writeError(w, fmt.Errorf("%v should be enabled", variable.TiDBEnableHistoricalStats)) + return + } + se.GetSessionVars().StmtCtx.TimeZone = time.Local t, err := types.ParseTime(se.GetSessionVars().StmtCtx, params[pSnapshot], mysql.TypeTimestamp, 6) if err != nil { From 71bf40aa4a98f20b761ba5eac80115629ff298d4 Mon Sep 17 00:00:00 2001 From: yisaer Date: Wed, 7 Dec 2022 13:30:42 +0800 Subject: [PATCH 5/6] fix lint --- statistics/handle/dump.go | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/statistics/handle/dump.go b/statistics/handle/dump.go index c3bc72a450afb..f37d325cf70d7 100644 --- a/statistics/handle/dump.go +++ b/statistics/handle/dump.go @@ -249,7 +249,7 @@ func (h *Handle) tableHistoricalStatsToJSON(physicalID int64, snapshot uint64) ( return nil, errors.AddStack(err) } if len(rows) < 1 { - return nil, errors.New(fmt.Sprintf("failed to get version of stats_meta_history for table_id = %v, snapshot = %v", physicalID, snapshot)) + return nil, fmt.Errorf("failed to get version of stats_meta_history for table_id = %v, snapshot = %v", physicalID, snapshot) } statsMetaVersion := rows[0].GetInt64(0) // get stats meta @@ -258,7 +258,7 @@ func (h *Handle) tableHistoricalStatsToJSON(physicalID int64, snapshot uint64) ( return nil, errors.AddStack(err) } if len(rows) < 1 { - return nil, errors.New(fmt.Sprintf("failed to get records of stats_meta_history for table_id = %v, version = %v", physicalID, statsMetaVersion)) + return nil, fmt.Errorf("failed to get records of stats_meta_history for table_id = %v, version = %v", physicalID, statsMetaVersion) } modifyCount, count := rows[0].GetInt64(0), rows[0].GetInt64(1) @@ -268,7 +268,7 @@ func (h *Handle) tableHistoricalStatsToJSON(physicalID int64, snapshot uint64) ( return nil, errors.AddStack(err) } if len(rows) < 1 { - return nil, errors.New(fmt.Sprintf("failed to get version of stats_history for table_id = %v, snapshot = %v", physicalID, snapshot)) + return nil, fmt.Errorf("failed to get version of stats_history for table_id = %v, snapshot = %v", physicalID, snapshot) } statsVersion := rows[0].GetInt64(0) @@ -278,7 +278,7 @@ func (h *Handle) tableHistoricalStatsToJSON(physicalID int64, snapshot uint64) ( return nil, errors.AddStack(err) } if len(rows) < 1 { - return nil, errors.New(fmt.Sprintf("failed to get records of stats_history for table_id = %v, version = %v", physicalID, statsVersion)) + return nil, fmt.Errorf("failed to get records of stats_history for table_id = %v, version = %v", physicalID, statsVersion) } blocks := make([][]byte, 0) for _, row := range rows { From 5356a46673f7fac34a65f3974868ba1718818753 Mon Sep 17 00:00:00 2001 From: yisaer Date: Wed, 7 Dec 2022 13:35:45 +0800 Subject: [PATCH 6/6] revise error --- statistics/handle/dump.go | 10 ++-------- 1 file changed, 2 insertions(+), 8 deletions(-) diff --git a/statistics/handle/dump.go b/statistics/handle/dump.go index f37d325cf70d7..546916751c32d 100644 --- a/statistics/handle/dump.go +++ b/statistics/handle/dump.go @@ -249,7 +249,7 @@ func (h *Handle) tableHistoricalStatsToJSON(physicalID int64, snapshot uint64) ( return nil, errors.AddStack(err) } if len(rows) < 1 { - return nil, fmt.Errorf("failed to get version of stats_meta_history for table_id = %v, snapshot = %v", physicalID, snapshot) + return nil, fmt.Errorf("failed to get records of stats_meta_history for table_id = %v, snapshot = %v", physicalID, snapshot) } statsMetaVersion := rows[0].GetInt64(0) // get stats meta @@ -257,9 +257,6 @@ func (h *Handle) tableHistoricalStatsToJSON(physicalID int64, snapshot uint64) ( if err != nil { return nil, errors.AddStack(err) } - if len(rows) < 1 { - return nil, fmt.Errorf("failed to get records of stats_meta_history for table_id = %v, version = %v", physicalID, statsMetaVersion) - } modifyCount, count := rows[0].GetInt64(0), rows[0].GetInt64(1) // get stats version @@ -268,7 +265,7 @@ func (h *Handle) tableHistoricalStatsToJSON(physicalID int64, snapshot uint64) ( return nil, errors.AddStack(err) } if len(rows) < 1 { - return nil, fmt.Errorf("failed to get version of stats_history for table_id = %v, snapshot = %v", physicalID, snapshot) + return nil, fmt.Errorf("failed to get record of stats_history for table_id = %v, snapshot = %v", physicalID, snapshot) } statsVersion := rows[0].GetInt64(0) @@ -277,9 +274,6 @@ func (h *Handle) tableHistoricalStatsToJSON(physicalID int64, snapshot uint64) ( if err != nil { return nil, errors.AddStack(err) } - if len(rows) < 1 { - return nil, fmt.Errorf("failed to get records of stats_history for table_id = %v, version = %v", physicalID, statsVersion) - } blocks := make([][]byte, 0) for _, row := range rows { blocks = append(blocks, row.GetBytes(0))