Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
30 commits
Select commit Hold shift + click to select a range
4c26d55
executor: spilt helper.GetTablesInfoWithKeyRange from GetRegionsTabl…
IcePigZDB Dec 7, 2021
79a6d8c
executor: use GetRegionsTableInfo to get region schema
IcePigZDB Dec 7, 2021
eff3da6
executor: formate
IcePigZDB Dec 7, 2021
0bdc394
helper: adapt to the current usage of dump, infoschema_reader.
IcePigZDB Dec 7, 2021
f6015c1
Merge branch 'master' into fix-hot-regions
IcePigZDB Dec 8, 2021
f203272
executor: filter virtual schema
IcePigZDB Dec 8, 2021
76a25e8
helper: formate FilterMemDBs
IcePigZDB Dec 10, 2021
a6b2a62
Merge branch 'master' into fix-hot-regions
IcePigZDB Dec 10, 2021
14793db
executor: formate tidb_hot_regions_history retriver test
IcePigZDB Dec 13, 2021
c2b65e7
Merge branch 'master' into fix-hot-regions
IcePigZDB Dec 13, 2021
3498de8
executor: check exist before iterate regionsTableInfos
IcePigZDB Dec 13, 2021
4c1099e
Merge branch 'master' into fix-hot-regions
IcePigZDB Dec 14, 2021
bd2b5d5
Merge branch 'master' into fix-hot-regions
IcePigZDB Dec 14, 2021
4772d25
Merge branch 'master' into fix-hot-regions
IcePigZDB Dec 15, 2021
6174c54
store/helper: support []*RegionInfo in ParseRegionsTableInfos
IcePigZDB Dec 18, 2021
5e8a9c4
executor: use []*RegionInfo in ParseRegionsTableInfos
IcePigZDB Dec 18, 2021
77bc777
Merge branch 'master' into fix-hot-regions
IcePigZDB Dec 18, 2021
e1470d6
Update store/helper/helper.go
IcePigZDB Dec 27, 2021
321f079
Update store/helper/helper.go
IcePigZDB Dec 27, 2021
53a4f6f
executor: cancel set null for hot metrics
IcePigZDB Dec 27, 2021
6c54c3b
store/helper: export BytesKeyToHex
IcePigZDB Dec 27, 2021
2bc8e18
address comment
IcePigZDB Dec 27, 2021
786c516
Merge branch 'master' into fix-hot-regions
IcePigZDB Dec 27, 2021
7d1039c
store/helper: update comment
IcePigZDB Dec 27, 2021
02385df
store/helper: export NewTableWithKeyRange and NewIndexWithKeyRange fo…
IcePigZDB Dec 28, 2021
8b54c3d
executor: update tidb_hot_regions_history retriver and it's test
IcePigZDB Dec 28, 2021
4688f85
Merge branch 'master' into fix-hot-regions
IcePigZDB Dec 28, 2021
5166f9d
Merge branch 'master' into fix-hot-regions
ti-chi-bot Dec 29, 2021
079a5b0
Merge branch 'master' into fix-hot-regions
ti-chi-bot Dec 29, 2021
17eb2b6
Merge branch 'master' into fix-hot-regions
ti-chi-bot Dec 29, 2021
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
161 changes: 74 additions & 87 deletions executor/memtable_reader.go
Original file line number Diff line number Diff line change
Expand Up @@ -40,15 +40,14 @@ import (
plannercore "github.com/pingcap/tidb/planner/core"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/sessiontxn"
"github.com/pingcap/tidb/store/helper"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util"
"github.com/pingcap/tidb/util/chunk"
"github.com/pingcap/tidb/util/codec"
"github.com/pingcap/tidb/util/execdetails"
"github.com/pingcap/tidb/util/pdapi"
"github.com/pingcap/tidb/util/set"
"github.com/tikv/client-go/v2/tikv"
"go.uber.org/zap"
"google.golang.org/grpc"
"google.golang.org/grpc/credentials"
Expand Down Expand Up @@ -767,19 +766,19 @@ type HistoryHotRegions struct {
// HistoryHotRegion records each hot region's statistics.
// it's the response of PD.
type HistoryHotRegion struct {
UpdateTime int64 `json:"update_time,omitempty"`
RegionID uint64 `json:"region_id,omitempty"`
StoreID uint64 `json:"store_id,omitempty"`
PeerID uint64 `json:"peer_id,omitempty"`
IsLearner bool `json:"is_learner,omitempty"`
IsLeader bool `json:"is_leader,omitempty"`
HotRegionType string `json:"hot_region_type,omitempty"`
HotDegree int64 `json:"hot_degree,omitempty"`
FlowBytes float64 `json:"flow_bytes,omitempty"`
KeyRate float64 `json:"key_rate,omitempty"`
QueryRate float64 `json:"query_rate,omitempty"`
StartKey []byte `json:"start_key,omitempty"`
EndKey []byte `json:"end_key,omitempty"`
UpdateTime int64 `json:"update_time"`
RegionID uint64 `json:"region_id"`
StoreID uint64 `json:"store_id"`
PeerID uint64 `json:"peer_id"`
IsLearner bool `json:"is_learner"`
IsLeader bool `json:"is_leader"`
HotRegionType string `json:"hot_region_type"`
HotDegree int64 `json:"hot_degree"`
FlowBytes float64 `json:"flow_bytes"`
KeyRate float64 `json:"key_rate"`
QueryRate float64 `json:"query_rate"`
StartKey string `json:"start_key"`
EndKey string `json:"end_key"`
}

func (e *hotRegionsHistoryRetriver) initialize(ctx context.Context, sctx sessionctx.Context) ([]chan hotRegionsResult, error) {
Expand Down Expand Up @@ -893,8 +892,6 @@ func (e *hotRegionsHistoryRetriver) retrieve(ctx context.Context, sctx sessionct
}
// Merge the results
var finalRows [][]types.Datum
allSchemas := sctx.GetInfoSchema().(infoschema.InfoSchema).AllSchemas()
tz := sctx.GetSessionVars().Location()
tikvStore, ok := sctx.GetStore().(helper.Storage)
if !ok {
return nil, errors.New("Information about hot region can be gotten only when the storage is TiKV")
Expand All @@ -903,14 +900,18 @@ func (e *hotRegionsHistoryRetriver) retrieve(ctx context.Context, sctx sessionct
Store: tikvStore,
RegionCache: tikvStore.GetRegionCache(),
}
tz := sctx.GetSessionVars().Location()
allSchemas := sessiontxn.GetTxnManager(sctx).GetTxnInfoSchema().AllSchemas()
schemas := tikvHelper.FilterMemDBs(allSchemas)
tables := tikvHelper.GetTablesInfoWithKeyRange(schemas)
for e.heap.Len() > 0 && len(finalRows) < hotRegionsHistoryBatchSize {
minTimeItem := heap.Pop(e.heap).(hotRegionsResult)
row, err := e.getHotRegionRowWithSchemaInfo(minTimeItem.messages.HistoryHotRegion[0], tikvHelper, allSchemas, tz)
rows, err := e.getHotRegionRowWithSchemaInfo(minTimeItem.messages.HistoryHotRegion[0], tikvHelper, tables, tz)
if err != nil {
return nil, err
}
if row != nil {
finalRows = append(finalRows, row)
if rows != nil {
finalRows = append(finalRows, rows...)
}
minTimeItem.messages.HistoryHotRegion = minTimeItem.messages.HistoryHotRegion[1:]
// Fetch next message item
Expand All @@ -926,74 +927,60 @@ func (e *hotRegionsHistoryRetriver) retrieve(ctx context.Context, sctx sessionct
func (e *hotRegionsHistoryRetriver) getHotRegionRowWithSchemaInfo(
hisHotRegion *HistoryHotRegion,
tikvHelper *helper.Helper,
allSchemas []*model.DBInfo,
tables []helper.TableInfoWithKeyRange,
tz *time.Location,
) ([]types.Datum, error) {
_, startKey, _ := codec.DecodeBytes(hisHotRegion.StartKey, []byte{})
_, endKey, _ := codec.DecodeBytes(hisHotRegion.EndKey, []byte{})
region := &tikv.KeyLocation{StartKey: startKey, EndKey: endKey}
hotRange, err := helper.NewRegionFrameRange(region)
if err != nil {
return nil, err
}

f := tikvHelper.FindTableIndexOfRegion(allSchemas, hotRange)
// Ignore row without corresponding schema f.
if f == nil {
return nil, nil
}
row := make([]types.Datum, len(infoschema.TableTiDBHotRegionsHistoryCols))
updateTimestamp := time.Unix(hisHotRegion.UpdateTime/1000, (hisHotRegion.UpdateTime%1000)*int64(time.Millisecond))

if updateTimestamp.Location() != tz {
updateTimestamp.In(tz)
}
updateTime := types.NewTime(types.FromGoTime(updateTimestamp), mysql.TypeTimestamp, types.MinFsp)
row[0].SetMysqlTime(updateTime)
row[1].SetString(strings.ToUpper(f.DBName), mysql.DefaultCollationName)
row[2].SetString(strings.ToUpper(f.TableName), mysql.DefaultCollationName)
row[3].SetInt64(f.TableID)
if f.IndexName != "" {
row[4].SetString(strings.ToUpper(f.IndexName), mysql.DefaultCollationName)
row[5].SetInt64(f.IndexID)
} else {
row[4].SetNull()
row[5].SetNull()
}
row[6].SetInt64(int64(hisHotRegion.RegionID))
row[7].SetInt64(int64(hisHotRegion.StoreID))
row[8].SetInt64(int64(hisHotRegion.PeerID))
if hisHotRegion.IsLearner {
row[9].SetInt64(1)
} else {
row[9].SetInt64(0)
}
if hisHotRegion.IsLeader {
row[10].SetInt64(1)
} else {
row[10].SetInt64(0)
) ([][]types.Datum, error) {
regionsInfo := []*helper.RegionInfo{
{
ID: int64(hisHotRegion.RegionID),
StartKey: hisHotRegion.StartKey,
EndKey: hisHotRegion.EndKey,
}}
regionsTableInfos := tikvHelper.ParseRegionsTableInfos(regionsInfo, tables)

var rows [][]types.Datum
// Ignore row without corresponding schema.
if tableInfos, ok := regionsTableInfos[int64(hisHotRegion.RegionID)]; ok {
for _, tableInfo := range tableInfos {
updateTimestamp := time.Unix(hisHotRegion.UpdateTime/1000, (hisHotRegion.UpdateTime%1000)*int64(time.Millisecond))
if updateTimestamp.Location() != tz {
updateTimestamp.In(tz)
}
updateTime := types.NewTime(types.FromGoTime(updateTimestamp), mysql.TypeTimestamp, types.MinFsp)
row := make([]types.Datum, len(infoschema.TableTiDBHotRegionsHistoryCols))

row[0].SetMysqlTime(updateTime)
row[1].SetString(strings.ToUpper(tableInfo.DB.Name.O), mysql.DefaultCollationName)
row[2].SetString(strings.ToUpper(tableInfo.Table.Name.O), mysql.DefaultCollationName)
row[3].SetInt64(tableInfo.Table.ID)
if tableInfo.IsIndex {
row[4].SetString(strings.ToUpper(tableInfo.Index.Name.O), mysql.DefaultCollationName)
row[5].SetInt64(tableInfo.Index.ID)
} else {
row[4].SetNull()
row[5].SetNull()
}
row[6].SetInt64(int64(hisHotRegion.RegionID))
row[7].SetInt64(int64(hisHotRegion.StoreID))
row[8].SetInt64(int64(hisHotRegion.PeerID))
if hisHotRegion.IsLearner {
row[9].SetInt64(1)
} else {
row[9].SetInt64(0)
}
if hisHotRegion.IsLeader {
row[10].SetInt64(1)
} else {
row[10].SetInt64(0)
}
row[11].SetString(strings.ToUpper(hisHotRegion.HotRegionType), mysql.DefaultCollationName)
row[12].SetInt64(hisHotRegion.HotDegree)
row[13].SetFloat64(hisHotRegion.FlowBytes)
row[14].SetFloat64(hisHotRegion.KeyRate)
row[15].SetFloat64(hisHotRegion.QueryRate)
rows = append(rows, row)
}
}

row[11].SetString(strings.ToUpper(hisHotRegion.HotRegionType), mysql.DefaultCollationName)
if hisHotRegion.HotDegree != 0 {
row[12].SetInt64(hisHotRegion.HotDegree)
} else {
row[12].SetNull()
}
if hisHotRegion.FlowBytes != 0 {
row[13].SetFloat64(hisHotRegion.FlowBytes)
} else {
row[13].SetNull()
}
if hisHotRegion.KeyRate != 0 {
row[14].SetFloat64(hisHotRegion.KeyRate)
} else {
row[14].SetNull()
}
if hisHotRegion.QueryRate != 0 {
row[15].SetFloat64(hisHotRegion.QueryRate)
} else {
row[15].SetNull()
}
return row, nil
return rows, nil
}
Loading