Skip to content

Commit

Permalink
server: support dumpPartitionStats param for statsHandler (#38190)
Browse files Browse the repository at this point in the history
ref #37977
  • Loading branch information
Yisaer authored Oct 28, 2022
1 parent d760405 commit 636a363
Show file tree
Hide file tree
Showing 9 changed files with 117 additions and 43 deletions.
2 changes: 1 addition & 1 deletion br/pkg/backup/schema.go
Original file line number Diff line number Diff line change
Expand Up @@ -181,7 +181,7 @@ func (s *schemaInfo) calculateChecksum(

func (s *schemaInfo) dumpStatsToJSON(statsHandle *handle.Handle) error {
jsonTable, err := statsHandle.DumpStatsToJSON(
s.dbInfo.Name.String(), s.tableInfo, nil)
s.dbInfo.Name.String(), s.tableInfo, nil, true)
if err != nil {
return errors.Trace(err)
}
Expand Down
2 changes: 1 addition & 1 deletion executor/analyzetest/analyze_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -2195,7 +2195,7 @@ func TestRecordHistoryStatsAfterAnalyze(t *testing.T) {
require.GreaterOrEqual(t, num, 1)

// 3. dump current stats json
dumpJSONTable, err := h.DumpStatsToJSON("test", tableInfo.Meta(), nil)
dumpJSONTable, err := h.DumpStatsToJSON("test", tableInfo.Meta(), nil, true)
require.NoError(t, err)
jsOrigin, _ := json.Marshal(dumpJSONTable)

Expand Down
2 changes: 1 addition & 1 deletion executor/plan_replayer.go
Original file line number Diff line number Diff line change
Expand Up @@ -593,7 +593,7 @@ func getStatsForTable(do *domain.Domain, pair tableNamePair) (*handle.JSONTable,
if err != nil {
return nil, err
}
js, err := h.DumpStatsToJSON(pair.DBName, tbl.Meta(), nil)
js, err := h.DumpStatsToJSON(pair.DBName, tbl.Meta(), nil, true)
return js, err
}

Expand Down
31 changes: 16 additions & 15 deletions server/http_handler.go
Original file line number Diff line number Diff line change
Expand Up @@ -72,21 +72,22 @@ import (
)

const (
pDBName = "db"
pHexKey = "hexKey"
pIndexName = "index"
pHandle = "handle"
pRegionID = "regionID"
pStartTS = "startTS"
pTableName = "table"
pTableID = "tableID"
pColumnID = "colID"
pColumnTp = "colTp"
pColumnFlag = "colFlag"
pColumnLen = "colLen"
pRowBin = "rowBin"
pSnapshot = "snapshot"
pFileName = "filename"
pDBName = "db"
pHexKey = "hexKey"
pIndexName = "index"
pHandle = "handle"
pRegionID = "regionID"
pStartTS = "startTS"
pTableName = "table"
pTableID = "tableID"
pColumnID = "colID"
pColumnTp = "colTp"
pColumnFlag = "colFlag"
pColumnLen = "colLen"
pRowBin = "rowBin"
pSnapshot = "snapshot"
pFileName = "filename"
pDumpPartitionStats = "dumpPartitionStats"
)

// For query string
Expand Down
24 changes: 22 additions & 2 deletions server/statistics_handler.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ package server

import (
"net/http"
"strconv"
"time"

"github.com/gorilla/mux"
Expand Down Expand Up @@ -53,11 +54,21 @@ func (sh StatsHandler) ServeHTTP(w http.ResponseWriter, req *http.Request) {

is := sh.do.InfoSchema()
h := sh.do.StatsHandle()
var err error
dumpPartitionStats := true
dumpParams := req.URL.Query()[pDumpPartitionStats]
if len(dumpParams) > 0 && len(dumpParams[0]) > 0 {
dumpPartitionStats, err = strconv.ParseBool(dumpParams[0])
if err != nil {
writeError(w, err)
return
}
}
tbl, err := is.TableByName(model.NewCIStr(params[pDBName]), model.NewCIStr(params[pTableName]))
if err != nil {
writeError(w, err)
} else {
js, err := h.DumpStatsToJSON(params[pDBName], tbl.Meta(), nil)
js, err := h.DumpStatsToJSON(params[pDBName], tbl.Meta(), nil, dumpPartitionStats)
if err != nil {
writeError(w, err)
} else {
Expand Down Expand Up @@ -95,6 +106,15 @@ func (sh StatsHistoryHandler) ServeHTTP(w http.ResponseWriter, req *http.Request
}
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 {
Expand Down Expand Up @@ -124,7 +144,7 @@ func (sh StatsHistoryHandler) ServeHTTP(w http.ResponseWriter, req *http.Request
writeError(w, err)
return
}
js, err := h.DumpStatsToJSONBySnapshot(params[pDBName], tbl.Meta(), snapshot)
js, err := h.DumpStatsToJSONBySnapshot(params[pDBName], tbl.Meta(), snapshot, dumpPartitionStats)
if err != nil {
writeError(w, err)
} else {
Expand Down
45 changes: 45 additions & 0 deletions server/statistics_handler_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ package server

import (
"database/sql"
"encoding/json"
"fmt"
"io"
"os"
Expand Down Expand Up @@ -119,6 +120,33 @@ func TestDumpStatsAPI(t *testing.T) {
_, err = fp1.Write(js)
require.NoError(t, err)
checkData(t, path1, client)

testDumpPartitionTableStats(t, client, statsHandler)
}

func testDumpPartitionTableStats(t *testing.T, client *testServerClient, handler *StatsHandler) {
preparePartitionData(t, client, handler)
check := func(dumpStats bool) {
expectedLen := 1
if dumpStats {
expectedLen = 2
}
url := fmt.Sprintf("/stats/dump/test/test2?dumpPartitionStats=%v", dumpStats)
resp0, err := client.fetchStatus(url)
require.NoError(t, err)
defer func() {
resp0.Body.Close()
}()
b, err := io.ReadAll(resp0.Body)
require.NoError(t, err)
jsonTable := &handle.JSONTable{}
err = json.Unmarshal(b, jsonTable)
require.NoError(t, err)
require.NotNil(t, jsonTable.Partitions["global"])
require.Len(t, jsonTable.Partitions, expectedLen)
}
check(false)
check(true)
}

func prepareData(t *testing.T, client *testServerClient, statHandle *StatsHandler) {
Expand Down Expand Up @@ -146,6 +174,23 @@ func prepareData(t *testing.T, client *testServerClient, statHandle *StatsHandle
require.NoError(t, h.Update(is))
}

func preparePartitionData(t *testing.T, client *testServerClient, statHandle *StatsHandler) {
db, err := sql.Open("mysql", client.getDSN())
require.NoError(t, err, "Error connecting")
defer func() {
err := db.Close()
require.NoError(t, err)
}()
h := statHandle.do.StatsHandle()
tk := testkit.NewDBTestKit(t, db)
tk.MustExec("create table test2(a int) PARTITION BY RANGE ( a ) (PARTITION p0 VALUES LESS THAN (6))")
tk.MustExec("insert into test2 (a) values (1)")
tk.MustExec("analyze table test2")
is := statHandle.do.InfoSchema()
require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll))
require.NoError(t, h.Update(is))
}

func prepare4DumpHistoryStats(t *testing.T, client *testServerClient) {
db, err := sql.Open("mysql", client.getDSN())
require.NoError(t, err, "Error connecting")
Expand Down
30 changes: 19 additions & 11 deletions statistics/handle/dump.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@ import (
"github.com/pingcap/tidb/parser/mysql"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/sessionctx/stmtctx"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/statistics"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/sqlexec"
Expand Down Expand Up @@ -119,17 +120,21 @@ func dumpJSONCol(hist *statistics.Histogram, CMSketch *statistics.CMSketch, topn
}

// DumpStatsToJSON dumps statistic to json.
func (h *Handle) DumpStatsToJSON(dbName string, tableInfo *model.TableInfo, historyStatsExec sqlexec.RestrictedSQLExecutor) (*JSONTable, error) {
func (h *Handle) DumpStatsToJSON(dbName string, tableInfo *model.TableInfo,
historyStatsExec sqlexec.RestrictedSQLExecutor, dumpPartitionStats bool) (*JSONTable, error) {
var snapshot uint64
if historyStatsExec != nil {
sctx := historyStatsExec.(sessionctx.Context)
snapshot = sctx.GetSessionVars().SnapshotTS
}
return h.DumpStatsToJSONBySnapshot(dbName, tableInfo, snapshot)
return h.DumpStatsToJSONBySnapshot(dbName, tableInfo, snapshot, dumpPartitionStats)
}

// DumpStatsToJSONBySnapshot dumps statistic to json.
func (h *Handle) DumpStatsToJSONBySnapshot(dbName string, tableInfo *model.TableInfo, snapshot uint64) (*JSONTable, error) {
func (h *Handle) DumpStatsToJSONBySnapshot(dbName string, tableInfo *model.TableInfo, snapshot uint64, dumpPartitionStats bool) (*JSONTable, error) {
h.mu.Lock()
isDynamicMode := variable.PartitionPruneMode(h.mu.ctx.GetSessionVars().PartitionPruneMode.Load()) == variable.Dynamic
h.mu.Unlock()
pi := tableInfo.GetPartitionInfo()
if pi == nil {
return h.tableStatsToJSON(dbName, tableInfo, tableInfo.ID, snapshot)
Expand All @@ -139,15 +144,18 @@ func (h *Handle) DumpStatsToJSONBySnapshot(dbName string, tableInfo *model.Table
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, snapshot)
if err != nil {
return nil, errors.Trace(err)
}
if tbl == nil {
continue
// dump partition stats only if in static mode or enable dumpPartitionStats flag in dynamic mode
if !isDynamicMode || dumpPartitionStats {
for _, def := range pi.Definitions {
tbl, err := h.tableStatsToJSON(dbName, tableInfo, def.ID, snapshot)
if err != nil {
return nil, errors.Trace(err)
}
if tbl == nil {
continue
}
jsonTbl.Partitions[def.Name.L] = tbl
}
jsonTbl.Partitions[def.Name.L] = tbl
}
// dump its global-stats if existed
tbl, err := h.tableStatsToJSON(dbName, tableInfo, tableInfo.ID, snapshot)
Expand Down
22 changes: 11 additions & 11 deletions statistics/handle/dump_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -90,7 +90,7 @@ func TestConversion(t *testing.T) {

tableInfo, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t"))
require.NoError(t, err)
jsonTbl, err := h.DumpStatsToJSON("test", tableInfo.Meta(), nil)
jsonTbl, err := h.DumpStatsToJSON("test", tableInfo.Meta(), nil, true)
require.NoError(t, err)
loadTbl, err := handle.TableStatsFromJSON(tableInfo.Meta(), tableInfo.Meta().ID, jsonTbl)
require.NoError(t, err)
Expand All @@ -117,7 +117,7 @@ func getStatsJSON(t *testing.T, dom *domain.Domain, db, tableName string) *handl
table, err := is.TableByName(model.NewCIStr(db), model.NewCIStr(tableName))
require.NoError(t, err)
tableInfo := table.Meta()
jsonTbl, err := h.DumpStatsToJSON("test", tableInfo, nil)
jsonTbl, err := h.DumpStatsToJSON("test", tableInfo, nil, true)
require.NoError(t, err)
return jsonTbl
}
Expand Down Expand Up @@ -198,7 +198,7 @@ PARTITION BY RANGE ( a ) (
table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t"))
require.NoError(t, err)
tableInfo := table.Meta()
jsonTbl, err := h.DumpStatsToJSON("test", tableInfo, nil)
jsonTbl, err := h.DumpStatsToJSON("test", tableInfo, nil, true)
require.NoError(t, err)
pi := tableInfo.GetPartitionInfo()
originTables := make([]*statistics.Table, 0, len(pi.Definitions))
Expand Down Expand Up @@ -233,7 +233,7 @@ func TestDumpAlteredTable(t *testing.T) {
tk.MustExec("alter table t drop column a")
table, err := dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t"))
require.NoError(t, err)
_, err = h.DumpStatsToJSON("test", table.Meta(), nil)
_, err = h.DumpStatsToJSON("test", table.Meta(), nil, true)
require.NoError(t, err)
}

Expand Down Expand Up @@ -270,7 +270,7 @@ func TestDumpCMSketchWithTopN(t *testing.T) {
require.NotNil(t, cmsFromStore)
require.True(t, cms.Equal(cmsFromStore))

jsonTable, err := h.DumpStatsToJSON("test", tableInfo, nil)
jsonTable, err := h.DumpStatsToJSON("test", tableInfo, nil, true)
require.NoError(t, err)
err = h.LoadStatsFromJSON(is, jsonTable)
require.NoError(t, err)
Expand All @@ -292,7 +292,7 @@ func TestDumpPseudoColumns(t *testing.T) {
tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t"))
require.NoError(t, err)
h := dom.StatsHandle()
_, err = h.DumpStatsToJSON("test", tbl.Meta(), nil)
_, err = h.DumpStatsToJSON("test", tbl.Meta(), nil, true)
require.NoError(t, err)
}

Expand All @@ -313,7 +313,7 @@ func TestDumpExtendedStats(t *testing.T) {
tableInfo, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t"))
require.NoError(t, err)
tbl := h.GetTableStats(tableInfo.Meta())
jsonTbl, err := h.DumpStatsToJSON("test", tableInfo.Meta(), nil)
jsonTbl, err := h.DumpStatsToJSON("test", tableInfo.Meta(), nil, true)
require.NoError(t, err)
loadTbl, err := handle.TableStatsFromJSON(tableInfo.Meta(), tableInfo.Meta().ID, jsonTbl)
require.NoError(t, err)
Expand Down Expand Up @@ -353,7 +353,7 @@ func TestDumpVer2Stats(t *testing.T) {
storageTbl, err := h.TableStatsFromStorage(tableInfo.Meta(), tableInfo.Meta().ID, false, 0)
require.NoError(t, err)

dumpJSONTable, err := h.DumpStatsToJSON("test", tableInfo.Meta(), nil)
dumpJSONTable, err := h.DumpStatsToJSON("test", tableInfo.Meta(), nil, true)
require.NoError(t, err)

jsonBytes, err := json.MarshalIndent(dumpJSONTable, "", " ")
Expand Down Expand Up @@ -405,7 +405,7 @@ func TestLoadStatsForNewCollation(t *testing.T) {
storageTbl, err := h.TableStatsFromStorage(tableInfo.Meta(), tableInfo.Meta().ID, false, 0)
require.NoError(t, err)

dumpJSONTable, err := h.DumpStatsToJSON("test", tableInfo.Meta(), nil)
dumpJSONTable, err := h.DumpStatsToJSON("test", tableInfo.Meta(), nil, true)
require.NoError(t, err)

jsonBytes, err := json.MarshalIndent(dumpJSONTable, "", " ")
Expand Down Expand Up @@ -453,12 +453,12 @@ func TestJSONTableToBlocks(t *testing.T) {
tableInfo, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t"))
require.NoError(t, err)

dumpJSONTable, err := h.DumpStatsToJSON("test", tableInfo.Meta(), nil)
dumpJSONTable, err := h.DumpStatsToJSON("test", tableInfo.Meta(), nil, true)
require.NoError(t, err)
jsOrigin, _ := json.Marshal(dumpJSONTable)

blockSize := 30
js, err := h.DumpStatsToJSON("test", tableInfo.Meta(), nil)
js, err := h.DumpStatsToJSON("test", tableInfo.Meta(), nil, true)
require.NoError(t, err)
dumpJSONBlocks, err := handle.JSONTableToBlocks(js, blockSize)
require.NoError(t, err)
Expand Down
2 changes: 1 addition & 1 deletion statistics/handle/handle.go
Original file line number Diff line number Diff line change
Expand Up @@ -2315,7 +2315,7 @@ const maxColumnSize = 6 << 20
// RecordHistoricalStatsToStorage records the given table's stats data to mysql.stats_history
func (h *Handle) RecordHistoricalStatsToStorage(dbName string, tableInfo *model.TableInfo) (uint64, error) {
ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnStats)
js, err := h.DumpStatsToJSON(dbName, tableInfo, nil)
js, err := h.DumpStatsToJSON(dbName, tableInfo, nil, true)
if err != nil {
return 0, errors.Trace(err)
}
Expand Down

0 comments on commit 636a363

Please sign in to comment.