Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

ddl: fix issue pre_split_regions table option doesn't work in the partition table. #18144

Merged
merged 4 commits into from
Jun 29, 2020
Merged
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
2 changes: 1 addition & 1 deletion ddl/ddl_api.go
Original file line number Diff line number Diff line change
Expand Up @@ -1703,7 +1703,7 @@ func (d *ddl) preSplitAndScatter(ctx sessionctx.Context, tbInfo *model.TableInfo
scatterRegion = variable.TiDBOptOn(val)
}
if pi != nil {
preSplit = func() { splitPartitionTableRegion(ctx, sp, pi, scatterRegion) }
preSplit = func() { splitPartitionTableRegion(ctx, sp, tbInfo, pi, scatterRegion) }
} else {
preSplit = func() { splitTableRegion(ctx, sp, tbInfo, scatterRegion) }
}
Expand Down
32 changes: 19 additions & 13 deletions ddl/split_region.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,13 +24,19 @@ import (
"go.uber.org/zap"
)

func splitPartitionTableRegion(ctx sessionctx.Context, store kv.SplittableStore, pi *model.PartitionInfo, scatter bool) {
func splitPartitionTableRegion(ctx sessionctx.Context, store kv.SplittableStore, tbInfo *model.TableInfo, pi *model.PartitionInfo, scatter bool) {
// Max partition count is 4096, should we sample and just choose some of the partition to split?
regionIDs := make([]uint64, 0, len(pi.Definitions))
ctxWithTimeout, cancel := context.WithTimeout(context.Background(), ctx.GetSessionVars().GetSplitRegionTimeout())
defer cancel()
for _, def := range pi.Definitions {
regionIDs = append(regionIDs, splitRecordRegion(ctxWithTimeout, store, def.ID, scatter))
if tbInfo.ShardRowIDBits > 0 && tbInfo.PreSplitRegions > 0 {
for _, def := range pi.Definitions {
regionIDs = append(regionIDs, preSplitPhysicalTableByShardRowID(ctxWithTimeout, store, tbInfo, def.ID, scatter)...)
}
} else {
for _, def := range pi.Definitions {
regionIDs = append(regionIDs, splitRecordRegion(ctxWithTimeout, store, def.ID, scatter))
}
}
if scatter {
waitScatterRegionFinish(ctxWithTimeout, store, regionIDs...)
Expand All @@ -40,17 +46,18 @@ func splitPartitionTableRegion(ctx sessionctx.Context, store kv.SplittableStore,
func splitTableRegion(ctx sessionctx.Context, store kv.SplittableStore, tbInfo *model.TableInfo, scatter bool) {
ctxWithTimeout, cancel := context.WithTimeout(context.Background(), ctx.GetSessionVars().GetSplitRegionTimeout())
defer cancel()
var regionIDs []uint64
if tbInfo.ShardRowIDBits > 0 && tbInfo.PreSplitRegions > 0 {
splitPreSplitedTable(ctxWithTimeout, store, tbInfo, scatter)
regionIDs = preSplitPhysicalTableByShardRowID(ctxWithTimeout, store, tbInfo, tbInfo.ID, scatter)
} else {
regionID := splitRecordRegion(ctxWithTimeout, store, tbInfo.ID, scatter)
if scatter {
waitScatterRegionFinish(ctxWithTimeout, store, regionID)
}
regionIDs = append(regionIDs, splitRecordRegion(ctxWithTimeout, store, tbInfo.ID, scatter))
}
if scatter {
waitScatterRegionFinish(ctxWithTimeout, store, regionIDs...)
}
}

func splitPreSplitedTable(ctx context.Context, store kv.SplittableStore, tbInfo *model.TableInfo, scatter bool) {
func preSplitPhysicalTableByShardRowID(ctx context.Context, store kv.SplittableStore, tbInfo *model.TableInfo, physicalID int64, scatter bool) []uint64 {
// Example:
// ShardRowIDBits = 4
// PreSplitRegions = 2
Expand Down Expand Up @@ -78,9 +85,10 @@ func splitPreSplitedTable(ctx context.Context, store kv.SplittableStore, tbInfo
step := int64(1 << (tbInfo.ShardRowIDBits - tbInfo.PreSplitRegions))
max := int64(1 << tbInfo.ShardRowIDBits)
splitTableKeys := make([][]byte, 0, 1<<(tbInfo.PreSplitRegions))
splitTableKeys = append(splitTableKeys, tablecodec.GenTablePrefix(physicalID))
for p := step; p < max; p += step {
recordID := p << (64 - tbInfo.ShardRowIDBits - 1)
recordPrefix := tablecodec.GenTableRecordPrefix(tbInfo.ID)
recordPrefix := tablecodec.GenTableRecordPrefix(physicalID)
key := tablecodec.EncodeRecordKey(recordPrefix, kv.IntHandle(recordID))
splitTableKeys = append(splitTableKeys, key)
}
Expand All @@ -91,9 +99,7 @@ func splitPreSplitedTable(ctx context.Context, store kv.SplittableStore, tbInfo
zap.Stringer("table", tbInfo.Name), zap.Int("successful region count", len(regionIDs)), zap.Error(err))
}
regionIDs = append(regionIDs, splitIndexRegion(store, tbInfo, scatter)...)
if scatter {
waitScatterRegionFinish(ctx, store, regionIDs...)
}
return regionIDs
}

func splitRecordRegion(ctx context.Context, store kv.SplittableStore, tableID int64, scatter bool) uint64 {
Expand Down
17 changes: 17 additions & 0 deletions executor/executor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -4410,6 +4410,23 @@ func (s *testSplitTable) TestShowTableRegion(c *C) {
c.Assert(rows[1][1], Equals, fmt.Sprintf("t_%d_r_2305843009213693952", tbl.Meta().ID))
c.Assert(rows[2][1], Equals, fmt.Sprintf("t_%d_r_4611686018427387904", tbl.Meta().ID))
c.Assert(rows[3][1], Equals, fmt.Sprintf("t_%d_r_6917529027641081856", tbl.Meta().ID))

// Test pre-split table region when create table.
tk.MustExec("drop table if exists pt_pre")
tk.MustExec("create table pt_pre (a int, b int) shard_row_id_bits = 2 pre_split_regions=2 partition by hash(a) partitions 3;")
re = tk.MustQuery("show table pt_pre regions")
rows = re.Rows()
// Table t_regions should have 4 regions now.
c.Assert(len(rows), Equals, 12)
tbl = testGetTableByName(c, tk.Se, "test", "pt_pre")
pi := tbl.Meta().GetPartitionInfo().Definitions
c.Assert(len(pi), Equals, 3)
for i, p := range pi {
c.Assert(rows[1+4*i][1], Equals, fmt.Sprintf("t_%d_r_2305843009213693952", p.ID))
c.Assert(rows[2+4*i][1], Equals, fmt.Sprintf("t_%d_r_4611686018427387904", p.ID))
c.Assert(rows[3+4*i][1], Equals, fmt.Sprintf("t_%d_r_6917529027641081856", p.ID))
}

defer atomic.StoreUint32(&ddl.EnableSplitTableRegion, 0)

// Test split partition table.
Expand Down