Skip to content

Commit

Permalink
*: add split index region syntax (pingcap#10203)
Browse files Browse the repository at this point in the history
  • Loading branch information
crazycs520 committed Jun 19, 2019
1 parent 73ef6f5 commit 19f29ef
Show file tree
Hide file tree
Showing 8 changed files with 171 additions and 0 deletions.
13 changes: 13 additions & 0 deletions executor/builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -168,6 +168,8 @@ func (b *executorBuilder) build(p plannercore.Plan) Executor {
return b.buildIndexReader(v)
case *plannercore.PhysicalIndexLookUpReader:
return b.buildIndexLookUpReader(v)
case *plannercore.SplitIndexRegion:
return b.buildSplitIndexRegion(v)
default:
b.err = ErrUnknownPlan.GenWithStack("Unknown Plan %T", p)
return nil
Expand Down Expand Up @@ -1322,6 +1324,17 @@ func (b *executorBuilder) buildUnionAll(v *plannercore.PhysicalUnionAll) Executo
return e
}

func (b *executorBuilder) buildSplitIndexRegion(v *plannercore.SplitIndexRegion) Executor {
base := newBaseExecutor(b.ctx, nil, v.ExplainID())
base.initCap = chunk.ZeroCapacity
return &SplitIndexRegionExec{
baseExecutor: base,
table: v.Table,
indexInfo: v.IndexInfo,
valueLists: v.ValueLists,
}
}

func (b *executorBuilder) buildUpdate(v *plannercore.Update) Executor {
tblID2table := make(map[int64]table.Table)
for id := range v.SelectPlan.Schema().TblID2Handle {
Expand Down
4 changes: 4 additions & 0 deletions executor/executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -1333,6 +1333,10 @@ func ResetContextOfStmt(ctx sessionctx.Context, s ast.StmtNode) (err error) {
sc.InShowWarning = true
sc.SetWarnings(vars.StmtCtx.GetWarnings())
}
case *ast.SplitIndexRegionStmt:
sc.IgnoreTruncate = false
sc.IgnoreZeroInDate = true
sc.AllowInvalidDate = vars.SQLMode.HasAllowInvalidDatesMode()
default:
sc.IgnoreTruncate = true
sc.IgnoreZeroInDate = true
Expand Down
12 changes: 12 additions & 0 deletions executor/executor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -3518,6 +3518,18 @@ func (s *testSuite) TestUnsignedFeedback(c *C) {
c.Assert(result.Rows()[2][3], Equals, "table:t, range:[0,+inf], keep order:false")
}

func (s *testSuite) TestSplitIndexRegion(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
tk.MustExec("drop table if exists t")
tk.MustExec("create table t(a varchar(100),b int, index idx1(b,a))")
tk.MustExec(`split table t index idx1 by (10000,"abcd"),(10000000);`)
_, err := tk.Exec(`split table t index idx1 by ("abcd");`)
c.Assert(err, NotNil)
terr := errors.Cause(err).(*terror.Error)
c.Assert(terr.Code(), Equals, terror.ErrCode(mysql.WarnDataTruncated))
}

type testOOMSuite struct {
store kv.Storage
do *domain.Domain
Expand Down
84 changes: 84 additions & 0 deletions executor/split.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,84 @@
// Copyright 2019 PingCAP, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// See the License for the specific language governing permissions and
// limitations under the License.

package executor

import (
"context"
"math"

"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/types"
"github.com/pingcap/tidb/util/chunk"
"github.com/pingcap/tidb/util/logutil"
"go.uber.org/zap"
)

// SplitIndexRegionExec represents a split index regions executor.
type SplitIndexRegionExec struct {
baseExecutor

table table.Table
indexInfo *model.IndexInfo
valueLists [][]types.Datum
}

type splitableStore interface {
SplitRegionAndScatter(splitKey kv.Key) (uint64, error)
WaitScatterRegionFinish(regionID uint64) error
}

// Next implements the Executor Next interface.
func (e *SplitIndexRegionExec) Next(ctx context.Context, _ *chunk.Chunk) error {
store := e.ctx.GetStore()
s, ok := store.(splitableStore)
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
}

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("index", e.indexInfo.Name.L),
zap.Error(err))
continue
}
regionIDs = append(regionIDs, regionID)

}
if !e.ctx.GetSessionVars().WaitTableSplitFinish {
return nil
}
for _, regionID := range regionIDs {
err := s.WaitScatterRegionFinish(regionID)
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("index", e.indexInfo.Name.L),
zap.Error(err))
}
}
return nil
}
2 changes: 2 additions & 0 deletions go.mod
Original file line number Diff line number Diff line change
Expand Up @@ -79,3 +79,5 @@ require (
gopkg.in/natefinch/lumberjack.v2 v2.0.0
gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7 // indirect
)

replace github.com/pingcap/parser => github.com/crazycs520/parser v0.0.0-20190619121853-a6bb78486e7f
4 changes: 4 additions & 0 deletions go.sum
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,10 @@ github.com/coreos/etcd v3.2.18+incompatible/go.mod h1:uF7uidLiAD3TWHmW31ZFd/JWoc
github.com/coreos/go-semver v0.2.0/go.mod h1:nnelYz7RCh+5ahJtPPxZlU+153eP4D4r3EedlOD2RNk=
github.com/coreos/go-systemd v0.0.0-20180202092358-40e2722dffea/go.mod h1:F5haX7vjVVG0kc13fIWeqUViNPyEJxv/OmvnBo0Yme4=
github.com/coreos/pkg v0.0.0-20160727233714-3ac0863d7acf/go.mod h1:E3G3o1h8I7cfcXa63jLwjI0eiQQMgzzUDFVpN/nH/eA=
github.com/crazycs520/parser v0.0.0-20190619121024-40e7ecaa504a h1:mij7LmNM+GCtIVFM/AZNwv4M2xCgm5w3WVVCLhMz3d8=
github.com/crazycs520/parser v0.0.0-20190619121024-40e7ecaa504a/go.mod h1:xLjI+gnWYexq011WPMEvCNS8rFM9qe1vdojIEzSKPuc=
github.com/crazycs520/parser v0.0.0-20190619121853-a6bb78486e7f h1:9oZ1G26gYo3ZvZBCeduAdHOIj1y1fm1dynqvEm+Iy10=
github.com/crazycs520/parser v0.0.0-20190619121853-a6bb78486e7f/go.mod h1:xLjI+gnWYexq011WPMEvCNS8rFM9qe1vdojIEzSKPuc=
github.com/cznic/mathutil v0.0.0-20160613104831-78ad7f262603 h1:hhR9hTi0ligs11JjfGDBP332clNOJRdW0Ci5oHtEC+0=
github.com/cznic/mathutil v0.0.0-20160613104831-78ad7f262603/go.mod h1:e6NPNENfs9mPDVNRekM7lKScauxd5kXTr1Mfyig6TDM=
github.com/cznic/sortutil v0.0.0-20150617083342-4c7342852e65 h1:hxuZop6tSoOi0sxFzoGGYdRqNrPubyaIf9KoBG9tPiE=
Expand Down
10 changes: 10 additions & 0 deletions planner/core/common_plans.go
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@ import (
"github.com/pingcap/tidb/metrics"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/table"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/types/parser_driver"
"github.com/pingcap/tidb/util/chunk"
"github.com/pingcap/tidb/util/kvcache"
Expand Down Expand Up @@ -403,6 +404,15 @@ type LoadStats struct {
Path string
}

// SplitIndexRegion represents a split index regions plan.
type SplitIndexRegion struct {
baseSchemaProducer

Table table.Table
IndexInfo *model.IndexInfo
ValueLists [][]types.Datum
}

// DDL represents a DDL statement plan.
type DDL struct {
baseSchemaProducer
Expand Down
42 changes: 42 additions & 0 deletions planner/core/planbuilder.go
Original file line number Diff line number Diff line change
Expand Up @@ -224,6 +224,8 @@ func (b *planBuilder) build(node ast.Node) (Plan, error) {
return b.buildSimple(node.(ast.StmtNode)), nil
case ast.DDLNode:
return b.buildDDL(x)
case *ast.SplitIndexRegionStmt:
return b.buildSplitIndexRegion(x)
}
return nil, ErrUnsupportedType.GenWithStack("Unsupported type %T", node)
}
Expand Down Expand Up @@ -1426,6 +1428,46 @@ func (b *planBuilder) buildLoadStats(ld *ast.LoadStatsStmt) Plan {
return p
}

func (b *planBuilder) buildSplitIndexRegion(node *ast.SplitIndexRegionStmt) (Plan, error) {
tblInfo := node.Table.TableInfo
indexInfo := tblInfo.FindIndexByName(strings.ToLower(node.IndexName))
if indexInfo == nil {
return nil, ErrKeyDoesNotExist.GenWithStackByArgs(node.IndexName, tblInfo.Name)
}

indexValues := make([][]types.Datum, 0, len(node.ValueLists))
for i, valuesItem := range node.ValueLists {
if len(valuesItem) > len(indexInfo.Columns) {
return nil, ErrWrongValueCountOnRow.GenWithStackByArgs(i + 1)
}
valueList := make([]types.Datum, 0, len(valuesItem))
for j, valueItem := range valuesItem {
x, ok := valueItem.(*driver.ValueExpr)
if !ok {
return nil, errors.New("expect constant values")
}
colOffset := indexInfo.Columns[j].Offset
value, err := x.Datum.ConvertTo(b.ctx.GetSessionVars().StmtCtx, &tblInfo.Columns[colOffset].FieldType)
if err != nil {
return nil, err
}

valueList = append(valueList, value)
}
indexValues = append(indexValues, valueList)
}
tableInPlan, ok := b.is.TableByID(tblInfo.ID)
if !ok {
return nil, errors.Errorf("Can't get table %s.", tblInfo.Name.O)
}
return &SplitIndexRegion{
Table: tableInPlan,
IndexInfo: indexInfo,
ValueLists: indexValues,
}, nil

}

func (b *planBuilder) buildDDL(node ast.DDLNode) (Plan, error) {
switch v := node.(type) {
case *ast.AlterDatabaseStmt:
Expand Down

0 comments on commit 19f29ef

Please sign in to comment.