Skip to content

Commit

Permalink
server, statistics: support dump history stats (#10291)
Browse files Browse the repository at this point in the history
  • Loading branch information
XuHuaiyu authored May 7, 2019
1 parent 17c3229 commit 4789904
Show file tree
Hide file tree
Showing 8 changed files with 224 additions and 40 deletions.
15 changes: 15 additions & 0 deletions docs/tidb_http_api.md
Original file line number Diff line number Diff line change
Expand Up @@ -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}
```
1 change: 1 addition & 0 deletions server/http_handler.go
Original file line number Diff line number Diff line change
Expand Up @@ -72,6 +72,7 @@ const (
pColumnFlag = "colFlag"
pColumnLen = "colLen"
pRowBin = "rowBin"
pSnapshot = "snapshot"
)

// For query string
Expand Down
1 change: 1 addition & 0 deletions server/http_status.go
Original file line number Diff line number Diff line change
Expand Up @@ -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")
Expand Down
74 changes: 73 additions & 1 deletion server/statistics_handler.go
Original file line number Diff line number Diff line change
Expand Up @@ -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.
Expand Down Expand Up @@ -51,11 +57,77 @@ 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 {
writeData(w, js)
}
}
}

// 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)
}
}
71 changes: 64 additions & 7 deletions server/statistics_handler_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ import (
"io/ioutil"
"net/http"
"os"
"time"

"github.com/go-sql-driver/mysql"
"github.com/gorilla/mux"
Expand Down Expand Up @@ -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) {
Expand All @@ -128,20 +160,33 @@ 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
config.Strict = false
}))
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")
Expand All @@ -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")
}
20 changes: 11 additions & 9 deletions statistics/handle/dump.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
)

Expand Down Expand Up @@ -59,18 +60,18 @@ 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,
TableName: tableInfo.Name.L,
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)
}
Expand All @@ -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,
Expand Down
8 changes: 4 additions & 4 deletions statistics/handle/dump_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand Down Expand Up @@ -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))
Expand Down Expand Up @@ -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)
}

Expand Down Expand Up @@ -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)
Expand Down
Loading

0 comments on commit 4789904

Please sign in to comment.