Skip to content

Commit

Permalink
address comment
Browse files Browse the repository at this point in the history
  • Loading branch information
XuHuaiyu committed May 7, 2019
1 parent 6dfbe96 commit 42d045d
Show file tree
Hide file tree
Showing 4 changed files with 41 additions and 10 deletions.
16 changes: 11 additions & 5 deletions server/statistics_handler.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,10 +14,10 @@
package server

import (
"errors"
"net/http"
"time"

"context"
"github.com/gorilla/mux"
"github.com/pingcap/parser/model"
"github.com/pingcap/parser/mysql"
Expand All @@ -26,9 +26,7 @@ import (
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/gcutil"
"github.com/pingcap/tidb/util/logutil"
"github.com/pingcap/tidb/util/sqlexec"
"go.uber.org/zap"
)

// StatsHandler is the handler for dumping statistics.
Expand All @@ -39,17 +37,21 @@ type StatsHandler struct {
func (s *Server) newStatsHandler() *StatsHandler {
store, ok := s.driver.(*TiDBDriver)
if !ok {
logutil.Logger(context.Background()).Error("Illegal driver")
panic("Illegal driver")
}

do, err := session.GetDomain(store.store)
if err != nil {
logutil.Logger(context.Background()).Error("Failed to get domain", zap.Error(err))
panic("Failed to get domain")
}
return &StatsHandler{do}
}

func (sh StatsHandler) ServeHTTP(w http.ResponseWriter, req *http.Request) {
if req.Method != http.MethodGet {
writeError(w, errors.New("only http GET method is supported"))
return
}
w.Header().Set("Content-Type", "application/json")

params := mux.Vars(req)
Expand Down Expand Up @@ -88,6 +90,10 @@ func (s *Server) newStatsHistoryHandler() *StatsHistoryHandler {
}

func (sh StatsHistoryHandler) ServeHTTP(w http.ResponseWriter, req *http.Request) {
if req.Method != http.MethodGet {
writeError(w, errors.New("only http GET method is supported"))
return
}
w.Header().Set("Content-Type", "application/json")

params := mux.Vars(req)
Expand Down
2 changes: 2 additions & 0 deletions server/statistics_handler_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@ import (
"github.com/pingcap/tidb/statistics/handle"
"github.com/pingcap/tidb/store/mockstore"
"github.com/pingcap/tidb/store/mockstore/mocktikv"
"github.com/sirupsen/logrus"
)

type testDumpStatsSuite struct {
Expand Down Expand Up @@ -130,6 +131,7 @@ func (ds *testDumpStatsSuite) TestDumpStatsAPI(c *C) {
c.Assert(os.Remove(path1), IsNil)
}()

logrus.Warning("after snapshot")
resp1, err = http.Get("http://127.0.0.1:10090/stats/dump/tidb/test/" + snapshot)
c.Assert(err, IsNil)

Expand Down
11 changes: 7 additions & 4 deletions statistics/handle/dump.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@ import (
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/sqlexec"
"github.com/pingcap/tipb/go-tipb"
"github.com/sirupsen/logrus"
)

// JSONTable is used for dumping statistics.
Expand Down Expand Up @@ -85,12 +86,14 @@ func (h *Handle) DumpStatsToJSON(dbName string, tableInfo *model.TableInfo, hist

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 {
return nil, errors.Trace(err)
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
}
logrus.Warning("tbl.ModifyCount", tbl.ModifyCount, "tbl.Indices", len(tbl.Indices))
jsonTbl := &JSONTable{
DatabaseName: dbName,
TableName: tableInfo.Name.L,
Expand Down
22 changes: 21 additions & 1 deletion statistics/handle/handle.go
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@ import (
"github.com/pingcap/tidb/util/chunk"
"github.com/pingcap/tidb/util/logutil"
"github.com/pingcap/tidb/util/sqlexec"
"github.com/sirupsen/logrus"
atomic2 "go.uber.org/atomic"
"go.uber.org/zap"
)
Expand Down Expand Up @@ -354,6 +355,7 @@ func (h *Handle) indexStatsFromStorage(row chunk.Row, table *statistics.Table, t
continue
}
if idx == nil || idx.LastUpdateVersion < histVer {
logrus.Warning("go into this")
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)
Expand Down Expand Up @@ -404,6 +406,7 @@ func (h *Handle) columnStatsFromStorage(row chunk.Row, table *statistics.Table,
!isHandle &&
(col == nil || col.Len() == 0 && col.LastUpdateVersion < histVer) &&
!loadAll
logrus.Warning("notNeedLoad", notNeedLoad)
if notNeedLoad {
count, err := h.columnCountFromStorage(table.PhysicalID, histID)
if err != nil {
Expand Down Expand Up @@ -463,7 +466,7 @@ func (h *Handle) tableStatsFromStorage(tableInfo *model.TableInfo, physicalID in
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,
Expand Down Expand Up @@ -673,3 +676,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
}

0 comments on commit 42d045d

Please sign in to comment.