*: add show table regions syntax (#10612)

This commit is contained in:
crazycs
2019-07-10 11:54:52 +08:00
committed by GitHub
parent 89baed8e17
commit 04cfda8699
11 changed files with 487 additions and 2 deletions

View File

@ -553,6 +553,7 @@ func (b *executorBuilder) buildShow(v *plannercore.Show) Executor {
DBName: model.NewCIStr(v.DBName),
Table: v.Table,
Column: v.Column,
IndexName: v.IndexName,
User: v.User,
Roles: v.Roles,
IfNotExists: v.IfNotExists,

View File

@ -4006,6 +4006,122 @@ func (s *testSuite) TestSplitRegion(c *C) {
tk.MustExec(`split table t by (0),(1000),(1000000)`)
}
func (s *testSuite) TestShowTableRegion(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
tk.MustExec("drop table if exists t_regions1, t_regions")
tk.MustExec("create table t_regions1 (a int key, b int, index idx(b))")
tk.MustExec("create table t_regions (a int key, b int, index idx(b))")
// Test show table regions.
tk.MustExec(`split table t_regions1 by (0)`)
tk.MustExec(`split table t_regions between (-10000) and (10000) regions 4;`)
re := tk.MustQuery("show table t_regions regions")
rows := re.Rows()
// Table t_regions should have 4 regions now.
c.Assert(len(rows), Equals, 4)
c.Assert(len(rows[0]), Equals, 7)
tbl1 := testGetTableByName(c, tk.Se, "test", "t_regions1")
tbl := testGetTableByName(c, tk.Se, "test", "t_regions")
// Check the region start key.
c.Assert(rows[0][1], Matches, fmt.Sprintf("t_%d_.*", tbl1.Meta().ID))
c.Assert(rows[1][1], Equals, fmt.Sprintf("t_%d_r_-5000", tbl.Meta().ID))
c.Assert(rows[2][1], Equals, fmt.Sprintf("t_%d_r_0", tbl.Meta().ID))
c.Assert(rows[3][1], Equals, fmt.Sprintf("t_%d_r_5000", tbl.Meta().ID))
// Test show table index regions.
tk.MustExec(`split table t_regions index idx between (-1000) and (1000) regions 4;`)
re = tk.MustQuery("show table t_regions index idx regions")
rows = re.Rows()
// The index `idx` of table t_regions should have 4 regions now.
c.Assert(len(rows), Equals, 4)
// Check the region start key.
c.Assert(rows[0][1], Equals, fmt.Sprintf("t_%d_i_1_", tbl.Meta().ID))
c.Assert(rows[1][1], Matches, fmt.Sprintf("t_%d_i_1_.*", tbl.Meta().ID))
c.Assert(rows[2][1], Matches, fmt.Sprintf("t_%d_i_1_.*", tbl.Meta().ID))
c.Assert(rows[3][1], Matches, fmt.Sprintf("t_%d_i_1_.*", tbl.Meta().ID))
re = tk.MustQuery("show table t_regions regions")
rows = re.Rows()
// The index `idx` of table t_regions should have 4 regions now.
c.Assert(len(rows), Equals, 7)
// Check the region start key.
c.Assert(rows[0][1], Matches, fmt.Sprintf("t_%d_i_1_.*", tbl.Meta().ID))
c.Assert(rows[1][1], Equals, fmt.Sprintf("t_%d_r_-5000", tbl.Meta().ID))
c.Assert(rows[2][1], Equals, fmt.Sprintf("t_%d_r_0", tbl.Meta().ID))
c.Assert(rows[3][1], Equals, fmt.Sprintf("t_%d_r_5000", tbl.Meta().ID))
c.Assert(rows[4][1], Matches, fmt.Sprintf("t_%d_i_1_.*", tbl.Meta().ID))
c.Assert(rows[5][1], Matches, fmt.Sprintf("t_%d_i_1_.*", tbl.Meta().ID))
c.Assert(rows[6][1], Matches, fmt.Sprintf("t_%d_i_1_.*", tbl.Meta().ID))
// Test unsigned primary key and wait scatter finish.
tk.MustExec("drop table if exists t_regions")
tk.MustExec("create table t_regions (a int unsigned key, b int, index idx(b))")
// Test show table regions.
tk.MustExec(`set @@session.tidb_wait_split_region_finish=1;`)
tk.MustExec(`split table t_regions between (0) and (10000) regions 4;`)
re = tk.MustQuery("show table t_regions regions")
rows = re.Rows()
// Table t_regions should have 4 regions now.
c.Assert(len(rows), Equals, 4)
tbl = testGetTableByName(c, tk.Se, "test", "t_regions")
// Check the region start key.
c.Assert(rows[0][1], Matches, "t_.*")
c.Assert(rows[1][1], Equals, fmt.Sprintf("t_%d_r_2500", tbl.Meta().ID))
c.Assert(rows[2][1], Equals, fmt.Sprintf("t_%d_r_5000", tbl.Meta().ID))
c.Assert(rows[3][1], Equals, fmt.Sprintf("t_%d_r_7500", tbl.Meta().ID))
// Test show table index regions.
tk.MustExec(`split table t_regions index idx between (0) and (1000) regions 4;`)
re = tk.MustQuery("show table t_regions index idx regions")
rows = re.Rows()
// The index `idx` of table t_regions should have 4 regions now.
c.Assert(len(rows), Equals, 4)
// Check the region start key.
c.Assert(rows[0][1], Equals, fmt.Sprintf("t_%d_i_1_", tbl.Meta().ID))
c.Assert(rows[1][1], Matches, fmt.Sprintf("t_%d_i_1_.*", tbl.Meta().ID))
c.Assert(rows[2][1], Matches, fmt.Sprintf("t_%d_i_1_.*", tbl.Meta().ID))
c.Assert(rows[3][1], Matches, fmt.Sprintf("t_%d_i_1_.*", tbl.Meta().ID))
// Test show table regions for partition table when disable split region when create table.
atomic.StoreUint32(&ddl.EnableSplitTableRegion, 0)
tk.MustExec("drop table if exists partition_t;")
tk.MustExec("set @@session.tidb_enable_table_partition = '1';")
tk.MustExec("create table partition_t (a int, b int,index(a)) partition by hash (a) partitions 3")
re = tk.MustQuery("show table partition_t regions")
rows = re.Rows()
// Table t_regions should have 4 regions now.
c.Assert(len(rows), Equals, 1)
c.Assert(rows[0][1], Matches, "t_.*")
// Test show table regions for partition table when enable split region when create table.
atomic.StoreUint32(&ddl.EnableSplitTableRegion, 1)
tk.MustExec("set @@global.tidb_scatter_region=1;")
tk.MustExec("drop table if exists partition_t;")
tk.MustExec("create table partition_t (a int, b int,index(a)) partition by hash (a) partitions 3")
re = tk.MustQuery("show table partition_t regions")
rows = re.Rows()
// Table t_regions should have 4 regions now.
c.Assert(len(rows), Equals, 3)
tbl = testGetTableByName(c, tk.Se, "test", "partition_t")
partitionDef := tbl.Meta().GetPartitionInfo().Definitions
c.Assert(rows[0][1], Matches, fmt.Sprintf("t_%d_.*", partitionDef[0].ID))
c.Assert(rows[1][1], Matches, fmt.Sprintf("t_%d_.*", partitionDef[1].ID))
c.Assert(rows[2][1], Matches, fmt.Sprintf("t_%d_.*", partitionDef[2].ID))
atomic.StoreUint32(&ddl.EnableSplitTableRegion, 0)
}
func testGetTableByName(c *C, ctx sessionctx.Context, db, table string) table.Table {
dom := domain.GetDomain(ctx)
// Make sure the table schema is the new schema.
err := dom.Reload()
c.Assert(err, IsNil)
tbl, err := dom.InfoSchema().TableByName(model.NewCIStr(db), model.NewCIStr(table))
c.Assert(err, IsNil)
return tbl
}
func (s *testSuite) TestIssue10435(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")

View File

@ -37,11 +37,13 @@ import (
"github.com/pingcap/tidb/config"
"github.com/pingcap/tidb/domain"
"github.com/pingcap/tidb/infoschema"
"github.com/pingcap/tidb/kv"
plannercore "github.com/pingcap/tidb/planner/core"
"github.com/pingcap/tidb/plugin"
"github.com/pingcap/tidb/privilege"
"github.com/pingcap/tidb/sessionctx/stmtctx"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/store/tikv"
"github.com/pingcap/tidb/table"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/types/json"
@ -60,6 +62,7 @@ type ShowExec struct {
DBName model.CIStr
Table *ast.TableName // Used for showing columns.
Column *ast.ColumnName // Used for `desc table column`.
IndexName model.CIStr // Used for show table regions.
Flag int // Some flag parsed from sql, such as FULL.
Full bool
User *auth.UserIdentity // Used by show grants, show create user.
@ -178,6 +181,8 @@ func (e *ShowExec) fetchAll() error {
case ast.ShowAnalyzeStatus:
e.fetchShowAnalyzeStatus()
return nil
case ast.ShowRegions:
return e.fetchShowTableRegions()
}
return nil
}
@ -1180,3 +1185,108 @@ func (e *ShowExec) appendRow(row []interface{}) {
}
}
}
func (e *ShowExec) fetchShowTableRegions() error {
store := e.ctx.GetStore()
tikvStore, ok := store.(tikv.Storage)
if !ok {
return nil
}
splitStore, ok := store.(kv.SplitableStore)
if !ok {
return nil
}
tb, err := e.getTable()
if err != nil {
return errors.Trace(err)
}
// Get table regions from from pd, not from regionCache, because the region cache maybe outdated.
var regions []regionMeta
if len(e.IndexName.L) != 0 {
indexInfo := tb.Meta().FindIndexByName(e.IndexName.L)
if indexInfo == nil {
return plannercore.ErrKeyDoesNotExist.GenWithStackByArgs(e.IndexName, tb.Meta().Name)
}
regions, err = getTableIndexRegions(tb, indexInfo, tikvStore, splitStore)
} else {
regions, err = getTableRegions(tb, tikvStore, splitStore)
}
if err != nil {
return err
}
e.fillRegionsToChunk(regions)
return nil
}
func getTableRegions(tb table.Table, tikvStore tikv.Storage, splitStore kv.SplitableStore) ([]regionMeta, error) {
if info := tb.Meta().GetPartitionInfo(); info != nil {
return getPartitionTableRegions(info, tb.(table.PartitionedTable), tikvStore, splitStore)
}
return getPhysicalTableRegions(tb.Meta().ID, tb.Meta(), tikvStore, splitStore, nil)
}
func getTableIndexRegions(tb table.Table, indexInfo *model.IndexInfo, tikvStore tikv.Storage, splitStore kv.SplitableStore) ([]regionMeta, error) {
if info := tb.Meta().GetPartitionInfo(); info != nil {
return getPartitionIndexRegions(info, tb.(table.PartitionedTable), indexInfo, tikvStore, splitStore)
}
return getPhysicalIndexRegions(tb.Meta().ID, indexInfo, tikvStore, splitStore, nil)
}
func getPartitionTableRegions(info *model.PartitionInfo, tbl table.PartitionedTable, tikvStore tikv.Storage, splitStore kv.SplitableStore) ([]regionMeta, error) {
regions := make([]regionMeta, 0, len(info.Definitions))
uniqueRegionMap := make(map[uint64]struct{})
for _, def := range info.Definitions {
pid := def.ID
partition := tbl.GetPartition(pid)
partition.GetPhysicalID()
partitionRegions, err := getPhysicalTableRegions(partition.GetPhysicalID(), tbl.Meta(), tikvStore, splitStore, uniqueRegionMap)
if err != nil {
return nil, err
}
regions = append(regions, partitionRegions...)
}
return regions, nil
}
func getPartitionIndexRegions(info *model.PartitionInfo, tbl table.PartitionedTable, indexInfo *model.IndexInfo, tikvStore tikv.Storage, splitStore kv.SplitableStore) ([]regionMeta, error) {
var regions []regionMeta
uniqueRegionMap := make(map[uint64]struct{})
for _, def := range info.Definitions {
pid := def.ID
partition := tbl.GetPartition(pid)
partition.GetPhysicalID()
partitionRegions, err := getPhysicalIndexRegions(partition.GetPhysicalID(), indexInfo, tikvStore, splitStore, uniqueRegionMap)
if err != nil {
return nil, err
}
regions = append(regions, partitionRegions...)
}
return regions, nil
}
func (e *ShowExec) fillRegionsToChunk(regions []regionMeta) {
for i := range regions {
e.result.AppendUint64(0, regions[i].region.Id)
e.result.AppendString(1, regions[i].start)
e.result.AppendString(2, regions[i].end)
e.result.AppendUint64(3, regions[i].leaderID)
e.result.AppendUint64(4, regions[i].storeID)
peers := ""
for i, peer := range regions[i].region.Peers {
if i > 0 {
peers += ", "
}
peers += strconv.FormatUint(peer.Id, 10)
}
e.result.AppendString(5, peers)
if regions[i].scattering {
e.result.AppendInt64(6, 1)
} else {
e.result.AppendInt64(6, 0)
}
}
}

View File

@ -17,19 +17,23 @@ import (
"bytes"
"context"
"encoding/binary"
"fmt"
"math"
"time"
"github.com/cznic/mathutil"
"github.com/pingcap/errors"
"github.com/pingcap/failpoint"
"github.com/pingcap/kvproto/pkg/metapb"
"github.com/pingcap/parser/model"
"github.com/pingcap/parser/mysql"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/store/tikv"
"github.com/pingcap/tidb/table/tables"
"github.com/pingcap/tidb/tablecodec"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/chunk"
"github.com/pingcap/tidb/util/codec"
"github.com/pingcap/tidb/util/logutil"
"go.uber.org/zap"
)
@ -347,3 +351,165 @@ func (e *SplitTableRegionExec) getSplitTableKeys() ([][]byte, error) {
}
return keys, nil
}
// 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
}
func getPhysicalTableRegions(physicalTableID int64, tableInfo *model.TableInfo, tikvStore tikv.Storage, s kv.SplitableStore, uniqueRegionMap map[uint64]struct{}) ([]regionMeta, error) {
if uniqueRegionMap == nil {
uniqueRegionMap = make(map[uint64]struct{})
}
// for record
startKey, endKey := tablecodec.GetTableHandleKeyRange(physicalTableID)
regionCache := tikvStore.GetRegionCache()
recordRegionMetas, err := regionCache.LoadRegionsInKeyRange(tikv.NewBackoffer(context.Background(), 20000), startKey, endKey)
if err != nil {
return nil, err
}
recordPrefix := tablecodec.GenTableRecordPrefix(physicalTableID)
tablePrefix := tablecodec.GenTablePrefix(physicalTableID)
recordRegions, err := getRegionMeta(recordRegionMetas, uniqueRegionMap, tablePrefix, recordPrefix, nil, physicalTableID, 0)
if err != nil {
return nil, err
}
regions := recordRegions
// for indices
for _, index := range tableInfo.Indices {
if index.State != model.StatePublic {
continue
}
startKey, endKey := tablecodec.GetTableIndexKeyRange(physicalTableID, index.ID)
regionMetas, err := regionCache.LoadRegionsInKeyRange(tikv.NewBackoffer(context.Background(), 20000), startKey, endKey)
if err != nil {
return nil, err
}
indexPrefix := tablecodec.EncodeTableIndexPrefix(physicalTableID, index.ID)
indexRegions, err := getRegionMeta(regionMetas, uniqueRegionMap, tablePrefix, recordPrefix, indexPrefix, physicalTableID, index.ID)
if err != nil {
return nil, err
}
regions = append(regions, indexRegions...)
}
err = checkRegionsStatus(s, regions)
if err != nil {
return nil, err
}
return regions, nil
}
func getPhysicalIndexRegions(physicalTableID int64, indexInfo *model.IndexInfo, tikvStore tikv.Storage, s kv.SplitableStore, uniqueRegionMap map[uint64]struct{}) ([]regionMeta, error) {
if uniqueRegionMap == nil {
uniqueRegionMap = make(map[uint64]struct{})
}
startKey, endKey := tablecodec.GetTableIndexKeyRange(physicalTableID, indexInfo.ID)
regionCache := tikvStore.GetRegionCache()
regions, err := regionCache.LoadRegionsInKeyRange(tikv.NewBackoffer(context.Background(), 20000), startKey, endKey)
if err != nil {
return nil, err
}
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)
if err != nil {
return nil, err
}
err = checkRegionsStatus(s, indexRegions)
if err != nil {
return nil, err
}
return indexRegions, nil
}
func checkRegionsStatus(store kv.SplitableStore, regions []regionMeta) error {
for i := range regions {
scattering, err := store.CheckRegionInScattering(regions[i].region.Id)
if err != nil {
return err
}
regions[i].scattering = scattering
}
return nil
}
func decodeRegionsKey(regions []regionMeta, tablePrefix, recordPrefix, indexPrefix []byte, physicalTableID, indexID int64) {
d := &regionKeyDecoder{
physicalTableID: physicalTableID,
tablePrefix: tablePrefix,
recordPrefix: recordPrefix,
indexPrefix: indexPrefix,
indexID: indexID,
}
for i := range regions {
regions[i].start = d.decodeRegionKey(regions[i].region.StartKey)
regions[i].end = d.decodeRegionKey(regions[i].region.EndKey)
}
}
type regionKeyDecoder struct {
physicalTableID int64
tablePrefix []byte
recordPrefix []byte
indexPrefix []byte
indexID int64
}
func (d *regionKeyDecoder) decodeRegionKey(key []byte) string {
if len(d.indexPrefix) > 0 && bytes.HasPrefix(key, d.indexPrefix) {
return fmt.Sprintf("t_%d_i_%d_%x", d.physicalTableID, d.indexID, key[len(d.indexPrefix):])
} else if len(d.recordPrefix) > 0 && bytes.HasPrefix(key, d.recordPrefix) {
_, handle, err := codec.DecodeInt(key[len(d.recordPrefix):])
if err == nil {
return fmt.Sprintf("t_%d_r_%d", d.physicalTableID, handle)
}
}
if len(d.tablePrefix) > 0 && bytes.HasPrefix(key, d.tablePrefix) {
key = key[len(d.tablePrefix):]
// Has index prefix.
if !bytes.HasPrefix(key, []byte("_i")) {
return fmt.Sprintf("t_%d_%x", d.physicalTableID, key)
}
key = key[2:]
// try to decode index ID.
if _, indexID, err := codec.DecodeInt(key); err == nil {
return fmt.Sprintf("t_%d_i_%d_%x", d.physicalTableID, indexID, key[8:])
}
return fmt.Sprintf("t_%d_i__%x", d.physicalTableID, key)
}
// Has table prefix.
if bytes.HasPrefix(key, []byte("t")) {
key = key[1:]
// try to decode table ID.
if _, tableID, err := codec.DecodeInt(key); err == nil {
return fmt.Sprintf("t_%d_%x", tableID, key[8:])
}
return fmt.Sprintf("t_%x", key)
}
return fmt.Sprintf("%x", key)
}
func getRegionMeta(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 {
continue
}
uniqueRegionMap[r.GetID()] = struct{}{}
regions = append(regions, regionMeta{
region: r.GetMeta(),
leaderID: r.GetLeaderID(),
storeID: r.GetLeaderStoreID(),
})
}
decodeRegionsKey(regions, tablePrefix, recordPrefix, indexPrefix, physicalTableID, indexID)
return regions, nil
}

2
go.mod
View File

@ -15,7 +15,7 @@ require (
github.com/dustin/go-humanize v1.0.0 // indirect
github.com/go-ole/go-ole v1.2.1 // indirect
github.com/go-sql-driver/mysql v0.0.0-20170715192408-3955978caca4
github.com/gogo/protobuf v1.2.0 // indirect
github.com/gogo/protobuf v1.2.0
github.com/golang/protobuf v1.2.0
github.com/google/btree v0.0.0-20180813153112-4030bb1f1f0c
github.com/google/uuid v1.1.1

View File

@ -308,4 +308,5 @@ type Iterator interface {
type SplitableStore interface {
SplitRegion(splitKey Key, scatter bool) (regionID uint64, err error)
WaitScatterRegionFinish(regionID uint64) error
CheckRegionInScattering(regionID uint64) (bool, error)
}

View File

@ -363,7 +363,8 @@ type Show struct {
DBName string
Table *ast.TableName // Used for showing columns.
Column *ast.ColumnName // Used for `desc table column`.
Flag int // Some flag parsed from sql, such as FULL.
IndexName model.CIStr
Flag int // Some flag parsed from sql, such as FULL.
Full bool
User *auth.UserIdentity // Used for show grants.
Roles []*auth.RoleIdentity // Used for show grants.
@ -530,6 +531,14 @@ type SplitRegion struct {
ValueLists [][]types.Datum
}
// SplitRegionStatus represents a split regions status plan.
type SplitRegionStatus struct {
baseSchemaProducer
Table table.Table
IndexInfo *model.IndexInfo
}
// DDL represents a DDL statement plan.
type DDL struct {
baseSchemaProducer

View File

@ -990,6 +990,18 @@ func buildShowDDLJobsFields() *expression.Schema {
return schema
}
func buildTableRegionsSchema() *expression.Schema {
schema := expression.NewSchema(make([]*expression.Column, 0, 10)...)
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))
return schema
}
func buildShowDDLJobQueriesFields() *expression.Schema {
schema := expression.NewSchema(make([]*expression.Column, 0, 1)...)
schema.Append(buildColumn("", "QUERY", mysql.TypeVarchar, 256))
@ -1077,6 +1089,7 @@ func (b *PlanBuilder) buildShow(show *ast.ShowStmt) (Plan, error) {
DBName: show.DBName,
Table: show.Table,
Column: show.Column,
IndexName: show.IndexName,
Flag: show.Flag,
Full: show.Full,
User: show.User,
@ -2139,6 +2152,8 @@ func buildShowSchema(s *ast.ShowStmt, isView bool) (schema *expression.Schema) {
return buildShowEventsSchema()
case ast.ShowWarnings, ast.ShowErrors:
return buildShowWarningsSchema()
case ast.ShowRegions:
return buildTableRegionsSchema()
case ast.ShowEngines:
names = []string{"Engine", "Support", "Comment", "Transactions", "XA", "Savepoints"}
case ast.ShowDatabases:

View File

@ -22,6 +22,7 @@ import (
"time"
"unsafe"
"github.com/gogo/protobuf/proto"
"github.com/google/btree"
"github.com/pingcap/errors"
"github.com/pingcap/kvproto/pkg/metapb"
@ -460,6 +461,26 @@ func (c *RegionCache) ListRegionIDsInKeyRange(bo *Backoffer, startKey, endKey []
return regionIDs, nil
}
// LoadRegionsInKeyRange lists ids of regions in [start_key,end_key].
func (c *RegionCache) LoadRegionsInKeyRange(bo *Backoffer, startKey, endKey []byte) (regions []*Region, err error) {
for {
curRegion, err := c.loadRegion(bo, startKey, false)
if err != nil {
return nil, errors.Trace(err)
}
c.mu.Lock()
c.insertRegionToCache(curRegion)
c.mu.Unlock()
regions = append(regions, curRegion)
if curRegion.Contains(endKey) {
break
}
startKey = curRegion.EndKey()
}
return regions, nil
}
// BatchLoadRegionsFromKey loads at most given numbers of regions to the RegionCache, from the given startKey. Returns
// the endKey of the last loaded region. If some of the regions has no leader, their entries in RegionCache will not be
// updated.
@ -848,6 +869,29 @@ func (r *Region) GetID() uint64 {
return r.meta.GetId()
}
// GetMeta returns region meta.
func (r *Region) GetMeta() *metapb.Region {
return proto.Clone(r.meta).(*metapb.Region)
}
// GetLeaderID returns leader region ID.
func (r *Region) GetLeaderID() uint64 {
store := r.getStore()
if int(store.workStoreIdx) >= len(r.meta.Peers) {
return 0
}
return r.meta.Peers[int(r.getStore().workStoreIdx)].Id
}
// GetLeaderStoreID returns the store ID of the leader region.
func (r *Region) GetLeaderStoreID() uint64 {
store := r.getStore()
if int(store.workStoreIdx) >= len(r.meta.Peers) {
return 0
}
return r.meta.Peers[int(r.getStore().workStoreIdx)].StoreId
}
// WorkStorePeer returns current work store with work peer.
func (r *Region) WorkStorePeer(rs *RegionStore) (store *Store, peer *metapb.Peer, idx int) {
idx = int(rs.workStoreIdx)

View File

@ -121,6 +121,8 @@ func (s *testRegionCacheSuite) TestSimple(c *C) {
c.Assert(r.GetID(), Equals, s.region1)
c.Assert(s.getAddr(c, []byte("a")), Equals, s.storeAddr(s.store1))
s.checkCache(c, 1)
c.Assert(r.GetMeta(), DeepEquals, r.meta)
c.Assert(r.GetLeaderID(), Equals, r.meta.Peers[r.getStore().workStoreIdx].Id)
s.cache.mu.regions[r.VerID()].lastAccess = 0
r = s.cache.searchCachedRegion([]byte("a"), true)
c.Assert(r, IsNil)

View File

@ -135,3 +135,24 @@ func (s *tikvStore) WaitScatterRegionFinish(regionID uint64) error {
}
}
}
// CheckRegionInScattering uses to check whether scatter region finished.
func (s *tikvStore) CheckRegionInScattering(regionID uint64) (bool, error) {
bo := NewBackoffer(context.Background(), locateRegionMaxBackoff)
for {
resp, err := s.pdClient.GetOperator(context.Background(), regionID)
if err == nil && resp != nil {
if !bytes.Equal(resp.Desc, []byte("scatter-region")) || resp.Status != pdpb.OperatorStatus_RUNNING {
return false, nil
}
}
if err != nil {
err = bo.Backoff(BoRegionMiss, errors.New(err.Error()))
} else {
return true, nil
}
if err != nil {
return true, errors.Trace(err)
}
}
}