Skip to content

Commit

Permalink
ddl: support scatter region in cluster/global level (#56157)
Browse files Browse the repository at this point in the history
ref tikv/pd#8424, ref #54886, close #55184
  • Loading branch information
River2000i committed Sep 25, 2024
1 parent 0d5e0e9 commit e5c9867
Show file tree
Hide file tree
Showing 20 changed files with 171 additions and 57 deletions.
2 changes: 1 addition & 1 deletion br/pkg/restore/snap_client/systable_restore_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -116,5 +116,5 @@ func TestCheckSysTableCompatibility(t *testing.T) {
//
// The above variables are in the file br/pkg/restore/systable_restore.go
func TestMonitorTheSystemTableIncremental(t *testing.T) {
require.Equal(t, int64(215), session.CurrentBootstrapVersion)
require.Equal(t, int64(216), session.CurrentBootstrapVersion)
}
18 changes: 9 additions & 9 deletions pkg/ddl/executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -1390,21 +1390,21 @@ func preSplitAndScatter(ctx sessionctx.Context, store kv.Storage, tbInfo *model.
return
}
var (
preSplit func()
scatterRegion bool
preSplit func()
scatterScope string
)
val, err := ctx.GetSessionVars().GetGlobalSystemVar(context.Background(), variable.TiDBScatterRegion)
if err != nil {
logutil.DDLLogger().Warn("won't scatter region", zap.Error(err))
val, ok := ctx.GetSessionVars().GetSystemVar(variable.TiDBScatterRegion)
if !ok {
logutil.DDLLogger().Warn("get system variable met problem, won't scatter region")
} else {
scatterRegion = variable.TiDBOptOn(val)
scatterScope = val
}
if len(parts) > 0 {
preSplit = func() { splitPartitionTableRegion(ctx, sp, tbInfo, parts, scatterRegion) }
preSplit = func() { splitPartitionTableRegion(ctx, sp, tbInfo, parts, scatterScope) }
} else {
preSplit = func() { splitTableRegion(ctx, sp, tbInfo, scatterRegion) }
preSplit = func() { splitTableRegion(ctx, sp, tbInfo, scatterScope) }
}
if scatterRegion {
if scatterScope != variable.ScatterOff {
preSplit()
} else {
go preSplit()
Expand Down
8 changes: 4 additions & 4 deletions pkg/ddl/index_modify_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -180,10 +180,10 @@ func testAddIndex(t *testing.T, tp testAddIndexType, createTableSQL, idxTp strin
isTestPartition := (testPartition & tp) > 0
if isTestShardRowID {
atomic.StoreUint32(&ddl.EnableSplitTableRegion, 1)
tk.MustExec("set global tidb_scatter_region = 1")
tk.MustExec("set global tidb_scatter_region = 'table'")
defer func() {
atomic.StoreUint32(&ddl.EnableSplitTableRegion, 0)
tk.MustExec("set global tidb_scatter_region = 0")
tk.MustExec("set global tidb_scatter_region = ''")
}()
}
if (testClusteredIndex & tp) > 0 {
Expand Down Expand Up @@ -473,10 +473,10 @@ func testAddIndexWithSplitTable(t *testing.T, createSQL, splitTableSQL string) {
hasAutoRandomField := len(splitTableSQL) > 0
if !hasAutoRandomField {
atomic.StoreUint32(&ddl.EnableSplitTableRegion, 1)
tk.MustExec("set global tidb_scatter_region = 1")
tk.MustExec("set global tidb_scatter_region = 'table'")
defer func() {
atomic.StoreUint32(&ddl.EnableSplitTableRegion, 0)
tk.MustExec("set global tidb_scatter_region = 0")
tk.MustExec("set global tidb_scatter_region = ''")
}()
}
tk.MustExec(createSQL)
Expand Down
5 changes: 3 additions & 2 deletions pkg/ddl/partition.go
Original file line number Diff line number Diff line change
Expand Up @@ -3209,8 +3209,9 @@ func (w *worker) onReorganizePartition(jobCtx *jobContext, t *meta.Meta, job *mo
// Doing the preSplitAndScatter here, since all checks are completed,
// and we will soon start writing to the new partitions.
if s, ok := jobCtx.store.(kv.SplittableStore); ok && s != nil {
// partInfo only contains the AddingPartitions
splitPartitionTableRegion(w.sess.Context, s, tblInfo, partInfo.Definitions, true)
// 1. partInfo only contains the AddingPartitions
// 2. ScatterTable control all new split region need waiting for scatter region finish at table level.
splitPartitionTableRegion(w.sess.Context, s, tblInfo, partInfo.Definitions, variable.ScatterTable)
}

// Assume we cannot have more than MaxUint64 rows, set the progress to 1/10 of that.
Expand Down
47 changes: 31 additions & 16 deletions pkg/ddl/split_region.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,48 +24,62 @@ import (
"github.com/pingcap/tidb/pkg/meta/model"
"github.com/pingcap/tidb/pkg/parser/mysql"
"github.com/pingcap/tidb/pkg/sessionctx"
"github.com/pingcap/tidb/pkg/sessionctx/variable"
"github.com/pingcap/tidb/pkg/tablecodec"
"github.com/pingcap/tidb/pkg/types"
tikverr "github.com/tikv/client-go/v2/error"
"go.uber.org/zap"
)

func splitPartitionTableRegion(ctx sessionctx.Context, store kv.SplittableStore, tbInfo *model.TableInfo, parts []model.PartitionDefinition, scatter bool) {
func splitPartitionTableRegion(ctx sessionctx.Context, store kv.SplittableStore, tbInfo *model.TableInfo, parts []model.PartitionDefinition, scatterScope string) {
// Max partition count is 8192, should we sample and just choose some partitions to split?
regionIDs := make([]uint64, 0, len(parts))
ctxWithTimeout, cancel := context.WithTimeout(context.Background(), ctx.GetSessionVars().GetSplitRegionTimeout())
defer cancel()
ctxWithTimeout = kv.WithInternalSourceType(ctxWithTimeout, kv.InternalTxnDDL)
if shardingBits(tbInfo) > 0 && tbInfo.PreSplitRegions > 0 {
for _, def := range parts {
regionIDs = append(regionIDs, preSplitPhysicalTableByShardRowID(ctxWithTimeout, store, tbInfo, def.ID, scatter)...)
regionIDs = append(regionIDs, preSplitPhysicalTableByShardRowID(ctxWithTimeout, store, tbInfo, def.ID, scatterScope)...)
}
} else {
for _, def := range parts {
regionIDs = append(regionIDs, SplitRecordRegion(ctxWithTimeout, store, def.ID, tbInfo.ID, scatter))
regionIDs = append(regionIDs, SplitRecordRegion(ctxWithTimeout, store, def.ID, tbInfo.ID, scatterScope))
}
}
if scatter {
if scatterScope != variable.ScatterOff {
WaitScatterRegionFinish(ctxWithTimeout, store, regionIDs...)
}
}

func splitTableRegion(ctx sessionctx.Context, store kv.SplittableStore, tbInfo *model.TableInfo, scatter bool) {
func splitTableRegion(ctx sessionctx.Context, store kv.SplittableStore, tbInfo *model.TableInfo, scatterScope string) {
ctxWithTimeout, cancel := context.WithTimeout(context.Background(), ctx.GetSessionVars().GetSplitRegionTimeout())
defer cancel()
ctxWithTimeout = kv.WithInternalSourceType(ctxWithTimeout, kv.InternalTxnDDL)
var regionIDs []uint64
if shardingBits(tbInfo) > 0 && tbInfo.PreSplitRegions > 0 {
regionIDs = preSplitPhysicalTableByShardRowID(ctxWithTimeout, store, tbInfo, tbInfo.ID, scatter)
regionIDs = preSplitPhysicalTableByShardRowID(ctxWithTimeout, store, tbInfo, tbInfo.ID, scatterScope)
} else {
regionIDs = append(regionIDs, SplitRecordRegion(ctxWithTimeout, store, tbInfo.ID, tbInfo.ID, scatter))
regionIDs = append(regionIDs, SplitRecordRegion(ctxWithTimeout, store, tbInfo.ID, tbInfo.ID, scatterScope))
}
if scatter {
if scatterScope != variable.ScatterOff {
WaitScatterRegionFinish(ctxWithTimeout, store, regionIDs...)
}
}

func preSplitPhysicalTableByShardRowID(ctx context.Context, store kv.SplittableStore, tbInfo *model.TableInfo, physicalID int64, scatter bool) []uint64 {
// `tID` is used to control the scope of scatter. If it is `ScatterTable`, the corresponding tableID is used.
// If it is `ScatterGlobal`, the scatter configured at global level uniformly use -1 as `tID`.
func getScatterConfig(scope string, tableID int64) (scatter bool, tID int64) {
switch scope {
case variable.ScatterTable:
return true, tableID
case variable.ScatterGlobal:
return true, -1
default:
return false, tableID
}
}

func preSplitPhysicalTableByShardRowID(ctx context.Context, store kv.SplittableStore, tbInfo *model.TableInfo, physicalID int64, scatterScope string) []uint64 {
// Example:
// sharding_bits = 4
// PreSplitRegions = 2
Expand Down Expand Up @@ -107,20 +121,21 @@ func preSplitPhysicalTableByShardRowID(ctx context.Context, store kv.SplittableS
key := tablecodec.EncodeRecordKey(recordPrefix, kv.IntHandle(recordID))
splitTableKeys = append(splitTableKeys, key)
}
var err error
regionIDs, err := store.SplitRegions(ctx, splitTableKeys, scatter, &tbInfo.ID)
scatter, tableID := getScatterConfig(scatterScope, tbInfo.ID)
regionIDs, err := store.SplitRegions(ctx, splitTableKeys, scatter, &tableID)
if err != nil {
logutil.DDLLogger().Warn("pre split some table regions failed",
zap.Stringer("table", tbInfo.Name), zap.Int("successful region count", len(regionIDs)), zap.Error(err))
}
regionIDs = append(regionIDs, splitIndexRegion(store, tbInfo, scatter)...)
regionIDs = append(regionIDs, splitIndexRegion(store, tbInfo, scatter, &tableID)...)
return regionIDs
}

// SplitRecordRegion is to split region in store by table prefix.
func SplitRecordRegion(ctx context.Context, store kv.SplittableStore, physicalTableID, tableID int64, scatter bool) uint64 {
func SplitRecordRegion(ctx context.Context, store kv.SplittableStore, physicalTableID, tableID int64, scatterScope string) uint64 {
tableStartKey := tablecodec.GenTablePrefix(physicalTableID)
regionIDs, err := store.SplitRegions(ctx, [][]byte{tableStartKey}, scatter, &tableID)
scatter, tID := getScatterConfig(scatterScope, tableID)
regionIDs, err := store.SplitRegions(ctx, [][]byte{tableStartKey}, scatter, &tID)
if err != nil {
// It will be automatically split by TiKV later.
logutil.DDLLogger().Warn("split table region failed", zap.Error(err))
Expand All @@ -131,13 +146,13 @@ func SplitRecordRegion(ctx context.Context, store kv.SplittableStore, physicalTa
return 0
}

func splitIndexRegion(store kv.SplittableStore, tblInfo *model.TableInfo, scatter bool) []uint64 {
func splitIndexRegion(store kv.SplittableStore, tblInfo *model.TableInfo, scatter bool, tableID *int64) []uint64 {
splitKeys := make([][]byte, 0, len(tblInfo.Indices))
for _, idx := range tblInfo.Indices {
indexPrefix := tablecodec.EncodeTableIndexPrefix(tblInfo.ID, idx.ID)
splitKeys = append(splitKeys, indexPrefix)
}
regionIDs, err := store.SplitRegions(context.Background(), splitKeys, scatter, &tblInfo.ID)
regionIDs, err := store.SplitRegions(context.Background(), splitKeys, scatter, tableID)
if err != nil {
logutil.DDLLogger().Warn("pre split some table index regions failed",
zap.Stringer("table", tblInfo.Name), zap.Int("successful region count", len(regionIDs)), zap.Error(err))
Expand Down
55 changes: 54 additions & 1 deletion pkg/ddl/table_split_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -45,11 +45,19 @@ func TestTableSplit(t *testing.T) {
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
// Synced split table region.
tk.MustExec("set global tidb_scatter_region = 1")
tk.MustExec("set @@session.tidb_scatter_region = 'table'")
tk.MustExec(`create table t_part (a int key) partition by range(a) (
partition p0 values less than (10),
partition p1 values less than (20)
)`)
tk.MustQuery("select @@global.tidb_scatter_region;").Check(testkit.Rows(""))
tk.MustExec("set @@global.tidb_scatter_region = 'table'")
tk = testkit.NewTestKit(t, store)
tk.MustExec("use test")
tk.MustExec(`create table t_part_2 (a int key) partition by range(a) (
partition p0 values less than (10),
partition p1 values less than (20)
)`)
defer dom.Close()
atomic.StoreUint32(&ddl.EnableSplitTableRegion, 0)
infoSchema := dom.InfoSchema()
Expand All @@ -65,6 +73,51 @@ func TestTableSplit(t *testing.T) {
for _, def := range pi.Definitions {
checkRegionStartWithTableID(t, def.ID, store.(kvStore))
}
tbl, err = infoSchema.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t_part_2"))
require.NoError(t, err)
pi = tbl.Meta().GetPartitionInfo()
require.NotNil(t, pi)
for _, def := range pi.Definitions {
checkRegionStartWithTableID(t, def.ID, store.(kvStore))
}
}

// TestScatterRegion test the behavior of the tidb_scatter_region system variable, for verifying:
// 1. The variable can be set and queried correctly at both session and global levels.
// 2. Changes to the global variable affect new sessions but not existing ones.
// 3. The variable only accepts valid values (”, 'table', 'global').
// 4. Attempts to set invalid values result in appropriate error messages.
func TestScatterRegion(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
tk2 := testkit.NewTestKit(t, store)

tk.MustQuery("select @@tidb_scatter_region;").Check(testkit.Rows(""))
tk.MustExec("set @@tidb_scatter_region = 'table';")
tk.MustQuery("select @@tidb_scatter_region;").Check(testkit.Rows("table"))
tk.MustExec("set @@tidb_scatter_region = 'global';")
tk.MustQuery("select @@tidb_scatter_region;").Check(testkit.Rows("global"))
tk.MustExec("set @@tidb_scatter_region = '';")
tk.MustQuery("select @@tidb_scatter_region;").Check(testkit.Rows(""))

tk.MustExec("set global tidb_scatter_region = 'table';")
tk.MustQuery("select @@global.tidb_scatter_region;").Check(testkit.Rows("table"))
tk.MustQuery("select @@tidb_scatter_region;").Check(testkit.Rows(""))
tk2.MustQuery("select @@tidb_scatter_region;").Check(testkit.Rows(""))
tk2 = testkit.NewTestKit(t, store)
tk2.MustQuery("select @@tidb_scatter_region;").Check(testkit.Rows("table"))

tk.MustExec("set global tidb_scatter_region = 'global';")
tk.MustQuery("select @@global.tidb_scatter_region;").Check(testkit.Rows("global"))
tk.MustExec("set global tidb_scatter_region = '';")
tk.MustQuery("select @@global.tidb_scatter_region;").Check(testkit.Rows(""))

err := tk.ExecToErr("set @@tidb_scatter_region = 'test';")
require.ErrorContains(t, err, "invalid value for 'test', it should be either '', 'table' or 'global'")
err = tk.ExecToErr("set @@tidb_scatter_region = '1';")
require.ErrorContains(t, err, "invalid value for '1', it should be either '', 'table' or 'global'")
err = tk.ExecToErr("set @@tidb_scatter_region = 0;")
require.ErrorContains(t, err, "invalid value for '0', it should be either '', 'table' or 'global'")
}

type kvStore interface {
Expand Down
4 changes: 2 additions & 2 deletions pkg/ddl/tests/partition/db_partition_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1210,7 +1210,7 @@ func TestAlterTableTruncatePartitionPreSplitRegion(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
atomic.StoreUint32(&ddl.EnableSplitTableRegion, 1)
tk.MustExec("set @@global.tidb_scatter_region=1;")
tk.MustExec("set @@session.tidb_scatter_region='table';")
tk.MustExec("use test;")

tk.MustExec("drop table if exists t1;")
Expand Down Expand Up @@ -1653,7 +1653,7 @@ func TestGlobalIndexShowTableRegions(t *testing.T) {
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
tk.MustExec("drop table if exists p")
tk.MustExec("set @@global.tidb_scatter_region = on")
tk.MustExec("set @@session.tidb_scatter_region = 'table'")
tk.MustExec(`create table p (id int, c int, d int, unique key uidx(c)) partition by range (c) (
partition p0 values less than (4),
partition p1 values less than (7),
Expand Down
4 changes: 2 additions & 2 deletions pkg/ddl/tests/serial/serial_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -156,7 +156,7 @@ func TestCreateTableWithLike(t *testing.T) {
// Test create table like for partition table.
atomic.StoreUint32(&ddl.EnableSplitTableRegion, 1)
tk.MustExec("use test")
tk.MustExec("set @@global.tidb_scatter_region=1")
tk.MustExec("set @@session.tidb_scatter_region='table'")
tk.MustExec("drop table if exists partition_t")
tk.MustExec("create table partition_t (a int, b int,index(a)) partition by hash (a) partitions 3")
tk.MustExec("drop table if exists t1")
Expand Down Expand Up @@ -1108,7 +1108,7 @@ func TestAutoRandomWithPreSplitRegion(t *testing.T) {
origin := atomic.LoadUint32(&ddl.EnableSplitTableRegion)
atomic.StoreUint32(&ddl.EnableSplitTableRegion, 1)
defer atomic.StoreUint32(&ddl.EnableSplitTableRegion, origin)
tk.MustExec("set @@global.tidb_scatter_region=1")
tk.MustExec("set @@session.tidb_scatter_region='table'")

// Test pre-split table region for auto_random table.
tk.MustExec("create table t (a bigint auto_random(2) primary key clustered, b int) pre_split_regions=2")
Expand Down
2 changes: 1 addition & 1 deletion pkg/executor/executor_failpoint_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -210,7 +210,7 @@ func TestSplitRegionTimeout(t *testing.T) {

// Test pre-split with timeout.
tk.MustExec("drop table if exists t")
tk.MustExec("set @@global.tidb_scatter_region=1;")
tk.MustExec("set @@session.tidb_scatter_region='table';")
require.NoError(t, failpoint.Enable("tikvclient/mockScatterRegionTimeout", `return(true)`))
atomic.StoreUint32(&ddl.EnableSplitTableRegion, 1)
start := time.Now()
Expand Down
2 changes: 1 addition & 1 deletion pkg/executor/sample_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -32,7 +32,7 @@ func createSampleTestkit(t *testing.T, store kv.Storage) *testkit.TestKit {
tk.MustExec("drop database if exists test_table_sample;")
tk.MustExec("create database test_table_sample;")
tk.MustExec("use test_table_sample;")
tk.MustExec("set @@global.tidb_scatter_region=1;")
tk.MustExec("set @@session.tidb_scatter_region='table';")
return tk
}

Expand Down
19 changes: 12 additions & 7 deletions pkg/executor/set_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -368,13 +368,18 @@ func TestSetVar(t *testing.T) {
tk.MustQuery(`select @@session.tidb_wait_split_region_finish;`).Check(testkit.Rows("0"))

// test for tidb_scatter_region
tk.MustQuery(`select @@global.tidb_scatter_region;`).Check(testkit.Rows("0"))
tk.MustExec("set global tidb_scatter_region = 1")
tk.MustQuery(`select @@global.tidb_scatter_region;`).Check(testkit.Rows("1"))
tk.MustExec("set global tidb_scatter_region = 0")
tk.MustQuery(`select @@global.tidb_scatter_region;`).Check(testkit.Rows("0"))
require.Error(t, tk.ExecToErr("set session tidb_scatter_region = 0"))
require.Error(t, tk.ExecToErr(`select @@session.tidb_scatter_region;`))
tk.MustQuery(`select @@global.tidb_scatter_region;`).Check(testkit.Rows(""))
tk.MustExec("set global tidb_scatter_region = 'table'")
tk.MustQuery(`select @@global.tidb_scatter_region;`).Check(testkit.Rows("table"))
tk.MustExec("set global tidb_scatter_region = 'global'")
tk.MustQuery(`select @@global.tidb_scatter_region;`).Check(testkit.Rows("global"))
tk.MustExec("set session tidb_scatter_region = ''")
tk.MustQuery(`select @@session.tidb_scatter_region;`).Check(testkit.Rows(""))
tk.MustExec("set session tidb_scatter_region = 'table'")
tk.MustQuery(`select @@session.tidb_scatter_region;`).Check(testkit.Rows("table"))
tk.MustExec("set session tidb_scatter_region = 'global'")
tk.MustQuery(`select @@session.tidb_scatter_region;`).Check(testkit.Rows("global"))
require.Error(t, tk.ExecToErr("set session tidb_scatter_region = 'test'"))

// test for tidb_wait_split_region_timeout
tk.MustQuery(`select @@session.tidb_wait_split_region_timeout;`).Check(testkit.Rows(strconv.Itoa(variable.DefWaitSplitRegionTimeout)))
Expand Down
2 changes: 1 addition & 1 deletion pkg/executor/test/showtest/show_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -804,7 +804,7 @@ func TestAutoRandomWithLargeSignedShowTableRegions(t *testing.T) {
tk.MustExec("drop table if exists t;")

tk.MustExec("create table t (a bigint unsigned auto_random primary key clustered);")
tk.MustExec("set @@global.tidb_scatter_region=1;")
tk.MustExec("set @@session.tidb_scatter_region='table';")
// 18446744073709541615 is MaxUint64 - 10000.
// 18446744073709551615 is the MaxUint64.
tk.MustQuery("split table t between (18446744073709541615) and (18446744073709551615) regions 2;").
Expand Down
6 changes: 3 additions & 3 deletions pkg/executor/test/splittest/split_table_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -41,7 +41,7 @@ func TestClusterIndexShowTableRegion(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
atomic.StoreUint32(&ddl.EnableSplitTableRegion, 1)
tk.MustExec("set global tidb_scatter_region = 1")
tk.MustExec("set global tidb_scatter_region = 'table'")
tk.MustExec("drop database if exists cluster_index_regions;")
tk.MustExec("create database cluster_index_regions;")
tk.MustExec("use cluster_index_regions;")
Expand Down Expand Up @@ -75,7 +75,7 @@ func TestShowTableRegion(t *testing.T) {
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
tk.MustExec("drop table if exists t_regions")
tk.MustExec("set global tidb_scatter_region = 1")
tk.MustExec("set global tidb_scatter_region = 'table'")
atomic.StoreUint32(&ddl.EnableSplitTableRegion, 1)
tk.MustExec("create table t_regions (a int key, b int, c int, index idx(b), index idx2(c))")
tk.MustGetErrMsg(
Expand Down Expand Up @@ -223,7 +223,7 @@ func TestShowTableRegion(t *testing.T) {

// Test show table regions for partition table when enable split region when create table.
atomic.StoreUint32(&ddl.EnableSplitTableRegion, 1)
tk.MustExec("set @@global.tidb_scatter_region=1;")
tk.MustExec("set @@session.tidb_scatter_region='table';")
tk.MustExec("drop table if exists partition_t;")
tk.MustExec("create table partition_t (a int, b int,index(a)) partition by hash (a) partitions 3")
re = tk.MustQuery("show table partition_t regions")
Expand Down
Loading

0 comments on commit e5c9867

Please sign in to comment.