*: add show table regions syntax (#10612)
This commit is contained in:
@ -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,
|
||||
|
||||
@ -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")
|
||||
|
||||
110
executor/show.go
110
executor/show.go
@ -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)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@ -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 := ®ionKeyDecoder{
|
||||
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
2
go.mod
@ -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
|
||||
|
||||
1
kv/kv.go
1
kv/kv.go
@ -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)
|
||||
}
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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:
|
||||
|
||||
@ -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)
|
||||
|
||||
@ -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)
|
||||
|
||||
@ -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)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user