Skip to content

Commit

Permalink
*: add region read/write bytes info and approximate size/keys in show…
Browse files Browse the repository at this point in the history
… table regions result. (#11847)
  • Loading branch information
crazycs520 authored and winkyao committed Aug 29, 2019
1 parent 8c5f4c7 commit ba4eb8f
Show file tree
Hide file tree
Showing 6 changed files with 70 additions and 16 deletions.
2 changes: 1 addition & 1 deletion executor/executor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -4145,7 +4145,7 @@ func (s *testSuite) TestShowTableRegion(c *C) {
// 4 regions to store record data.
// 1 region to store index data.
c.Assert(len(rows), Equals, 5)
c.Assert(len(rows[0]), Equals, 7)
c.Assert(len(rows[0]), Equals, 11)
tbl := testGetTableByName(c, tk.Se, "test", "t_regions")
// Check the region start key.
c.Assert(rows[0][1], Equals, fmt.Sprintf("t_%d_r", tbl.Meta().ID))
Expand Down
5 changes: 5 additions & 0 deletions executor/show.go
Original file line number Diff line number Diff line change
Expand Up @@ -1288,5 +1288,10 @@ func (e *ShowExec) fillRegionsToChunk(regions []regionMeta) {
} else {
e.result.AppendInt64(6, 0)
}

e.result.AppendInt64(7, regions[i].writtenBytes)
e.result.AppendInt64(8, regions[i].readBytes)
e.result.AppendInt64(9, regions[i].approximateSize)
e.result.AppendInt64(10, regions[i].approximateKeys)
}
}
56 changes: 46 additions & 10 deletions executor/split.go
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@ import (
"github.com/pingcap/parser/mysql"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/store/helper"
"github.com/pingcap/tidb/store/tikv"
"github.com/pingcap/tidb/table/tables"
"github.com/pingcap/tidb/tablecodec"
Expand Down Expand Up @@ -439,12 +440,16 @@ func (e *SplitTableRegionExec) getSplitTableKeys() ([][]byte, error) {

// RegionMeta contains a region's peer detail
type regionMeta struct {
region *metapb.Region
leaderID uint64
storeID uint64 // storeID is the store ID of the leader region.
start string
end string
scattering bool
region *metapb.Region
leaderID uint64
storeID uint64 // storeID is the store ID of the leader region.
start string
end string
scattering bool
writtenBytes int64
readBytes int64
approximateSize int64
approximateKeys int64
}

func getPhysicalTableRegions(physicalTableID int64, tableInfo *model.TableInfo, tikvStore tikv.Storage, s kv.SplitableStore, uniqueRegionMap map[uint64]struct{}) ([]regionMeta, error) {
Expand All @@ -460,7 +465,7 @@ func getPhysicalTableRegions(physicalTableID int64, tableInfo *model.TableInfo,
}
recordPrefix := tablecodec.GenTableRecordPrefix(physicalTableID)
tablePrefix := tablecodec.GenTablePrefix(physicalTableID)
recordRegions, err := getRegionMeta(recordRegionMetas, uniqueRegionMap, tablePrefix, recordPrefix, nil, physicalTableID, 0)
recordRegions, err := getRegionMeta(tikvStore, recordRegionMetas, uniqueRegionMap, tablePrefix, recordPrefix, nil, physicalTableID, 0)
if err != nil {
return nil, err
}
Expand All @@ -477,7 +482,7 @@ func getPhysicalTableRegions(physicalTableID int64, tableInfo *model.TableInfo,
return nil, err
}
indexPrefix := tablecodec.EncodeTableIndexPrefix(physicalTableID, index.ID)
indexRegions, err := getRegionMeta(regionMetas, uniqueRegionMap, tablePrefix, recordPrefix, indexPrefix, physicalTableID, index.ID)
indexRegions, err := getRegionMeta(tikvStore, regionMetas, uniqueRegionMap, tablePrefix, recordPrefix, indexPrefix, physicalTableID, index.ID)
if err != nil {
return nil, err
}
Expand All @@ -504,7 +509,7 @@ func getPhysicalIndexRegions(physicalTableID int64, indexInfo *model.IndexInfo,
recordPrefix := tablecodec.GenTableRecordPrefix(physicalTableID)
tablePrefix := tablecodec.GenTablePrefix(physicalTableID)
indexPrefix := tablecodec.EncodeTableIndexPrefix(physicalTableID, indexInfo.ID)
indexRegions, err := getRegionMeta(regions, uniqueRegionMap, tablePrefix, recordPrefix, indexPrefix, physicalTableID, indexInfo.ID)
indexRegions, err := getRegionMeta(tikvStore, regions, uniqueRegionMap, tablePrefix, recordPrefix, indexPrefix, physicalTableID, indexInfo.ID)
if err != nil {
return nil, err
}
Expand Down Expand Up @@ -585,7 +590,7 @@ func (d *regionKeyDecoder) decodeRegionKey(key []byte) string {
return fmt.Sprintf("%x", key)
}

func getRegionMeta(regionMetas []*tikv.Region, uniqueRegionMap map[uint64]struct{}, tablePrefix, recordPrefix, indexPrefix []byte, physicalTableID, indexID int64) ([]regionMeta, error) {
func getRegionMeta(tikvStore tikv.Storage, regionMetas []*tikv.Region, uniqueRegionMap map[uint64]struct{}, tablePrefix, recordPrefix, indexPrefix []byte, physicalTableID, indexID int64) ([]regionMeta, error) {
regions := make([]regionMeta, 0, len(regionMetas))
for _, r := range regionMetas {
if _, ok := uniqueRegionMap[r.GetID()]; ok {
Expand All @@ -598,6 +603,37 @@ func getRegionMeta(regionMetas []*tikv.Region, uniqueRegionMap map[uint64]struct
storeID: r.GetLeaderStoreID(),
})
}
regions, err := getRegionInfo(tikvStore, regions)
if err != nil {
return regions, err
}
decodeRegionsKey(regions, tablePrefix, recordPrefix, indexPrefix, physicalTableID, indexID)
return regions, nil
}

func getRegionInfo(store tikv.Storage, regions []regionMeta) ([]regionMeta, error) {
// check pd server exists.
etcd, ok := store.(tikv.EtcdBackend)
if !ok {
return regions, nil
}
pdHosts := etcd.EtcdAddrs()
if len(pdHosts) == 0 {
return regions, nil
}
tikvHelper := &helper.Helper{
Store: store,
RegionCache: store.GetRegionCache(),
}
for i := range regions {
regionInfo, err := tikvHelper.GetRegionInfoByID(regions[i].region.Id)
if err != nil {
return nil, err
}
regions[i].writtenBytes = regionInfo.WrittenBytes
regions[i].readBytes = regionInfo.ReadBytes
regions[i].approximateSize = regionInfo.ApproximateSize
regions[i].approximateKeys = regionInfo.ApproximateKeys
}
return regions, nil
}
6 changes: 5 additions & 1 deletion planner/core/planbuilder.go
Original file line number Diff line number Diff line change
Expand Up @@ -1230,14 +1230,18 @@ func buildShowDDLJobsFields() *expression.Schema {
}

func buildTableRegionsSchema() *expression.Schema {
schema := expression.NewSchema(make([]*expression.Column, 0, 10)...)
schema := expression.NewSchema(make([]*expression.Column, 0, 11)...)
schema.Append(buildColumn("", "REGION_ID", mysql.TypeLonglong, 4))
schema.Append(buildColumn("", "START_KEY", mysql.TypeVarchar, 64))
schema.Append(buildColumn("", "END_KEY", mysql.TypeVarchar, 64))
schema.Append(buildColumn("", "LEADER_ID", mysql.TypeLonglong, 4))
schema.Append(buildColumn("", "LEADER_STORE_ID", mysql.TypeLonglong, 4))
schema.Append(buildColumn("", "PEERS", mysql.TypeVarchar, 64))
schema.Append(buildColumn("", "SCATTERING", mysql.TypeTiny, 1))
schema.Append(buildColumn("", "WRITTEN_BYTES", mysql.TypeLonglong, 4))
schema.Append(buildColumn("", "READ_BYTES", mysql.TypeLonglong, 4))
schema.Append(buildColumn("", "APPROXIMATE_SIZE(MB)", mysql.TypeLonglong, 4))
schema.Append(buildColumn("", "APPROXIMATE_KEYS", mysql.TypeLonglong, 4))
return schema
}

Expand Down
8 changes: 8 additions & 0 deletions store/helper/helper.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ import (
"math"
"net/http"
"sort"
"strconv"
"strings"
"time"

Expand Down Expand Up @@ -580,6 +581,13 @@ func (h *Helper) GetRegionsInfo() (*RegionsInfo, error) {
return &regionsInfo, err
}

// GetRegionInfoByID gets the region information of the region ID by using PD's api.
func (h *Helper) GetRegionInfoByID(regionID uint64) (*RegionInfo, error) {
var regionInfo RegionInfo
err := h.requestPD("GET", pdapi.RegionByID+strconv.FormatUint(regionID, 10), nil, &regionInfo)
return &regionInfo, err
}

// request PD API, decode the response body into res
func (h *Helper) requestPD(method, uri string, body io.Reader, res interface{}) error {
etcd, ok := h.Store.(tikv.EtcdBackend)
Expand Down
9 changes: 5 additions & 4 deletions util/pdapi/const.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,8 +15,9 @@ package pdapi

// The following constants are the APIs of PD server.
const (
HotRead = "/pd/api/v1/hotspot/regions/read"
HotWrite = "/pd/api/v1/hotspot/regions/write"
Regions = "/pd/api/v1/regions"
Stores = "/pd/api/v1/stores"
HotRead = "/pd/api/v1/hotspot/regions/read"
HotWrite = "/pd/api/v1/hotspot/regions/write"
Regions = "/pd/api/v1/regions"
RegionByID = "/pd/api/v1//region/id/"
Stores = "/pd/api/v1/stores"
)

0 comments on commit ba4eb8f

Please sign in to comment.