diff --git a/infoschema/tables.go b/infoschema/tables.go index 67646bd64a684..faa83fd22570d 100644 --- a/infoschema/tables.go +++ b/infoschema/tables.go @@ -14,6 +14,7 @@ package infoschema import ( + "encoding/json" "fmt" "sort" "sync" @@ -33,6 +34,7 @@ import ( "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/types" + binaryJson "github.com/pingcap/tidb/types/json" "github.com/pingcap/tidb/util/pdapi" "github.com/pingcap/tidb/util/sqlexec" ) @@ -73,6 +75,7 @@ const ( tableTiDBIndexes = "TIDB_INDEXES" tableSlowLog = "SLOW_QUERY" tableTiDBHotRegions = "TIDB_HOT_REGIONS" + tableTiKVStoreStatus = "TIKV_STORE_STATUS" tableAnalyzeStatus = "ANALYZE_STATUS" ) @@ -563,6 +566,28 @@ var tableTiDBHotRegionsCols = []columnInfo{ {"FLOW_BYTES", mysql.TypeLonglong, 21, 0, nil, nil}, } +var tableTiKVStoreStatusCols = []columnInfo{ + {"STORE_ID", mysql.TypeLonglong, 21, 0, nil, nil}, + {"ADDRESS", mysql.TypeVarchar, 64, 0, nil, nil}, + {"STORE_STATE", mysql.TypeLonglong, 21, 0, nil, nil}, + {"STORE_STATE_NAME", mysql.TypeVarchar, 64, 0, nil, nil}, + {"LABEL", mysql.TypeJSON, 51, 0, nil, nil}, + {"VERSION", mysql.TypeVarchar, 64, 0, nil, nil}, + {"CAPACITY", mysql.TypeVarchar, 64, 0, nil, nil}, + {"AVAILABLE", mysql.TypeVarchar, 64, 0, nil, nil}, + {"LEADER_COUNT", mysql.TypeLonglong, 21, 0, nil, nil}, + {"LEADER_WEIGHT", mysql.TypeLonglong, 21, 0, nil, nil}, + {"LEADER_SCORE", mysql.TypeLonglong, 21, 0, nil, nil}, + {"LEADER_SIZE", mysql.TypeLonglong, 21, 0, nil, nil}, + {"REGION_COUNT", mysql.TypeLonglong, 21, 0, nil, nil}, + {"REGION_WEIGHT", mysql.TypeLonglong, 21, 0, nil, nil}, + {"REGION_SCORE", mysql.TypeLonglong, 21, 0, nil, nil}, + {"REGION_SIZE", mysql.TypeLonglong, 21, 0, nil, nil}, + {"START_TS", mysql.TypeDatetime, 0, 0, nil, nil}, + {"LAST_HEARTBEAT_TS", mysql.TypeDatetime, 0, 0, nil, nil}, + {"UPTIME", mysql.TypeVarchar, 64, 0, nil, nil}, +} + var tableAnalyzeStatusCols = []columnInfo{ {"TABLE_SCHEMA", mysql.TypeVarchar, 64, 0, nil, nil}, {"TABLE_NAME", mysql.TypeVarchar, 64, 0, nil, nil}, @@ -573,6 +598,63 @@ var tableAnalyzeStatusCols = []columnInfo{ {"STATE", mysql.TypeVarchar, 64, 0, nil, nil}, } +func dataForTiKVStoreStatus(ctx sessionctx.Context) (records [][]types.Datum, err error) { + tikvStore, ok := ctx.GetStore().(tikv.Storage) + if !ok { + return nil, errors.New("Information about TiKV store status can be gotten only when the storage is TiKV") + } + tikvHelper := &helper.Helper{ + Store: tikvStore, + RegionCache: tikvStore.GetRegionCache(), + } + storesStat, err := tikvHelper.GetStoresStat() + if err != nil { + return nil, err + } + for _, storeStat := range storesStat.Stores { + row := make([]types.Datum, len(tableTiKVStoreStatusCols)) + row[0].SetInt64(storeStat.Store.ID) + row[1].SetString(storeStat.Store.Address) + row[2].SetInt64(storeStat.Store.State) + row[3].SetString(storeStat.Store.StateName) + data, err := json.Marshal(storeStat.Store.Labels) + if err != nil { + return nil, err + } + bj := binaryJson.BinaryJSON{} + if err = bj.UnmarshalJSON(data); err != nil { + return nil, err + } + row[4].SetMysqlJSON(bj) + row[5].SetString(storeStat.Store.Version) + row[6].SetString(storeStat.Status.Capacity) + row[7].SetString(storeStat.Status.Available) + row[8].SetInt64(storeStat.Status.LeaderCount) + row[9].SetInt64(storeStat.Status.LeaderWeight) + row[10].SetInt64(storeStat.Status.LeaderScore) + row[11].SetInt64(storeStat.Status.LeaderSize) + row[12].SetInt64(storeStat.Status.RegionCount) + row[13].SetInt64(storeStat.Status.RegionWeight) + row[14].SetInt64(storeStat.Status.RegionScore) + row[15].SetInt64(storeStat.Status.RegionSize) + startTs := types.Time{ + Time: types.FromGoTime(storeStat.Status.StartTs), + Type: mysql.TypeDatetime, + Fsp: types.DefaultFsp, + } + row[16].SetMysqlTime(startTs) + lastHeartbeatTs := types.Time{ + Time: types.FromGoTime(storeStat.Status.LastHeartbeatTs), + Type: mysql.TypeDatetime, + Fsp: types.DefaultFsp, + } + row[17].SetMysqlTime(lastHeartbeatTs) + row[18].SetString(storeStat.Status.Uptime) + records = append(records, row) + } + return records, nil +} + func dataForCharacterSets() (records [][]types.Datum) { charsets := charset.GetSupportedCharsets() @@ -1552,6 +1634,7 @@ var tableNameToColumns = map[string][]columnInfo{ tableTiDBIndexes: tableTiDBIndexesCols, tableSlowLog: slowQueryCols, tableTiDBHotRegions: tableTiDBHotRegionsCols, + tableTiKVStoreStatus: tableTiKVStoreStatusCols, tableAnalyzeStatus: tableAnalyzeStatusCols, } @@ -1648,6 +1731,8 @@ func (it *infoschemaTable) getRows(ctx sessionctx.Context, cols []*table.Column) fullRows, err = dataForSlowLog(ctx) case tableTiDBHotRegions: fullRows, err = dataForTiDBHotRegions(ctx) + case tableTiKVStoreStatus: + fullRows, err = dataForTiKVStoreStatus(ctx) case tableAnalyzeStatus: fullRows = DataForAnalyzeStatus() } diff --git a/store/helper/helper.go b/store/helper/helper.go index 392a67404704b..d9f05f86c15d2 100644 --- a/store/helper/helper.go +++ b/store/helper/helper.go @@ -29,6 +29,7 @@ import ( "github.com/pingcap/tidb/store/tikv/tikvrpc" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/util/logutil" + "github.com/pingcap/tidb/util/pdapi" "go.uber.org/zap" ) @@ -365,3 +366,82 @@ func (r *RegionFrameRange) GetIndexFrame(tableID, indexID int64, dbName, tableNa } return nil } + +// StoresStat stores all information get from PD's api. +type StoresStat struct { + Count int `json:"count"` + Stores []StoreStat `json:"stores"` +} + +// StoreStat stores information of one store. +type StoreStat struct { + Store StoreBaseStat `json:"store"` + Status StoreDetailStat `json:"status"` +} + +// StoreBaseStat stores the basic information of one store. +type StoreBaseStat struct { + ID int64 `json:"id"` + Address string `json:"address"` + State int64 `json:"state"` + StateName string `json:"state_name"` + Version string `json:"version"` + Labels []StoreLabel `json:"labels"` +} + +// StoreLabel stores the information of one store label. +type StoreLabel struct { + Key string `json:"key"` + Value string `json:"value"` +} + +// StoreDetailStat stores the detail information of one store. +type StoreDetailStat struct { + Capacity string `json:"capacity"` + Available string `json:"available"` + LeaderCount int64 `json:"leader_count"` + LeaderWeight int64 `json:"leader_weight"` + LeaderScore int64 `json:"leader_score"` + LeaderSize int64 `json:"leader_size"` + RegionCount int64 `json:"region_count"` + RegionWeight int64 `json:"region_weight"` + RegionScore int64 `json:"region_score"` + RegionSize int64 `json:"region_size"` + StartTs time.Time `json:"start_ts"` + LastHeartbeatTs time.Time `json:"last_heartbeat_ts"` + Uptime string `json:"uptime"` +} + +// GetStoresStat gets the TiKV store information by accessing PD's api. +func (h *Helper) GetStoresStat() (*StoresStat, error) { + etcd, ok := h.Store.(tikv.EtcdBackend) + if !ok { + return nil, errors.WithStack(errors.New("not implemented")) + } + pdHosts := etcd.EtcdAddrs() + if len(pdHosts) == 0 { + return nil, errors.New("pd unavailable") + } + req, err := http.NewRequest("GET", protocol+pdHosts[0]+pdapi.Stores, nil) + if err != nil { + return nil, errors.Trace(err) + } + timeout, cancelFunc := context.WithTimeout(context.Background(), 50*time.Millisecond) + resp, err := http.DefaultClient.Do(req.WithContext(timeout)) + defer cancelFunc() + if err != nil { + return nil, errors.Trace(err) + } + defer func() { + err = resp.Body.Close() + if err != nil { + logutil.Logger(context.Background()).Error("close body failed", zap.Error(err)) + } + }() + var storesStat StoresStat + err = json.NewDecoder(resp.Body).Decode(&storesStat) + if err != nil { + return nil, errors.Trace(err) + } + return &storesStat, nil +} diff --git a/store/helper/helper_test.go b/store/helper/helper_test.go index db98ede717a00..7343ce99e5add 100644 --- a/store/helper/helper_test.go +++ b/store/helper/helper_test.go @@ -82,9 +82,22 @@ func (s *HelperTestSuite) TestHotRegion(c *C) { c.Assert(fmt.Sprintf("%v", regionMetric), Equals, "map[1:{100 1 0}]") } +func (s *HelperTestSuite) TestTiKVStoresStat(c *C) { + h := helper.Helper{ + Store: s.store, + RegionCache: s.store.GetRegionCache(), + } + stat, err := h.GetStoresStat() + c.Assert(err, IsNil, Commentf("err: %+v", err)) + data, err := json.Marshal(stat) + c.Assert(err, IsNil) + c.Assert(fmt.Sprintf("%s", data), Equals, "{\"count\":1,\"stores\":[{\"store\":{\"id\":1,\"address\":\"127.0.0.1:20160\",\"state\":0,\"state_name\":\"Up\",\"version\":\"3.0.0-beta\",\"labels\":[{\"key\":\"test\",\"value\":\"test\"}]},\"status\":{\"capacity\":\"60 GiB\",\"available\":\"100 GiB\",\"leader_count\":10,\"leader_weight\":1,\"leader_score\":1000,\"leader_size\":1000,\"region_count\":200,\"region_weight\":1,\"region_score\":1000,\"region_size\":1000,\"start_ts\":\"2019-04-23T19:30:30+08:00\",\"last_heartbeat_ts\":\"2019-04-23T19:31:30+08:00\",\"uptime\":\"1h30m\"}}]}") +} + func (s *HelperTestSuite) mockPDHTTPServer(c *C) { router := mux.NewRouter() - router.HandleFunc("/pd/api/v1/hotspot/regions/read", s.mockHotRegionResponse) + router.HandleFunc(pdapi.HotRead, s.mockHotRegionResponse) + router.HandleFunc(pdapi.Stores, s.mockStoreStatResponse) serverMux := http.NewServeMux() serverMux.Handle("/", router) server := &http.Server{Addr: "127.0.0.1:10100", Handler: serverMux} @@ -118,3 +131,59 @@ func (s *HelperTestSuite) mockHotRegionResponse(w http.ResponseWriter, req *http } } + +func (s *HelperTestSuite) mockStoreStatResponse(w http.ResponseWriter, req *http.Request) { + w.Header().Set("Content-Type", "application/json") + w.WriteHeader(http.StatusOK) + startTs, err := time.Parse(time.RFC3339, "2019-04-23T19:30:30+08:00") + if err != nil { + log.Panic("mock tikv store api response failed", zap.Error(err)) + } + lastHeartbeatTs, err := time.Parse(time.RFC3339, "2019-04-23T19:31:30+08:00") + if err != nil { + log.Panic("mock tikv store api response failed", zap.Error(err)) + } + storesStat := helper.StoresStat{ + Count: 1, + Stores: []helper.StoreStat{ + { + Store: helper.StoreBaseStat{ + ID: 1, + Address: "127.0.0.1:20160", + State: 0, + StateName: "Up", + Version: "3.0.0-beta", + Labels: []helper.StoreLabel{ + { + Key: "test", + Value: "test", + }, + }, + }, + Status: helper.StoreDetailStat{ + Capacity: "60 GiB", + Available: "100 GiB", + LeaderCount: 10, + LeaderWeight: 1, + LeaderScore: 1000, + LeaderSize: 1000, + RegionCount: 200, + RegionWeight: 1, + RegionScore: 1000, + RegionSize: 1000, + StartTs: startTs, + LastHeartbeatTs: lastHeartbeatTs, + Uptime: "1h30m", + }, + }, + }, + } + data, err := json.MarshalIndent(storesStat, "", " ") + if err != nil { + log.Panic("json marshal failed", zap.Error(err)) + } + _, err = w.Write(data) + if err != nil { + log.Panic("write http response failed", zap.Error(err)) + } +} diff --git a/util/pdapi/const.go b/util/pdapi/const.go index 1cf4d935ff4d0..086d55f99a781 100644 --- a/util/pdapi/const.go +++ b/util/pdapi/const.go @@ -17,4 +17,5 @@ package pdapi const ( HotRead = "/pd/api/v1/hotspot/regions/read" HotWrite = "/pd/api/v1/hotspot/regions/read" + Stores = "/pd/api/v1/stores" )