Skip to content
Merged
Show file tree
Hide file tree
Changes from 32 commits
Commits
Show all changes
33 commits
Select commit Hold shift + click to select a range
58e4627
support session scope and add comment
River2000i Sep 19, 2024
9312f45
support scatter region in `global` level
River2000i Sep 19, 2024
601e21a
add test
River2000i Sep 19, 2024
5986bac
Merge branch 'master' into fix-55184
River2000i Sep 19, 2024
082d6a8
fix test
River2000i Sep 19, 2024
38fe283
compatible upgrade
River2000i Sep 19, 2024
3731bb6
refactor scatterScope to const
River2000i Sep 20, 2024
8256974
fix test
River2000i Sep 20, 2024
65f01a8
fix test
River2000i Sep 20, 2024
886e679
fix test
River2000i Sep 20, 2024
673936d
fix test
River2000i Sep 20, 2024
5979ef6
fix test
River2000i Sep 20, 2024
c521838
fix warn log
River2000i Sep 23, 2024
5eeeba9
refactor
River2000i Sep 23, 2024
34e1e7a
add comment
River2000i Sep 23, 2024
ef34ff0
Merge remote-tracking branch 'origin/fix-55184' into fix-55184
River2000i Sep 23, 2024
ed32c70
add `tidb_scatter_region` at global level test
River2000i Sep 23, 2024
eb0e81a
revert
River2000i Sep 23, 2024
318cb17
validate `tidb_scatter_region` in PossibleValues
River2000i Sep 23, 2024
1294345
fix error message
River2000i Sep 23, 2024
e297536
adapt case
River2000i Sep 23, 2024
1e374be
address comment
River2000i Sep 23, 2024
4f2bb51
Merge branch 'master' into fix-55184
River2000i Sep 24, 2024
8765169
resolve conflict
River2000i Sep 24, 2024
4964a01
fix
River2000i Sep 24, 2024
9928422
address comment
River2000i Sep 24, 2024
ca180a3
address comment
River2000i Sep 24, 2024
df2b7ee
address comment
River2000i Sep 24, 2024
8c36330
fix comment
River2000i Sep 24, 2024
1f3e2ae
add comment to describe the testcase(TestScatterRegion) target
River2000i Sep 24, 2024
868fc4f
Merge branch 'master' into fix-55184
River2000i Sep 25, 2024
7a16058
resolve conflict
River2000i Sep 25, 2024
6102476
Merge branch 'master' into fix-55184
River2000i Sep 25, 2024
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 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 @@ -1379,21 +1379,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 isTestPartition {
Expand Down Expand Up @@ -475,10 +475,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 @@ -3214,8 +3214,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)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

so -1 is ok or it should be nil here?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Both is ok. In client-go, the tableID will convert from *int64 to string. If it's nil, it will equal "0". The -1 is used to distinguish it from the default setting of 0 in client-go.
refer: https://github.com/tikv/client-go/blob/6ba909c4ad2de65b5b36d0e5036d0a85f3154cc0/tikv/split_region.go#L245-L248

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 @@ -1212,7 +1212,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 @@ -1655,7 +1655,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 @@ -224,7 +224,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