From 19f29ef26b8411d987fea392c74cd98c58bc4809 Mon Sep 17 00:00:00 2001 From: crazycs520 Date: Wed, 19 Jun 2019 20:21:04 +0800 Subject: [PATCH] *: add split index region syntax (#10203) --- executor/builder.go | 13 ++++++ executor/executor.go | 4 ++ executor/executor_test.go | 12 ++++++ executor/split.go | 84 ++++++++++++++++++++++++++++++++++++ go.mod | 2 + go.sum | 4 ++ planner/core/common_plans.go | 10 +++++ planner/core/planbuilder.go | 42 ++++++++++++++++++ 8 files changed, 171 insertions(+) create mode 100644 executor/split.go diff --git a/executor/builder.go b/executor/builder.go index c66cc5182e665..2cce6ffb5237c 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -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 @@ -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 { diff --git a/executor/executor.go b/executor/executor.go index f64266c2553f7..c3e2e96e1d39f 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -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 diff --git a/executor/executor_test.go b/executor/executor_test.go index 134e4603a3dd3..6e0a8a24a75f0 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -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 diff --git a/executor/split.go b/executor/split.go new file mode 100644 index 0000000000000..dffdafe5d7336 --- /dev/null +++ b/executor/split.go @@ -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 +} diff --git a/go.mod b/go.mod index a38b9a410d0a5..d01bd5b7b014b 100644 --- a/go.mod +++ b/go.mod @@ -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 diff --git a/go.sum b/go.sum index 816de6e1462ac..23b071f7df154 100644 --- a/go.sum +++ b/go.sum @@ -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= diff --git a/planner/core/common_plans.go b/planner/core/common_plans.go index 69b0f1242de3f..4cdd1775639d7 100644 --- a/planner/core/common_plans.go +++ b/planner/core/common_plans.go @@ -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" @@ -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 diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index 4a60f30dc71e9..c88b77469bddb 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -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) } @@ -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: