Skip to content

Commit

Permalink
executor: fix the display of large unsigned handle when show table re…
Browse files Browse the repository at this point in the history
…gions (#21026)
  • Loading branch information
tangenta authored Nov 13, 2020
1 parent 99bdecc commit 5080730
Show file tree
Hide file tree
Showing 2 changed files with 51 additions and 16 deletions.
22 changes: 22 additions & 0 deletions executor/show_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ package executor_test
import (
"context"
"fmt"
"strings"

. "github.com/pingcap/check"
"github.com/pingcap/errors"
Expand Down Expand Up @@ -1010,6 +1011,27 @@ func (s *testAutoRandomSuite) TestAutoRandomBase(c *C) {
))
}

func (s *testSerialSuite) TestAutoRandomWithLargeSignedShowTableRegions(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("create database if not exists auto_random_db;")
defer tk.MustExec("drop database if exists auto_random_db;")
tk.MustExec("use auto_random_db;")
tk.MustExec("drop table if exists t;")

testutil.ConfigTestUtils.SetupAutoRandomTestConfig()
defer testutil.ConfigTestUtils.RestoreAutoRandomTestConfig()
tk.MustExec("create table t (a bigint unsigned auto_random primary key);")
tk.MustExec("set @@global.tidb_scatter_region=1;")
// 18446744073709541615 is MaxUint64 - 10000.
// 18446744073709551615 is the MaxUint64.
tk.MustQuery("split table t between (18446744073709541615) and (18446744073709551615) regions 2;").
Check(testkit.Rows("1 1"))
startKey := tk.MustQuery("show table t regions;").Rows()[1][1].(string)
idx := strings.Index(startKey, "_r_")
c.Assert(idx == -1, IsFalse)
c.Assert(startKey[idx+3] == '-', IsFalse, Commentf("actual key: %s", startKey))
}

func (s *testSuite5) TestShowEscape(c *C) {
tk := testkit.NewTestKit(c, s.store)

Expand Down
45 changes: 29 additions & 16 deletions executor/split.go
Original file line number Diff line number Diff line change
Expand Up @@ -627,6 +627,11 @@ func getPhysicalTableRegions(physicalTableID int64, tableInfo *model.TableInfo,
if uniqueRegionMap == nil {
uniqueRegionMap = make(map[uint64]struct{})
}
// This is used to decode the int handle properly.
var hasUnsignedIntHandle bool
if pkInfo := tableInfo.GetPkColInfo(); pkInfo != nil {
hasUnsignedIntHandle = mysql.HasUnsignedFlag(pkInfo.Flag)
}
// for record
startKey, endKey := tablecodec.GetTableHandleKeyRange(physicalTableID)
regionCache := tikvStore.GetRegionCache()
Expand All @@ -636,7 +641,7 @@ func getPhysicalTableRegions(physicalTableID int64, tableInfo *model.TableInfo,
}
recordPrefix := tablecodec.GenTableRecordPrefix(physicalTableID)
tablePrefix := tablecodec.GenTablePrefix(physicalTableID)
recordRegions, err := getRegionMeta(tikvStore, recordRegionMetas, uniqueRegionMap, tablePrefix, recordPrefix, nil, physicalTableID, 0)
recordRegions, err := getRegionMeta(tikvStore, recordRegionMetas, uniqueRegionMap, tablePrefix, recordPrefix, nil, physicalTableID, 0, hasUnsignedIntHandle)
if err != nil {
return nil, err
}
Expand All @@ -653,7 +658,7 @@ func getPhysicalTableRegions(physicalTableID int64, tableInfo *model.TableInfo,
return nil, err
}
indexPrefix := tablecodec.EncodeTableIndexPrefix(physicalTableID, index.ID)
indexRegions, err := getRegionMeta(tikvStore, regionMetas, uniqueRegionMap, tablePrefix, recordPrefix, indexPrefix, physicalTableID, index.ID)
indexRegions, err := getRegionMeta(tikvStore, regionMetas, uniqueRegionMap, tablePrefix, recordPrefix, indexPrefix, physicalTableID, index.ID, hasUnsignedIntHandle)
if err != nil {
return nil, err
}
Expand All @@ -680,7 +685,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(tikvStore, regions, uniqueRegionMap, tablePrefix, recordPrefix, indexPrefix, physicalTableID, indexInfo.ID)
indexRegions, err := getRegionMeta(tikvStore, regions, uniqueRegionMap, tablePrefix, recordPrefix, indexPrefix, physicalTableID, indexInfo.ID, false)
if err != nil {
return nil, err
}
Expand All @@ -702,13 +707,15 @@ func checkRegionsStatus(store kv.SplittableStore, regions []regionMeta) error {
return nil
}

func decodeRegionsKey(regions []regionMeta, tablePrefix, recordPrefix, indexPrefix []byte, physicalTableID, indexID int64) {
func decodeRegionsKey(regions []regionMeta, tablePrefix, recordPrefix, indexPrefix []byte,
physicalTableID, indexID int64, hasUnsignedIntHandle bool) {
d := &regionKeyDecoder{
physicalTableID: physicalTableID,
tablePrefix: tablePrefix,
recordPrefix: recordPrefix,
indexPrefix: indexPrefix,
indexID: indexID,
physicalTableID: physicalTableID,
tablePrefix: tablePrefix,
recordPrefix: recordPrefix,
indexPrefix: indexPrefix,
indexID: indexID,
hasUnsignedIntHandle: hasUnsignedIntHandle,
}
for i := range regions {
regions[i].start = d.decodeRegionKey(regions[i].region.StartKey)
Expand All @@ -717,11 +724,12 @@ func decodeRegionsKey(regions []regionMeta, tablePrefix, recordPrefix, indexPref
}

type regionKeyDecoder struct {
physicalTableID int64
tablePrefix []byte
recordPrefix []byte
indexPrefix []byte
indexID int64
physicalTableID int64
tablePrefix []byte
recordPrefix []byte
indexPrefix []byte
indexID int64
hasUnsignedIntHandle bool
}

func (d *regionKeyDecoder) decodeRegionKey(key []byte) string {
Expand All @@ -735,6 +743,9 @@ func (d *regionKeyDecoder) decodeRegionKey(key []byte) string {
if isIntHandle {
_, handle, err := codec.DecodeInt(key[len(d.recordPrefix):])
if err == nil {
if d.hasUnsignedIntHandle {
return fmt.Sprintf("t_%d_r_%d", d.physicalTableID, uint64(handle))
}
return fmt.Sprintf("t_%d_r_%d", d.physicalTableID, handle)
}
}
Expand Down Expand Up @@ -765,7 +776,9 @@ func (d *regionKeyDecoder) decodeRegionKey(key []byte) string {
return fmt.Sprintf("%x", key)
}

func getRegionMeta(tikvStore tikv.Storage, 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,
hasUnsignedIntHandle bool) ([]regionMeta, error) {
regions := make([]regionMeta, 0, len(regionMetas))
for _, r := range regionMetas {
if _, ok := uniqueRegionMap[r.GetID()]; ok {
Expand All @@ -782,7 +795,7 @@ func getRegionMeta(tikvStore tikv.Storage, regionMetas []*tikv.Region, uniqueReg
if err != nil {
return regions, err
}
decodeRegionsKey(regions, tablePrefix, recordPrefix, indexPrefix, physicalTableID, indexID)
decodeRegionsKey(regions, tablePrefix, recordPrefix, indexPrefix, physicalTableID, indexID, hasUnsignedIntHandle)
return regions, nil
}

Expand Down

0 comments on commit 5080730

Please sign in to comment.