Skip to content

Commit

Permalink
*: split index region with lower upper syntax (#10409)
Browse files Browse the repository at this point in the history
  • Loading branch information
crazycs520 authored and zz-jason committed Jun 6, 2019
1 parent f67352d commit 7bf3d69
Show file tree
Hide file tree
Showing 9 changed files with 571 additions and 48 deletions.
9 changes: 6 additions & 3 deletions executor/builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -194,7 +194,7 @@ func (b *executorBuilder) build(p plannercore.Plan) Executor {
return b.buildWindow(v)
case *plannercore.SQLBindPlan:
return b.buildSQLBindExec(v)
case *plannercore.SplitIndexRegion:
case *plannercore.SplitRegion:
return b.buildSplitIndexRegion(v)
default:
if mp, ok := p.(MockPhysicalPlan); ok {
Expand Down Expand Up @@ -1261,13 +1261,16 @@ func (b *executorBuilder) buildUnionAll(v *plannercore.PhysicalUnionAll) Executo
return e
}

func (b *executorBuilder) buildSplitIndexRegion(v *plannercore.SplitIndexRegion) Executor {
func (b *executorBuilder) buildSplitIndexRegion(v *plannercore.SplitRegion) Executor {
base := newBaseExecutor(b.ctx, nil, v.ExplainID())
base.initCap = chunk.ZeroCapacity
return &SplitIndexRegionExec{
baseExecutor: base,
table: v.Table,
tableInfo: v.TableInfo,
indexInfo: v.IndexInfo,
lower: v.Lower,
upper: v.Upper,
num: v.Num,
valueLists: v.ValueLists,
}
}
Expand Down
2 changes: 1 addition & 1 deletion executor/executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -1386,7 +1386,7 @@ func ResetContextOfStmt(ctx sessionctx.Context, s ast.StmtNode) (err error) {
sc.InShowWarning = true
sc.SetWarnings(vars.StmtCtx.GetWarnings())
}
case *ast.SplitIndexRegionStmt:
case *ast.SplitRegionStmt:
sc.IgnoreTruncate = false
sc.IgnoreZeroInDate = true
sc.AllowInvalidDate = vars.SQLMode.HasAllowInvalidDatesMode()
Expand Down
31 changes: 31 additions & 0 deletions executor/executor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -3801,6 +3801,37 @@ func (s *testSuite) TestSplitIndexRegion(c *C) {
c.Assert(err, NotNil)
terr := errors.Cause(err).(*terror.Error)
c.Assert(terr.Code(), Equals, terror.ErrCode(mysql.WarnDataTruncated))

// Check min value is more than max value.
tk.MustExec(`split table t index idx1 between (0) and (1000000000) regions 10`)
_, err = tk.Exec(`split table t index idx1 between (2,'a') and (1,'c') regions 10`)
c.Assert(err, NotNil)
c.Assert(err.Error(), Equals, "Split index region `idx1` lower value (2,a) should less than the upper value (1,c)")

// Check min value is invalid.
_, err = tk.Exec(`split table t index idx1 between () and (1) regions 10`)
c.Assert(err, NotNil)
c.Assert(err.Error(), Equals, "Split index region `idx1` lower value count should more than 0")

// Check max value is invalid.
_, err = tk.Exec(`split table t index idx1 between (1) and () regions 10`)
c.Assert(err, NotNil)
c.Assert(err.Error(), Equals, "Split index region `idx1` upper value count should more than 0")

// Check pre-split region num is too large.
_, err = tk.Exec(`split table t index idx1 between (0) and (1000000000) regions 10000`)
c.Assert(err, NotNil)
c.Assert(err.Error(), Equals, "Split index region num is exceed the limit 1000")

// Check pre-split region num 0 is invalid.
_, err = tk.Exec(`split table t index idx1 between (0) and (1000000000) regions 0`)
c.Assert(err, NotNil)
c.Assert(err.Error(), Equals, "Split index region num should more than 0")

// Test truncate error msg.
_, err = tk.Exec(`split table t index idx1 between ("aa") and (1000000000) regions 0`)
c.Assert(err, NotNil)
c.Assert(err.Error(), Equals, "[types:1265]Incorrect value: 'aa' for index column 'b'")
}

func (s *testSuite) TestIssue10435(c *C) {
Expand Down
146 changes: 134 additions & 12 deletions executor/split.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,13 +14,17 @@
package executor

import (
"bytes"
"context"
"encoding/binary"
"math"

"github.com/cznic/mathutil"
"github.com/pingcap/errors"
"github.com/pingcap/parser/model"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/table"
"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/logutil"
Expand All @@ -31,8 +35,11 @@ import (
type SplitIndexRegionExec struct {
baseExecutor

table table.Table
tableInfo *model.TableInfo
indexInfo *model.IndexInfo
lower []types.Datum
upper []types.Datum
num int
valueLists [][]types.Datum
}

Expand All @@ -48,18 +55,16 @@ func (e *SplitIndexRegionExec) Next(ctx context.Context, _ *chunk.RecordBatch) e
if !ok {
return nil
}
regionIDs := make([]uint64, 0, len(e.valueLists))
index := tables.NewIndex(e.table.Meta().ID, e.table.Meta(), e.indexInfo)
for _, values := range e.valueLists {
idxKey, _, err := index.GenIndexKey(e.ctx.GetSessionVars().StmtCtx, values, math.MinInt64, nil)
if err != nil {
return err
}

splitIdxKeys, err := e.getSplitIdxKeys()
if err != nil {
return err
}
regionIDs := make([]uint64, 0, len(splitIdxKeys))
for _, idxKey := range splitIdxKeys {
regionID, err := s.SplitRegionAndScatter(idxKey)
if err != nil {
logutil.Logger(context.Background()).Warn("split table index region failed",
zap.String("table", e.table.Meta().Name.L),
zap.String("table", e.tableInfo.Name.L),
zap.String("index", e.indexInfo.Name.L),
zap.Error(err))
continue
Expand All @@ -75,10 +80,127 @@ func (e *SplitIndexRegionExec) Next(ctx context.Context, _ *chunk.RecordBatch) e
if err != nil {
logutil.Logger(context.Background()).Warn("wait scatter region failed",
zap.Uint64("regionID", regionID),
zap.String("table", e.table.Meta().Name.L),
zap.String("table", e.tableInfo.Name.L),
zap.String("index", e.indexInfo.Name.L),
zap.Error(err))
}
}
return nil
}

func (e *SplitIndexRegionExec) getSplitIdxKeys() ([][]byte, error) {
var idxKeys [][]byte
if e.num > 0 {
idxKeys = make([][]byte, 0, e.num)
} else {
idxKeys = make([][]byte, 0, len(e.valueLists)+1)
}
// Split in the start of the index key.
startIdxKey := tablecodec.EncodeTableIndexPrefix(e.tableInfo.ID, e.indexInfo.ID)
idxKeys = append(idxKeys, startIdxKey)

index := tables.NewIndex(e.tableInfo.ID, e.tableInfo, e.indexInfo)
// Split index regions by user specified value lists.
if len(e.valueLists) > 0 {
for _, v := range e.valueLists {
idxKey, _, err := index.GenIndexKey(e.ctx.GetSessionVars().StmtCtx, v, math.MinInt64, nil)
if err != nil {
return nil, err
}
idxKeys = append(idxKeys, idxKey)
}
return idxKeys, nil
}
// Split index regions by lower, upper value and calculate the step by (upper - lower)/num.
lowerIdxKey, _, err := index.GenIndexKey(e.ctx.GetSessionVars().StmtCtx, e.lower, math.MinInt64, nil)
if err != nil {
return nil, err
}
// Use math.MinInt64 as handle_id for the upper index key to avoid affecting calculate split point.
// If use math.MaxInt64 here, test of `TestSplitIndex` will report error.
upperIdxKey, _, err := index.GenIndexKey(e.ctx.GetSessionVars().StmtCtx, e.upper, math.MinInt64, nil)
if err != nil {
return nil, err
}
if bytes.Compare(lowerIdxKey, upperIdxKey) >= 0 {
lowerStr, err1 := datumSliceToString(e.lower)
upperStr, err2 := datumSliceToString(e.upper)
if err1 != nil || err2 != nil {
return nil, errors.Errorf("Split index region `%v` lower value %v should less than the upper value %v", e.indexInfo.Name, e.lower, e.upper)
}
return nil, errors.Errorf("Split index region `%v` lower value %v should less than the upper value %v", e.indexInfo.Name, lowerStr, upperStr)
}
return getValuesList(lowerIdxKey, upperIdxKey, e.num, idxKeys), nil
}

// getValuesList is used to get `num` values between lower and upper value.
// To Simplify the explain, suppose lower and upper value type is int64, and lower=0, upper=100, num=10,
// then calculate the step=(upper-lower)/num=10, then the function should return 0+10, 10+10, 20+10... all together 9 (num-1) values.
// Then the function will return [10,20,30,40,50,60,70,80,90].
// The difference is the value type of upper,lower is []byte, So I use getUint64FromBytes to convert []byte to uint64.
func getValuesList(lower, upper []byte, num int, valuesList [][]byte) [][]byte {
commonPrefixIdx := longestCommonPrefixLen(lower, upper)
step := getStepValue(lower[commonPrefixIdx:], upper[commonPrefixIdx:], num)
startV := getUint64FromBytes(lower[commonPrefixIdx:], 0)
// To get `num` regions, only need to split `num-1` idx keys.
buf := make([]byte, 8)
for i := 0; i < num-1; i++ {
value := make([]byte, 0, commonPrefixIdx+8)
value = append(value, lower[:commonPrefixIdx]...)
startV += step
binary.BigEndian.PutUint64(buf, startV)
value = append(value, buf...)
valuesList = append(valuesList, value)
}
return valuesList
}

// longestCommonPrefixLen gets the longest common prefix byte length.
func longestCommonPrefixLen(s1, s2 []byte) int {
l := mathutil.Min(len(s1), len(s2))
i := 0
for ; i < l; i++ {
if s1[i] != s2[i] {
break
}
}
return i
}

// getStepValue gets the step of between the lower and upper value. step = (upper-lower)/num.
// Convert byte slice to uint64 first.
func getStepValue(lower, upper []byte, num int) uint64 {
lowerUint := getUint64FromBytes(lower, 0)
upperUint := getUint64FromBytes(upper, 0xff)
return (upperUint - lowerUint) / uint64(num)
}

// getUint64FromBytes gets a uint64 from the `bs` byte slice.
// If len(bs) < 8, then padding with `pad`.
func getUint64FromBytes(bs []byte, pad byte) uint64 {
buf := bs
if len(buf) < 8 {
buf = make([]byte, 0, 8)
buf = append(buf, bs...)
for i := len(buf); i < 8; i++ {
buf = append(buf, pad)
}
}
return binary.BigEndian.Uint64(buf)
}

func datumSliceToString(ds []types.Datum) (string, error) {
str := "("
for i, d := range ds {
s, err := d.ToString()
if err != nil {
return str, err
}
if i > 0 {
str += ","
}
str += s
}
str += ")"
return str, nil
}
Loading

0 comments on commit 7bf3d69

Please sign in to comment.