diff --git a/docs/tidb_http_api.md b/docs/tidb_http_api.md index 1db55ec710643..e2a1bef7fcb7a 100644 --- a/docs/tidb_http_api.md +++ b/docs/tidb_http_api.md @@ -396,3 +396,18 @@ timezone.* Param: - seconds: profile time(s), default is 10s. + +1. Get statistics data of specified table. + + ```shell + curl http://{TiDBIP}:10080/stats/dump/{db}/{table} + ``` + +1. Get statistics data of specific table and timestamp. + + ```shell + curl http://{TiDBIP}:10080/stats/dump/{db}/{table}/{yyyyMMddHHmmss} + ``` + ```shell + curl http://{TiDBIP}:10080/stats/dump/{db}/{table}/{yyyy-MM-dd HH:mm:ss} + ``` diff --git a/server/http_handler.go b/server/http_handler.go index 2e3814ab24656..d856e80c930e6 100644 --- a/server/http_handler.go +++ b/server/http_handler.go @@ -72,6 +72,7 @@ const ( pColumnFlag = "colFlag" pColumnLen = "colLen" pRowBin = "rowBin" + pSnapshot = "snapshot" ) // For query string diff --git a/server/http_status.go b/server/http_status.go index af90a49e54e00..c7ca853b252de 100644 --- a/server/http_status.go +++ b/server/http_status.go @@ -58,6 +58,7 @@ func (s *Server) startHTTPServer() { // HTTP path for dump statistics. router.Handle("/stats/dump/{db}/{table}", s.newStatsHandler()).Name("StatsDump") + router.Handle("/stats/dump/{db}/{table}/{snapshot}", s.newStatsHistoryHandler()).Name("StatsHistoryDump") router.Handle("/settings", settingsHandler{}).Name("Settings") router.Handle("/reload-config", configReloadHandler{}).Name("ConfigReload") diff --git a/server/statistics_handler.go b/server/statistics_handler.go index 40761114073f1..b80674c2ef028 100644 --- a/server/statistics_handler.go +++ b/server/statistics_handler.go @@ -15,11 +15,17 @@ package server import ( "net/http" + "time" "github.com/gorilla/mux" "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/session" + "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util/gcutil" + "github.com/pingcap/tidb/util/sqlexec" ) // StatsHandler is the handler for dumping statistics. @@ -51,7 +57,7 @@ func (sh StatsHandler) ServeHTTP(w http.ResponseWriter, req *http.Request) { if err != nil { writeError(w, err) } else { - js, err := h.DumpStatsToJSON(params[pDBName], tbl.Meta()) + js, err := h.DumpStatsToJSON(params[pDBName], tbl.Meta(), nil) if err != nil { writeError(w, err) } else { @@ -59,3 +65,69 @@ func (sh StatsHandler) ServeHTTP(w http.ResponseWriter, req *http.Request) { } } } + +// StatsHistoryHandler is the handler for dumping statistics. +type StatsHistoryHandler struct { + do *domain.Domain +} + +func (s *Server) newStatsHistoryHandler() *StatsHistoryHandler { + store, ok := s.driver.(*TiDBDriver) + if !ok { + panic("Illegal driver") + } + + do, err := session.GetDomain(store.store) + if err != nil { + panic("Failed to get domain") + } + return &StatsHistoryHandler{do} +} + +func (sh StatsHistoryHandler) ServeHTTP(w http.ResponseWriter, req *http.Request) { + w.Header().Set("Content-Type", "application/json") + + params := mux.Vars(req) + se, err := session.CreateSession(sh.do.Store()) + 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 { + writeError(w, err) + return + } + t1, err := t.Time.GoTime(time.Local) + if err != nil { + writeError(w, err) + return + } + snapshot := variable.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) + return + } + h := sh.do.StatsHandle() + tbl, err := is.TableByName(model.NewCIStr(params[pDBName]), model.NewCIStr(params[pTableName])) + if err != nil { + writeError(w, err) + return + } + se.GetSessionVars().SnapshotInfoschema, se.GetSessionVars().SnapshotTS = is, snapshot + historyStatsExec := se.(sqlexec.RestrictedSQLExecutor) + js, err := h.DumpStatsToJSON(params[pDBName], tbl.Meta(), historyStatsExec) + if err != nil { + writeError(w, err) + } else { + writeData(w, js) + } +} diff --git a/server/statistics_handler_test.go b/server/statistics_handler_test.go index 8a7b5ff47113e..ee5b287778904 100644 --- a/server/statistics_handler_test.go +++ b/server/statistics_handler_test.go @@ -19,6 +19,7 @@ import ( "io/ioutil" "net/http" "os" + "time" "github.com/go-sql-driver/mysql" "github.com/gorilla/mux" @@ -105,6 +106,37 @@ func (ds *testDumpStatsSuite) TestDumpStatsAPI(c *C) { c.Assert(err, IsNil) fp.Write(js) ds.checkData(c, path) + + // sleep for 1 seconds to ensure the existence of tidb.test + time.Sleep(time.Second) + timeBeforeDropStats := time.Now() + snapshot := timeBeforeDropStats.Format("20060102150405") + ds.prepare4DumpHistoryStats(c) + + // test dump history stats + resp1, err := http.Get("http://127.0.0.1:10090/stats/dump/tidb/test") + c.Assert(err, IsNil) + defer resp1.Body.Close() + js, err = ioutil.ReadAll(resp1.Body) + c.Assert(err, IsNil) + c.Assert(string(js), Equals, "null") + + path1 := "/tmp/stats_history.json" + fp1, err := os.Create(path1) + c.Assert(err, IsNil) + c.Assert(fp1, NotNil) + defer func() { + c.Assert(fp1.Close(), IsNil) + c.Assert(os.Remove(path1), IsNil) + }() + + resp1, err = http.Get("http://127.0.0.1:10090/stats/dump/tidb/test/" + snapshot) + c.Assert(err, IsNil) + + js, err = ioutil.ReadAll(resp1.Body) + c.Assert(err, IsNil) + fp1.Write(js) + ds.checkData(c, path1) } func (ds *testDumpStatsSuite) prepareData(c *C) { @@ -128,6 +160,25 @@ func (ds *testDumpStatsSuite) prepareData(c *C) { c.Assert(h.Update(is), IsNil) } +func (ds *testDumpStatsSuite) prepare4DumpHistoryStats(c *C) { + db, err := sql.Open("mysql", getDSN()) + c.Assert(err, IsNil, Commentf("Error connecting")) + defer db.Close() + + dbt := &DBTest{c, db} + + safePointName := "tikv_gc_safe_point" + safePointValue := "20060102-15:04:05 -0700" + safePointComment := "All versions after safe point can be accessed. (DO NOT EDIT)" + updateSafePoint := fmt.Sprintf(`INSERT INTO mysql.tidb VALUES ('%[1]s', '%[2]s', '%[3]s') + ON DUPLICATE KEY + UPDATE variable_value = '%[2]s', comment = '%[3]s'`, safePointName, safePointValue, safePointComment) + dbt.mustExec(updateSafePoint) + + dbt.mustExec("drop table tidb.test") + dbt.mustExec("create table tidb.test (a int, b varchar(20))") +} + func (ds *testDumpStatsSuite) checkData(c *C, path string) { db, err := sql.Open("mysql", getDSN(func(config *mysql.Config) { config.AllowAllFiles = true @@ -135,13 +186,7 @@ func (ds *testDumpStatsSuite) checkData(c *C, path string) { })) c.Assert(err, IsNil, Commentf("Error connecting")) dbt := &DBTest{c, db} - defer func() { - dbt.mustExec("drop database tidb") - dbt.mustExec("truncate table mysql.stats_meta") - dbt.mustExec("truncate table mysql.stats_histograms") - dbt.mustExec("truncate table mysql.stats_buckets") - db.Close() - }() + defer db.Close() dbt.mustExec("use tidb") dbt.mustExec("drop stats test") @@ -160,3 +205,15 @@ func (ds *testDumpStatsSuite) checkData(c *C, path string) { dbt.Check(modifyCount, Equals, int64(3)) dbt.Check(count, Equals, int64(4)) } + +func (ds *testDumpStatsSuite) clearData(c *C, path string) { + db, err := sql.Open("mysql", getDSN()) + c.Assert(err, IsNil, Commentf("Error connecting")) + defer db.Close() + + dbt := &DBTest{c, db} + dbt.mustExec("drop database tidb") + dbt.mustExec("truncate table mysql.stats_meta") + dbt.mustExec("truncate table mysql.stats_histograms") + dbt.mustExec("truncate table mysql.stats_buckets") +} diff --git a/statistics/handle/dump.go b/statistics/handle/dump.go index 1a4098bfde8f4..7c6a931f9c501 100644 --- a/statistics/handle/dump.go +++ b/statistics/handle/dump.go @@ -23,6 +23,7 @@ import ( "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util/sqlexec" "github.com/pingcap/tipb/go-tipb" ) @@ -59,10 +60,10 @@ func dumpJSONCol(hist *statistics.Histogram, CMSketch *statistics.CMSketch) *jso } // DumpStatsToJSON dumps statistic to json. -func (h *Handle) DumpStatsToJSON(dbName string, tableInfo *model.TableInfo) (*JSONTable, error) { +func (h *Handle) DumpStatsToJSON(dbName string, tableInfo *model.TableInfo, historyStatsExec sqlexec.RestrictedSQLExecutor) (*JSONTable, error) { pi := tableInfo.GetPartitionInfo() if pi == nil { - return h.tableStatsToJSON(dbName, tableInfo, tableInfo.ID) + return h.tableStatsToJSON(dbName, tableInfo, tableInfo.ID, historyStatsExec) } jsonTbl := &JSONTable{ DatabaseName: dbName, @@ -70,7 +71,7 @@ func (h *Handle) DumpStatsToJSON(dbName string, tableInfo *model.TableInfo) (*JS Partitions: make(map[string]*JSONTable, len(pi.Definitions)), } for _, def := range pi.Definitions { - tbl, err := h.tableStatsToJSON(dbName, tableInfo, def.ID) + tbl, err := h.tableStatsToJSON(dbName, tableInfo, def.ID, historyStatsExec) if err != nil { return nil, errors.Trace(err) } @@ -82,13 +83,14 @@ func (h *Handle) DumpStatsToJSON(dbName string, tableInfo *model.TableInfo) (*JS return jsonTbl, nil } -func (h *Handle) tableStatsToJSON(dbName string, tableInfo *model.TableInfo, physicalID int64) (*JSONTable, error) { - tbl, err := h.tableStatsFromStorage(tableInfo, physicalID, true) - if err != nil { - return nil, errors.Trace(err) +func (h *Handle) tableStatsToJSON(dbName string, tableInfo *model.TableInfo, physicalID int64, historyStatsExec sqlexec.RestrictedSQLExecutor) (*JSONTable, error) { + tbl, err := h.tableStatsFromStorage(tableInfo, physicalID, true, historyStatsExec) + if err != nil || tbl == nil { + return nil, err } - if tbl == nil { - return nil, nil + tbl.Version, tbl.ModifyCount, tbl.Count, err = h.statsMetaByTableIDFromStorage(physicalID, historyStatsExec) + if err != nil { + return nil, err } jsonTbl := &JSONTable{ DatabaseName: dbName, diff --git a/statistics/handle/dump_test.go b/statistics/handle/dump_test.go index a1371e8e7f925..7f080b92ad32c 100644 --- a/statistics/handle/dump_test.go +++ b/statistics/handle/dump_test.go @@ -40,7 +40,7 @@ func (s *testStatsSuite) TestConversion(c *C) { tableInfo, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) c.Assert(err, IsNil) - jsonTbl, err := h.DumpStatsToJSON("test", tableInfo.Meta()) + jsonTbl, err := h.DumpStatsToJSON("test", tableInfo.Meta(), nil) c.Assert(err, IsNil) loadTbl, err := handle.TableStatsFromJSON(tableInfo.Meta(), tableInfo.Meta().ID, jsonTbl) c.Assert(err, IsNil) @@ -78,7 +78,7 @@ PARTITION BY RANGE ( a ) ( table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) c.Assert(err, IsNil) tableInfo := table.Meta() - jsonTbl, err := h.DumpStatsToJSON("test", tableInfo) + jsonTbl, err := h.DumpStatsToJSON("test", tableInfo, nil) c.Assert(err, IsNil) pi := tableInfo.GetPartitionInfo() originTables := make([]*statistics.Table, 0, len(pi.Definitions)) @@ -113,7 +113,7 @@ func (s *testStatsSuite) TestDumpAlteredTable(c *C) { tk.MustExec("alter table t drop column a") table, err := s.do.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t")) c.Assert(err, IsNil) - _, err = h.DumpStatsToJSON("test", table.Meta()) + _, err = h.DumpStatsToJSON("test", table.Meta(), nil) c.Assert(err, IsNil) } @@ -150,7 +150,7 @@ func (s *testStatsSuite) TestDumpCMSketchWithTopN(c *C) { c.Assert(cmsFromStore, NotNil) c.Check(cms.Equal(cmsFromStore), IsTrue) - jsonTable, err := h.DumpStatsToJSON("test", tableInfo) + jsonTable, err := h.DumpStatsToJSON("test", tableInfo, nil) c.Check(err, IsNil) err = h.LoadStatsFromJSON(is, jsonTable) c.Check(err, IsNil) diff --git a/statistics/handle/handle.go b/statistics/handle/handle.go index 8432fdfba2535..6cfbf16450bcc 100644 --- a/statistics/handle/handle.go +++ b/statistics/handle/handle.go @@ -21,6 +21,7 @@ import ( "time" "github.com/pingcap/errors" + "github.com/pingcap/parser/ast" "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/ddl/util" @@ -161,7 +162,7 @@ func (h *Handle) Update(is infoschema.InfoSchema) error { continue } tableInfo := table.Meta() - tbl, err := h.tableStatsFromStorage(tableInfo, physicalID, false) + tbl, err := h.tableStatsFromStorage(tableInfo, physicalID, false, nil) // Error is not nil may mean that there are some ddl changes on this table, we will not update it. if err != nil { logutil.Logger(context.Background()).Debug("error occurred when read table stats", zap.String("table", tableInfo.Name.O), zap.Error(err)) @@ -265,11 +266,11 @@ func (h *Handle) LoadNeededHistograms() error { statistics.HistogramNeededColumns.Delete(col) continue } - hg, err := h.histogramFromStorage(col.TableID, c.ID, &c.Info.FieldType, c.NDV, 0, c.LastUpdateVersion, c.NullCount, c.TotColSize, c.Correlation) + hg, err := h.histogramFromStorage(col.TableID, c.ID, &c.Info.FieldType, c.NDV, 0, c.LastUpdateVersion, c.NullCount, c.TotColSize, c.Correlation, nil) if err != nil { return errors.Trace(err) } - cms, err := h.cmSketchFromStorage(col.TableID, 0, col.ColumnID) + cms, err := h.cmSketchFromStorage(col.TableID, 0, col.ColumnID, nil) if err != nil { return errors.Trace(err) } @@ -317,9 +318,14 @@ func (h *Handle) FlushStats() { } } -func (h *Handle) cmSketchFromStorage(tblID int64, isIndex, histID int64) (*statistics.CMSketch, error) { +func (h *Handle) cmSketchFromStorage(tblID int64, isIndex, histID int64, historyStatsExec sqlexec.RestrictedSQLExecutor) (_ *statistics.CMSketch, err error) { selSQL := fmt.Sprintf("select cm_sketch from mysql.stats_histograms where table_id = %d and is_index = %d and hist_id = %d", tblID, isIndex, histID) - rows, _, err := h.restrictedExec.ExecRestrictedSQL(nil, selSQL) + var rows []chunk.Row + if historyStatsExec != nil { + rows, _, err = historyStatsExec.ExecRestrictedSQLWithSnapshot(nil, selSQL) + } else { + rows, _, err = h.restrictedExec.ExecRestrictedSQL(nil, selSQL) + } if err != nil { return nil, errors.Trace(err) } @@ -329,7 +335,7 @@ func (h *Handle) cmSketchFromStorage(tblID int64, isIndex, histID int64) (*stati return statistics.LoadCMSketchWithTopN(h.restrictedExec, tblID, isIndex, histID, rows[0].GetBytes(0)) } -func (h *Handle) indexStatsFromStorage(row chunk.Row, table *statistics.Table, tableInfo *model.TableInfo) error { +func (h *Handle) indexStatsFromStorage(row chunk.Row, table *statistics.Table, tableInfo *model.TableInfo, historyStatsExec sqlexec.RestrictedSQLExecutor) error { histID := row.GetInt64(2) distinct := row.GetInt64(3) histVer := row.GetUint64(4) @@ -348,11 +354,11 @@ func (h *Handle) indexStatsFromStorage(row chunk.Row, table *statistics.Table, t continue } if idx == nil || idx.LastUpdateVersion < histVer { - hg, err := h.histogramFromStorage(table.PhysicalID, histID, types.NewFieldType(mysql.TypeBlob), distinct, 1, histVer, nullCount, 0, 0) + hg, err := h.histogramFromStorage(table.PhysicalID, histID, types.NewFieldType(mysql.TypeBlob), distinct, 1, histVer, nullCount, 0, 0, historyStatsExec) if err != nil { return errors.Trace(err) } - cms, err := h.cmSketchFromStorage(table.PhysicalID, 1, idxInfo.ID) + cms, err := h.cmSketchFromStorage(table.PhysicalID, 1, idxInfo.ID, historyStatsExec) if err != nil { return errors.Trace(err) } @@ -368,7 +374,7 @@ func (h *Handle) indexStatsFromStorage(row chunk.Row, table *statistics.Table, t return nil } -func (h *Handle) columnStatsFromStorage(row chunk.Row, table *statistics.Table, tableInfo *model.TableInfo, loadAll bool) error { +func (h *Handle) columnStatsFromStorage(row chunk.Row, table *statistics.Table, tableInfo *model.TableInfo, loadAll bool, historyStatsExec sqlexec.RestrictedSQLExecutor) error { histID := row.GetInt64(2) distinct := row.GetInt64(3) histVer := row.GetUint64(4) @@ -415,11 +421,11 @@ func (h *Handle) columnStatsFromStorage(row chunk.Row, table *statistics.Table, break } if col == nil || col.LastUpdateVersion < histVer || loadAll { - hg, err := h.histogramFromStorage(table.PhysicalID, histID, &colInfo.FieldType, distinct, 0, histVer, nullCount, totColSize, correlation) + hg, err := h.histogramFromStorage(table.PhysicalID, histID, &colInfo.FieldType, distinct, 0, histVer, nullCount, totColSize, correlation, historyStatsExec) if err != nil { return errors.Trace(err) } - cms, err := h.cmSketchFromStorage(table.PhysicalID, 0, colInfo.ID) + cms, err := h.cmSketchFromStorage(table.PhysicalID, 0, colInfo.ID, historyStatsExec) if err != nil { return errors.Trace(err) } @@ -453,11 +459,11 @@ func (h *Handle) columnStatsFromStorage(row chunk.Row, table *statistics.Table, } // tableStatsFromStorage loads table stats info from storage. -func (h *Handle) tableStatsFromStorage(tableInfo *model.TableInfo, physicalID int64, loadAll bool) (*statistics.Table, error) { +func (h *Handle) tableStatsFromStorage(tableInfo *model.TableInfo, physicalID int64, loadAll bool, historyStatsExec sqlexec.RestrictedSQLExecutor) (_ *statistics.Table, err error) { table, ok := h.StatsCache.Load().(StatsCache)[physicalID] // If table stats is pseudo, we also need to copy it, since we will use the column stats when // the average error rate of it is small. - if !ok { + if !ok || historyStatsExec != nil { histColl := statistics.HistColl{ PhysicalID: physicalID, HavePhysicalID: true, @@ -473,9 +479,14 @@ func (h *Handle) tableStatsFromStorage(tableInfo *model.TableInfo, physicalID in } table.Pseudo = false selSQL := fmt.Sprintf("select table_id, is_index, hist_id, distinct_count, version, null_count, tot_col_size, stats_ver, flag, correlation from mysql.stats_histograms where table_id = %d", physicalID) - rows, _, err := h.restrictedExec.ExecRestrictedSQL(nil, selSQL) + var rows []chunk.Row + if historyStatsExec != nil { + rows, _, err = historyStatsExec.ExecRestrictedSQLWithSnapshot(nil, selSQL) + } else { + rows, _, err = h.restrictedExec.ExecRestrictedSQL(nil, selSQL) + } if err != nil { - return nil, errors.Trace(err) + return nil, err } // Check deleted table. if len(rows) == 0 { @@ -483,11 +494,11 @@ func (h *Handle) tableStatsFromStorage(tableInfo *model.TableInfo, physicalID in } for _, row := range rows { if row.GetInt64(1) > 0 { - if err := h.indexStatsFromStorage(row, table, tableInfo); err != nil { + if err := h.indexStatsFromStorage(row, table, tableInfo, historyStatsExec); err != nil { return nil, errors.Trace(err) } } else { - if err := h.columnStatsFromStorage(row, table, tableInfo, loadAll); err != nil { + if err := h.columnStatsFromStorage(row, table, tableInfo, loadAll, historyStatsExec); err != nil { return nil, errors.Trace(err) } } @@ -606,9 +617,17 @@ func (h *Handle) SaveMetaToStorage(tableID, count, modifyCount int64) (err error return } -func (h *Handle) histogramFromStorage(tableID int64, colID int64, tp *types.FieldType, distinct int64, isIndex int, ver uint64, nullCount int64, totColSize int64, corr float64) (*statistics.Histogram, error) { +func (h *Handle) histogramFromStorage(tableID int64, colID int64, tp *types.FieldType, distinct int64, isIndex int, ver uint64, nullCount int64, totColSize int64, corr float64, historyStatsExec sqlexec.RestrictedSQLExecutor) (_ *statistics.Histogram, err error) { selSQL := fmt.Sprintf("select count, repeats, lower_bound, upper_bound from mysql.stats_buckets where table_id = %d and is_index = %d and hist_id = %d order by bucket_id", tableID, isIndex, colID) - rows, fields, err := h.restrictedExec.ExecRestrictedSQL(nil, selSQL) + var ( + rows []chunk.Row + fields []*ast.ResultField + ) + if historyStatsExec != nil { + rows, fields, err = historyStatsExec.ExecRestrictedSQLWithSnapshot(nil, selSQL) + } else { + rows, fields, err = h.restrictedExec.ExecRestrictedSQL(nil, selSQL) + } if err != nil { return nil, errors.Trace(err) } @@ -654,3 +673,20 @@ func (h *Handle) columnCountFromStorage(tableID, colID int64) (int64, error) { } return rows[0].GetMyDecimal(0).ToInt() } + +func (h *Handle) statsMetaByTableIDFromStorage(tableID int64, historyStatsExec sqlexec.RestrictedSQLExecutor) (version uint64, modifyCount, count int64, err error) { + selSQL := fmt.Sprintf("SELECT version, modify_count, count from mysql.stats_meta where table_id = %d order by version", tableID) + var rows []chunk.Row + if historyStatsExec == nil { + rows, _, err = h.restrictedExec.ExecRestrictedSQL(nil, selSQL) + } else { + rows, _, err = historyStatsExec.ExecRestrictedSQLWithSnapshot(nil, selSQL) + } + if err != nil || len(rows) == 0 { + return + } + version = rows[0].GetUint64(0) + modifyCount = rows[0].GetInt64(1) + count = rows[0].GetInt64(2) + return +}