dxf/ddl: always enable DXF and fast reorg in nextgen (#64179)

ref pingcap/tidb#61702
This commit is contained in:
D3Hunter
2025-10-31 01:24:13 +08:00
committed by GitHub
parent 9c0cfaf558
commit bdd2b6f284
25 changed files with 265 additions and 71 deletions

View File

@ -22,6 +22,7 @@ import (
"time"
"github.com/pingcap/errors"
"github.com/pingcap/tidb/pkg/config/kerneltype"
"github.com/pingcap/tidb/pkg/ddl"
"github.com/pingcap/tidb/pkg/kv"
"github.com/pingcap/tidb/pkg/meta/model"
@ -393,6 +394,9 @@ func checkDropDeleteOnly(ctx sessionctx.Context, writeTbl, delTbl table.Table) e
}
func TestAddIndexRowCountUpdate(t *testing.T) {
if kerneltype.IsNextGen() {
t.Skip("add-index always runs on DXF with ingest mode in nextgen")
}
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
@ -432,3 +436,28 @@ func TestAddIndexRowCountUpdate(t *testing.T) {
}()
tk.MustExec("alter table t add index idx(c2);")
}
func TestFastReOrgAlwaysEnabledOnNextGen(t *testing.T) {
if kerneltype.IsClassic() {
t.Skip("This test is only for next-gen TiDB")
}
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
tk.MustQuery("select @@global.tidb_ddl_enable_fast_reorg").Equal(testkit.Rows("1"))
require.ErrorContains(t, tk.ExecToErr("set global tidb_ddl_enable_fast_reorg=0"),
"setting tidb_ddl_enable_fast_reorg is not supported in the next generation of TiDB")
}
func TestReadOnlyVarsInNextGen(t *testing.T) {
if kerneltype.IsClassic() {
t.Skip("This test is only for next-gen TiDB")
}
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
require.ErrorContains(t, tk.ExecToErr("set global tidb_max_dist_task_nodes=5"),
"setting tidb_max_dist_task_nodes is not supported in the next generation of TiDB")
require.ErrorContains(t, tk.ExecToErr("set global tidb_ddl_reorg_max_write_speed=5"),
"setting tidb_ddl_reorg_max_write_speed is not supported in the next generation of TiDB")
require.ErrorContains(t, tk.ExecToErr("set global tidb_ddl_disk_quota=5"),
"setting tidb_ddl_disk_quota is not supported in the next generation of TiDB")
}

View File

@ -1194,6 +1194,9 @@ func TestAddIndexWithDupIndex(t *testing.T) {
}
func TestAddIndexUniqueFailOnDuplicate(t *testing.T) {
if kerneltype.IsNextGen() {
t.Skip("add-index always runs on DXF with ingest mode in nextgen")
}
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
@ -1954,6 +1957,9 @@ func TestAddColumnarIndexRollback(t *testing.T) {
}
func TestInsertDuplicateBeforeIndexMerge(t *testing.T) {
if kerneltype.IsNextGen() {
t.Skip("add-index always runs on DXF with ingest mode in nextgen")
}
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
tk2 := testkit.NewTestKit(t, store)

View File

@ -86,6 +86,9 @@ func TestAddIndexIngestGeneratedColumns(t *testing.T) {
}
func TestIngestError(t *testing.T) {
if kerneltype.IsNextGen() {
t.Skip("add-index always runs on DXF with ingest mode in nextgen")
}
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test;")
@ -134,6 +137,9 @@ func TestIngestError(t *testing.T) {
}
func TestAddIndexIngestPanic(t *testing.T) {
if kerneltype.IsNextGen() {
t.Skip("add-index always runs on DXF with ingest mode in nextgen")
}
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test;")
@ -161,6 +167,9 @@ func TestAddIndexIngestPanic(t *testing.T) {
}
func TestAddIndexSetInternalSessions(t *testing.T) {
if kerneltype.IsNextGen() {
t.Skip("add-index always runs on DXF with ingest mode in nextgen")
}
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test;")
@ -417,6 +426,9 @@ func TestMultiSchemaAddIndexMerge(t *testing.T) {
}
func TestAddIndexIngestJobWriteConflict(t *testing.T) {
if kerneltype.IsNextGen() {
t.Skip("add-index always runs on DXF with ingest mode in nextgen")
}
store := testkit.CreateMockStore(t)
defer ingesttestutil.InjectMockBackendCtx(t, store)()
tk := testkit.NewTestKit(t, store)
@ -450,6 +462,9 @@ func TestAddIndexIngestJobWriteConflict(t *testing.T) {
}
func TestAddIndexIngestPartitionCheckpoint(t *testing.T) {
if kerneltype.IsNextGen() {
t.Skip("add-index always runs on DXF with ingest mode in nextgen")
}
store := testkit.CreateMockStore(t)
defer ingesttestutil.InjectMockBackendCtx(t, store)()
tk := testkit.NewTestKit(t, store)
@ -650,8 +665,13 @@ func TestModifyColumnWithMultipleIndex(t *testing.T) {
) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin`
for _, tc := range testcases {
t.Run(tc.caseName, func(t *testing.T) {
tk.MustExec(fmt.Sprintf("set global tidb_enable_dist_task = %s;", tc.enableDistTask))
tk.MustExec(fmt.Sprintf("set global tidb_ddl_enable_fast_reorg = %s;", tc.enableFastReorg))
if kerneltype.IsNextGen() && tc.enableDistTask == "off" {
t.Skip("add-index always runs on DXF with ingest mode in nextgen")
}
if kerneltype.IsClassic() {
tk.MustExec(fmt.Sprintf("set global tidb_enable_dist_task = %s;", tc.enableDistTask))
tk.MustExec(fmt.Sprintf("set global tidb_ddl_enable_fast_reorg = %s;", tc.enableFastReorg))
}
tk.MustExec("DROP TABLE IF EXISTS t")
tk.MustExec(createTableSQL)
tk.MustExec("insert into t values(19,1,1),(17,2,2)")
@ -678,8 +698,13 @@ func TestModifyColumnWithIndexWithDefaultValue(t *testing.T) {
}
for _, tc := range testcases {
t.Run(tc.caseName, func(t *testing.T) {
tk.MustExec(fmt.Sprintf("set global tidb_enable_dist_task = %s;", tc.enableDistTask))
tk.MustExec(fmt.Sprintf("set global tidb_ddl_enable_fast_reorg = %s;", tc.enableFastReorg))
if kerneltype.IsNextGen() && tc.enableDistTask == "off" {
t.Skip("add-index always runs on DXF with ingest mode in nextgen")
}
if kerneltype.IsClassic() {
tk.MustExec(fmt.Sprintf("set global tidb_enable_dist_task = %s;", tc.enableDistTask))
tk.MustExec(fmt.Sprintf("set global tidb_ddl_enable_fast_reorg = %s;", tc.enableFastReorg))
}
tk.MustExec("drop table if exists t1")
tk.MustExec("create table t1 (c int(10), c1 datetime default (date_format(now(),'%Y-%m-%d')));")
tk.MustExec("insert into t1(c) values (1), (2);")

View File

@ -638,6 +638,9 @@ func TestMultiSchemaChangeAlterIndex(t *testing.T) {
}
func TestMultiSchemaChangeMixCancelled(t *testing.T) {
if kerneltype.IsNextGen() {
t.Skip("add-index always runs on DXF with ingest mode in nextgen")
}
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test;")
@ -668,7 +671,6 @@ func TestMultiSchemaChangeAdminShowDDLJobs(t *testing.T) {
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
tk.MustExec("set global tidb_ddl_enable_fast_reorg = 1;")
tk.MustExec("create table t (a int, b int, c int)")
tk.MustExec("insert into t values (1, 2, 3)")
@ -886,6 +888,9 @@ func TestMultiSchemaChangeBlockedByRowLevelChecksum(t *testing.T) {
}
func TestMultiSchemaChangePollJobCount(t *testing.T) {
if kerneltype.IsNextGen() {
t.Skip("add-index always runs on DXF with ingest mode in nextgen")
}
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")

View File

@ -45,7 +45,6 @@ func TestAddIndexMergeProcess(t *testing.T) {
tk.MustExec("insert into t values (1, 2, 3), (4, 5, 6);")
// Force onCreateIndex use the txn-merge process.
ingest.LitInitialized = false
tk.MustExec("set @@global.tidb_ddl_enable_fast_reorg = 1;")
var checkErr error
var runDML, backfillDone bool
@ -84,7 +83,6 @@ func TestAddPrimaryKeyMergeProcess(t *testing.T) {
tk.MustExec("insert into t values (1, 2, 3), (4, 5, 6);")
// Force onCreateIndex use the backfill-merge process.
ingest.LitInitialized = false
tk.MustExec("set @@global.tidb_ddl_enable_fast_reorg = 1;")
var checkErr error
var runDML, backfillDone bool
@ -126,7 +124,6 @@ func TestAddIndexMergeVersionIndexValue(t *testing.T) {
tk.MustExec("insert into t values (1);")
// Force onCreateIndex use the txn-merge process.
ingest.LitInitialized = false
tk.MustExec("set @@global.tidb_ddl_enable_fast_reorg = 1;")
var checkErr error
var runDML bool
@ -175,7 +172,6 @@ func TestAddIndexMergeIndexUntouchedValue(t *testing.T) {
tk.MustExec("insert into t values (1, 1, 'a', 'a')")
// Force onCreateIndex use the txn-merge process.
ingest.LitInitialized = false
tk.MustExec("set @@global.tidb_ddl_enable_fast_reorg = 1;")
var checkErrs []error
var runInsert bool
@ -301,7 +297,6 @@ func TestAddIndexMergeIndexUpdateOnDeleteOnly(t *testing.T) {
// Force onCreateIndex use the txn-merge process.
ingest.LitInitialized = false
tk.MustExec("set @@global.tidb_ddl_enable_fast_reorg = 1;")
tk.MustExec("set @@global.tidb_enable_mutation_checker = 1;")
tk.MustExec("set @@global.tidb_txn_assertion_level = 'STRICT';")
@ -432,7 +427,6 @@ func TestAddIndexMergeConflictWithPessimistic(t *testing.T) {
// Force onCreateIndex use the txn-merge process.
ingest.LitInitialized = false
tk.MustExec("set @@global.tidb_ddl_enable_fast_reorg = 1;")
tk.MustExec("set @@global.tidb_enable_metadata_lock = 0;")
runPessimisticTxn := false

View File

@ -17,7 +17,7 @@ go_test(
],
flaky = True,
race = "off",
shard_count = 23,
shard_count = 24,
deps = [
"//pkg/config",
"//pkg/config/kerneltype",

View File

@ -24,6 +24,7 @@ import (
"github.com/pingcap/errors"
"github.com/pingcap/failpoint"
"github.com/pingcap/tidb/pkg/config/kerneltype"
"github.com/pingcap/tidb/pkg/disttask/framework/handle"
mockexecute "github.com/pingcap/tidb/pkg/disttask/framework/mock/execute"
"github.com/pingcap/tidb/pkg/disttask/framework/proto"
@ -327,3 +328,14 @@ func TestTaskCancelledBeforeUpdateTask(t *testing.T) {
task := testutil.SubmitAndWaitTask(c.Ctx, t, "key1", scope, 1)
require.Equal(t, proto.TaskStateReverted, task.State)
}
func TestDXFAlwaysEnabledOnNextGen(t *testing.T) {
if kerneltype.IsClassic() {
t.Skip("This test is only for next-gen TiDB")
}
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
tk.MustQuery("select @@global.tidb_enable_dist_task").Equal(testkit.Rows("1"))
require.ErrorContains(t, tk.ExecToErr("set global tidb_enable_dist_task=0"),
"setting tidb_enable_dist_task is not supported in the next generation of TiDB")
}

View File

@ -172,6 +172,9 @@ func TestVerifyChecksum(t *testing.T) {
}
func TestGetTargetNodeCpuCnt(t *testing.T) {
if kerneltype.IsNextGen() {
t.Skip("DXF is always enabled in nextgen")
}
store, tm, ctx := testutil.InitTableTest(t)
tk := testkit.NewTestKit(t, store)

View File

@ -825,6 +825,9 @@ func TestSetDDLReorgMaxWriteSpeed(t *testing.T) {
}
func TestLoadDDLDistributeVars(t *testing.T) {
if kerneltype.IsNextGen() {
t.Skip("DXF is always enabled in nextgen")
}
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")

View File

@ -69,7 +69,8 @@ func GetPlanReplayerGCLease() time.Duration {
func IsReadOnlyVarInNextGen(name string) bool {
name = strings.ToLower(name)
switch name {
case TiDBEnableMDL, TiDBMaxDistTaskNodes, TiDBDDLReorgMaxWriteSpeed, TiDBDDLDiskQuota:
case TiDBEnableMDL, TiDBMaxDistTaskNodes, TiDBDDLReorgMaxWriteSpeed, TiDBDDLDiskQuota,
TiDBEnableDistTask, TiDBDDLEnableFastReorg:
return true
}
return false

View File

@ -31,4 +31,6 @@ func TestIsReadOnlyVarInNextGen(t *testing.T) {
require.True(t, IsReadOnlyVarInNextGen(TiDBMaxDistTaskNodes))
require.True(t, IsReadOnlyVarInNextGen(TiDBDDLReorgMaxWriteSpeed))
require.True(t, IsReadOnlyVarInNextGen(TiDBDDLDiskQuota))
require.True(t, IsReadOnlyVarInNextGen(TiDBDDLEnableFastReorg))
require.True(t, IsReadOnlyVarInNextGen(TiDBEnableDistTask))
}

View File

@ -569,7 +569,6 @@ func TestAddIndexAccelerationAndMDL(t *testing.T) {
tk.MustExec("set global tidb_enable_metadata_lock = 1")
}
tk.MustExec("set @@global.tidb_ddl_enable_fast_reorg = on")
allow := vardef.EnableFastReorg.Load()
require.Equal(t, true, allow)
usage, err = telemetry.GetFeatureUsage(tk.Session())

View File

@ -14,6 +14,7 @@ go_test(
shard_count = 30,
deps = [
"//pkg/config",
"//pkg/config/kerneltype",
"//pkg/disttask/framework/testutil",
"//pkg/meta/model",
"//pkg/testkit",

View File

@ -19,6 +19,7 @@ import (
"testing"
"github.com/pingcap/tidb/pkg/config"
"github.com/pingcap/tidb/pkg/config/kerneltype"
"github.com/pingcap/tidb/pkg/disttask/framework/testutil"
"github.com/pingcap/tidb/pkg/meta/model"
"github.com/pingcap/tidb/pkg/testkit"
@ -104,7 +105,9 @@ func TestAddForeignKeyWithAutoCreateIndex(t *testing.T) {
tk.MustExec("drop database if exists fk_index;")
tk.MustExec("create database fk_index;")
tk.MustExec("use fk_index;")
tk.MustExec(`set global tidb_ddl_enable_fast_reorg=1;`)
if kerneltype.IsClassic() {
tk.MustExec(`set global tidb_ddl_enable_fast_reorg=1;`)
}
tk.MustExec("create table employee (id bigint auto_increment key, pid bigint)")
tk.MustExec("insert into employee (id) values (1),(2),(3),(4),(5),(6),(7),(8)")
for range 14 {
@ -153,7 +156,9 @@ func TestAddUKWithSmallIntHandles(t *testing.T) {
tk.MustExec("drop database if exists small;")
tk.MustExec("create database small;")
tk.MustExec("use small;")
tk.MustExec(`set global tidb_ddl_enable_fast_reorg=1;`)
if kerneltype.IsClassic() {
tk.MustExec(`set global tidb_ddl_enable_fast_reorg=1;`)
}
tk.MustExec("create table t (a bigint, b int, primary key (a) clustered)")
tk.MustExec("insert into t values (-9223372036854775808, 1),(-9223372036854775807, 1)")
tk.MustContainErrMsg("alter table t add unique index uk(b)", "Duplicate entry '1' for key 't.uk'")
@ -164,7 +169,9 @@ func TestAddUniqueDuplicateIndexes(t *testing.T) {
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
tk.MustExec(`set global tidb_ddl_enable_fast_reorg=1;`)
if kerneltype.IsClassic() {
tk.MustExec(`set global tidb_ddl_enable_fast_reorg=1;`)
}
tk.MustExec("create table t(a int DEFAULT '-13202', b varchar(221) NOT NULL DEFAULT 'duplicatevalue', " +
"c int NOT NULL DEFAULT '0');")

View File

@ -165,7 +165,12 @@ func TestAddIndexDistCancelWithPartition(t *testing.T) {
tk.MustExec("drop database if exists test;")
tk.MustExec("create database test;")
tk.MustExec("use test;")
tk.MustExec(`set global tidb_enable_dist_task=1;`)
if kerneltype.IsClassic() {
tk.MustExec(`set global tidb_enable_dist_task=1;`)
t.Cleanup(func() {
tk.MustExec(`set global tidb_enable_dist_task=0;`)
})
}
tk.MustExec("create table t(a bigint auto_random primary key) partition by hash(a) partitions 8;")
tk.MustExec("insert into t values (), (), (), (), (), ()")
@ -188,8 +193,6 @@ func TestAddIndexDistCancelWithPartition(t *testing.T) {
tk.MustExec("admin check table t;")
tk.MustExec("alter table t add index idx2(a);")
tk.MustExec("admin check table t;")
tk.MustExec(`set global tidb_enable_dist_task=0;`)
}
func TestAddIndexDistCancel(t *testing.T) {
@ -199,14 +202,15 @@ func TestAddIndexDistCancel(t *testing.T) {
tk.MustExec("drop database if exists addindexlit;")
tk.MustExec("create database addindexlit;")
tk.MustExec("use addindexlit;")
tk.MustExec(`set global tidb_ddl_enable_fast_reorg=on;`)
tk.MustExec("create table t (a int, b int);")
tk.MustExec("insert into t values (1, 1), (2, 2), (3, 3);")
tk2 := testkit.NewTestKit(t, store)
tk2.MustExec("use addindexlit;")
tk2.MustExec(`set global tidb_ddl_enable_fast_reorg=on;`)
tk.MustExec("set @@global.tidb_enable_dist_task = 1;")
if kerneltype.IsClassic() {
tk2.MustExec(`set global tidb_ddl_enable_fast_reorg=on;`)
tk.MustExec("set @@global.tidb_enable_dist_task = 1;")
}
tk2.MustExec("create table t2 (a int, b int);")
tk2.MustExec("insert into t2 values (1, 1), (2, 2), (3, 3);")
@ -321,7 +325,12 @@ func TestAddIndexDistPauseAndResume(t *testing.T) {
// make sure the task is paused.
syncChan <- struct{}{}
}))
tk.MustExec(`set global tidb_enable_dist_task=1;`)
if kerneltype.IsClassic() {
tk.MustExec(`set global tidb_enable_dist_task=1;`)
t.Cleanup(func() {
tk.MustExec(`set global tidb_enable_dist_task=0;`)
})
}
tk.MustExec("alter table t add index idx1(a);")
tk.MustExec("admin check table t;")
require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/ddl/mockDMLExecutionAddIndexSubTaskFinish"))
@ -351,11 +360,12 @@ func TestAddIndexDistPauseAndResume(t *testing.T) {
tk.MustExec("admin check table t;")
require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/ddl/pauseAfterDistTaskFinished"))
require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/ddl/syncDDLTaskPause"))
tk.MustExec(`set global tidb_enable_dist_task=0;`)
}
func TestAddIndexInvalidDistTaskVariableSetting(t *testing.T) {
if kerneltype.IsNextGen() {
t.Skip("DXF and fast reorg are always enabled on nextgen")
}
store := realtikvtest.CreateMockStoreAndSetup(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("drop database if exists addindexlit;")
@ -379,11 +389,10 @@ func TestAddIndexForCurrentTimestampColumn(t *testing.T) {
tk.MustExec("drop database if exists addindexlit;")
tk.MustExec("create database addindexlit;")
tk.MustExec("use addindexlit;")
t.Cleanup(func() {
tk.MustExec("set global tidb_enable_dist_task = off;")
})
tk.MustExec(`set global tidb_ddl_enable_fast_reorg=on;`)
tk.MustExec("set global tidb_enable_dist_task = on;")
if kerneltype.IsClassic() {
tk.MustExec(`set global tidb_ddl_enable_fast_reorg=on;`)
tk.MustExec("set global tidb_enable_dist_task = on;")
}
tk.MustExec("create table t (a timestamp default current_timestamp);")
tk.MustExec("insert into t values ();")
@ -402,11 +411,10 @@ func TestAddUKErrorMessage(t *testing.T) {
tk.MustExec("drop database if exists addindexlit;")
tk.MustExec("create database addindexlit;")
tk.MustExec("use addindexlit;")
t.Cleanup(func() {
tk.MustExec("set global tidb_enable_dist_task = off;")
})
tk.MustExec(`set global tidb_ddl_enable_fast_reorg=on;`)
tk.MustExec("set global tidb_enable_dist_task = on;")
if kerneltype.IsClassic() {
tk.MustExec(`set global tidb_ddl_enable_fast_reorg=on;`)
tk.MustExec("set global tidb_enable_dist_task = on;")
}
tk.MustExec("create table t (a int primary key, b int);")
tk.MustExec("insert into t values (5, 1), (10005, 1), (20005, 1), (30005, 1);")
@ -419,10 +427,12 @@ func TestAddIndexDistLockAcquireFailed(t *testing.T) {
store := realtikvtest.CreateMockStoreAndSetup(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
tk.MustExec("set global tidb_enable_dist_task = on;")
t.Cleanup(func() {
tk.MustExec("set global tidb_enable_dist_task = off;")
})
if kerneltype.IsClassic() {
tk.MustExec("set global tidb_enable_dist_task = on;")
t.Cleanup(func() {
tk.MustExec("set global tidb_enable_dist_task = off;")
})
}
retryableErrs := []string{
"requested lease not found",
"mvcc: required revision has been compacted",
@ -447,10 +457,12 @@ func TestAddIndexScheduleAway(t *testing.T) {
store := realtikvtest.CreateMockStoreAndSetup(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
tk.MustExec("set global tidb_enable_dist_task = on;")
t.Cleanup(func() {
tk.MustExec("set global tidb_enable_dist_task = off;")
})
if kerneltype.IsClassic() {
tk.MustExec("set global tidb_enable_dist_task = on;")
t.Cleanup(func() {
tk.MustExec("set global tidb_enable_dist_task = off;")
})
}
tk.MustExec("create table t (a int, b int);")
tk.MustExec("insert into t values (1, 1);")
@ -510,7 +522,9 @@ func TestAddIndexDistCleanUpBlock(t *testing.T) {
tk.MustExec("drop database if exists test;")
tk.MustExec("create database test;")
tk.MustExec("use test;")
tk.MustExec(`set global tidb_enable_dist_task=1;`)
if kerneltype.IsClassic() {
tk.MustExec(`set global tidb_enable_dist_task=1;`)
}
var wg sync.WaitGroup
for i := range 4 {
wg.Add(1)

View File

@ -33,6 +33,9 @@ import (
)
func TestAlterThreadRightAfterJobFinish(t *testing.T) {
if kerneltype.IsNextGen() {
t.Skip("DXF is always enabled on nextgen")
}
store := realtikvtest.CreateMockStoreAndSetup(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
@ -78,7 +81,9 @@ func TestAlterJobOnDXF(t *testing.T) {
tk.MustExec("drop database if exists test;")
tk.MustExec("create database test;")
tk.MustExec("use test;")
tk.MustExec(`set global tidb_enable_dist_task=1;`)
if kerneltype.IsClassic() {
tk.MustExec(`set global tidb_enable_dist_task=1;`)
}
tk.MustExec("create table t1(a bigint auto_random primary key);")
for range 16 {
tk.MustExec("insert into t1 values (), (), (), ()")

View File

@ -169,7 +169,9 @@ func TestGlobalSortBasic(t *testing.T) {
tk.MustExec("drop database if exists addindexlit;")
tk.MustExec("create database addindexlit;")
tk.MustExec("use addindexlit;")
tk.MustExec(`set @@global.tidb_ddl_enable_fast_reorg = 1;`)
if kerneltype.IsClassic() {
tk.MustExec(`set @@global.tidb_ddl_enable_fast_reorg = 1;`)
}
tk.MustExec(fmt.Sprintf(`set @@global.tidb_cloud_storage_uri = "%s"`, cloudStorageURI))
cloudStorageURI = handle.GetCloudStorageURI(context.Background(), store) // path with cluster id
defer func() {
@ -268,11 +270,18 @@ func TestGlobalSortMultiSchemaChange(t *testing.T) {
}
for _, tc := range testCases {
t.Run(tc.name, func(t *testing.T) {
if kerneltype.IsNextGen() && tc.cloudStorageURI == "" {
t.Skip("local sort might ingest duplicate KV, cause overlapped sst")
if kerneltype.IsNextGen() {
if tc.cloudStorageURI == "" {
t.Skip("local sort might ingest duplicate KV, cause overlapped sst")
}
if tc.enableDistTask == "0" {
t.Skip("DXF is always enabled on nextgen")
}
}
if kerneltype.IsClassic() {
tk.MustExec("set @@global.tidb_ddl_enable_fast_reorg = " + tc.enableFastReorg + ";")
tk.MustExec("set @@global.tidb_enable_dist_task = " + tc.enableDistTask + ";")
}
tk.MustExec("set @@global.tidb_ddl_enable_fast_reorg = " + tc.enableFastReorg + ";")
tk.MustExec("set @@global.tidb_enable_dist_task = " + tc.enableDistTask + ";")
tk.MustExec("set @@global.tidb_cloud_storage_uri = '" + tc.cloudStorageURI + "';")
for _, tn := range tableNames {
if kerneltype.IsNextGen() && tc.cloudStorageURI != "" && tn == "t_partition" {
@ -295,11 +304,16 @@ func TestGlobalSortMultiSchemaChange(t *testing.T) {
})
}
tk.MustExec("set @@global.tidb_enable_dist_task = 1;")
if kerneltype.IsClassic() {
tk.MustExec("set @@global.tidb_enable_dist_task = 1;")
}
tk.MustExec("set @@global.tidb_cloud_storage_uri = '';")
}
func TestAddIndexIngestShowReorgTp(t *testing.T) {
if kerneltype.IsNextGen() {
t.Skip("DXF is always enabled on nextgen")
}
_, cloudStorageURI := genServerWithStorage(t)
store := realtikvtest.CreateMockStoreAndSetup(t)
@ -346,7 +360,9 @@ func TestGlobalSortDuplicateErrMsg(t *testing.T) {
tk.MustExec("drop database if exists addindexlit;")
tk.MustExec("create database addindexlit;")
tk.MustExec("use addindexlit;")
tk.MustExec(`set @@global.tidb_ddl_enable_fast_reorg = 1;`)
if kerneltype.IsClassic() {
tk.MustExec(`set @@global.tidb_ddl_enable_fast_reorg = 1;`)
}
tk.MustExec(fmt.Sprintf(`set @@global.tidb_cloud_storage_uri = "%s"`, cloudStorageURI))
atomic.StoreUint32(&ddl.EnableSplitTableRegion, 1)
tk.MustExec("set @@session.tidb_scatter_region = 'table'")
@ -526,7 +542,9 @@ func TestIngestUseGivenTS(t *testing.T) {
tk.MustExec("drop database if exists addindexlit;")
tk.MustExec("create database addindexlit;")
tk.MustExec("use addindexlit;")
tk.MustExec(`set global tidb_ddl_enable_fast_reorg = on;`)
if kerneltype.IsClassic() {
tk.MustExec(`set global tidb_ddl_enable_fast_reorg = on;`)
}
tk.MustExec("set @@global.tidb_cloud_storage_uri = '" + cloudStorageURI + "';")
t.Cleanup(func() {
tk.MustExec("set @@global.tidb_cloud_storage_uri = '';")
@ -569,7 +587,9 @@ func TestAlterJobOnDXFWithGlobalSort(t *testing.T) {
store := realtikvtest.CreateMockStoreAndSetup(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec(`set global tidb_ddl_enable_fast_reorg = on;`)
if kerneltype.IsClassic() {
tk.MustExec(`set global tidb_ddl_enable_fast_reorg = on;`)
}
tk.MustExec("set @@global.tidb_cloud_storage_uri = '" + cloudStorageURI + "';")
t.Cleanup(func() {
tk.MustExec("set @@global.tidb_cloud_storage_uri = '';")
@ -637,7 +657,9 @@ func TestDXFAddIndexRealtimeSummary(t *testing.T) {
store := realtikvtest.CreateMockStoreAndSetup(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec(`set global tidb_ddl_enable_fast_reorg = on;`)
if kerneltype.IsClassic() {
tk.MustExec(`set global tidb_ddl_enable_fast_reorg = on;`)
}
tk.MustExec("set @@global.tidb_cloud_storage_uri = '" + cloudStorageURI + "';")
t.Cleanup(func() {
tk.MustExec("set @@global.tidb_cloud_storage_uri = '';")

View File

@ -22,6 +22,7 @@ import (
"sync"
"testing"
"github.com/pingcap/tidb/pkg/config/kerneltype"
"github.com/pingcap/tidb/pkg/ddl"
"github.com/pingcap/tidb/pkg/ddl/ingest"
"github.com/pingcap/tidb/pkg/disttask/framework/testutil"
@ -162,6 +163,9 @@ func TestTiDBEncodeKeyTempIndexKey(t *testing.T) {
}
func TestAddIndexPresplitIndexRegions(t *testing.T) {
if kerneltype.IsNextGen() {
t.Skip("DXF and fast reorg are always enabled on nextgen")
}
testutil.ReduceCheckInterval(t)
store := realtikvtest.CreateMockStoreAndSetup(t)
tk := testkit.NewTestKit(t, store)

View File

@ -53,7 +53,9 @@ func TestAddIndexIngestMemoryUsage(t *testing.T) {
tk.MustExec("drop database if exists addindexlit;")
tk.MustExec("create database addindexlit;")
tk.MustExec("use addindexlit;")
tk.MustExec(`set global tidb_ddl_enable_fast_reorg=on;`)
if kerneltype.IsClassic() {
tk.MustExec(`set global tidb_ddl_enable_fast_reorg=on;`)
}
local.RunInTest = true
@ -78,6 +80,9 @@ func TestAddIndexIngestMemoryUsage(t *testing.T) {
}
func TestAddIndexIngestLimitOneBackend(t *testing.T) {
if kerneltype.IsNextGen() {
t.Skip("DXF is always enabled on nextgen")
}
store := realtikvtest.CreateMockStoreAndSetup(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("drop database if exists addindexlit;")
@ -153,7 +158,9 @@ func TestAddIndexIngestWriterCountOnPartitionTable(t *testing.T) {
tk.MustExec("drop database if exists addindexlit;")
tk.MustExec("create database addindexlit;")
tk.MustExec("use addindexlit;")
tk.MustExec(`set global tidb_ddl_enable_fast_reorg=on;`)
if kerneltype.IsClassic() {
tk.MustExec(`set global tidb_ddl_enable_fast_reorg=on;`)
}
tk.MustExec("create table t (a int primary key) partition by hash(a) partitions 32;")
var sb strings.Builder
@ -187,7 +194,9 @@ func TestIngestMVIndexOnPartitionTable(t *testing.T) {
tk.MustExec("drop database if exists addindexlit;")
tk.MustExec("create database addindexlit;")
tk.MustExec("use addindexlit;")
tk.MustExec(`set global tidb_ddl_enable_fast_reorg=on;`)
if kerneltype.IsClassic() {
tk.MustExec(`set global tidb_ddl_enable_fast_reorg=on;`)
}
var sb strings.Builder
@ -234,7 +243,9 @@ func TestAddIndexIngestAdjustBackfillWorkerCountFail(t *testing.T) {
tk.MustExec("drop database if exists addindexlit;")
tk.MustExec("create database addindexlit;")
tk.MustExec("use addindexlit;")
tk.MustExec(`set global tidb_ddl_enable_fast_reorg=on;`)
if kerneltype.IsClassic() {
tk.MustExec(`set global tidb_ddl_enable_fast_reorg=on;`)
}
ingest.ImporterRangeConcurrencyForTest = &atomic.Int32{}
ingest.ImporterRangeConcurrencyForTest.Store(2)
@ -270,7 +281,9 @@ func TestAddIndexIngestEmptyTable(t *testing.T) {
tk.MustExec("create database addindexlit;")
tk.MustExec("use addindexlit;")
tk.MustExec("create table t (a int);")
tk.MustExec(`set global tidb_ddl_enable_fast_reorg=on;`)
if kerneltype.IsClassic() {
tk.MustExec(`set global tidb_ddl_enable_fast_reorg=on;`)
}
tk.MustExec("alter table t add index idx(a);")
rows := tk.MustQuery("admin show ddl jobs 1;").Rows()
@ -289,7 +302,9 @@ func TestAddIndexIngestRestoredData(t *testing.T) {
tk.MustExec("drop database if exists addindexlit;")
tk.MustExec("create database addindexlit;")
tk.MustExec("use addindexlit;")
tk.MustExec(`set global tidb_ddl_enable_fast_reorg=on;`)
if kerneltype.IsClassic() {
tk.MustExec(`set global tidb_ddl_enable_fast_reorg=on;`)
}
tk.MustExec(`
CREATE TABLE tbl_5 (
@ -321,7 +336,9 @@ func TestAddIndexIngestUniqueKey(t *testing.T) {
tk.MustExec("drop database if exists addindexlit;")
tk.MustExec("create database addindexlit;")
tk.MustExec("use addindexlit;")
tk.MustExec(`set global tidb_ddl_enable_fast_reorg=on;`)
if kerneltype.IsClassic() {
tk.MustExec(`set global tidb_ddl_enable_fast_reorg=on;`)
}
tk.MustExec("create table t (a int primary key, b int);")
tk.MustExec("insert into t values (1, 1), (10000, 1);")
@ -347,7 +364,9 @@ func TestAddIndexSplitTableRanges(t *testing.T) {
tk.MustExec("drop database if exists addindexlit;")
tk.MustExec("create database addindexlit;")
tk.MustExec("use addindexlit;")
tk.MustExec(`set global tidb_ddl_enable_fast_reorg=on;`)
if kerneltype.IsClassic() {
tk.MustExec(`set global tidb_ddl_enable_fast_reorg=on;`)
}
tk.MustExec("create table t (a int primary key, b int);")
for i := range 8 {
@ -374,6 +393,9 @@ func TestAddIndexSplitTableRanges(t *testing.T) {
}
func TestAddIndexLoadTableRangeError(t *testing.T) {
if kerneltype.IsNextGen() {
t.Skip("DXF is always enabled on nextgen")
}
disttestutil.ReduceCheckInterval(t)
store := realtikvtest.CreateMockStoreAndSetup(t)
tk := testkit.NewTestKit(t, store)
@ -403,6 +425,9 @@ func TestAddIndexLoadTableRangeError(t *testing.T) {
}
func TestAddIndexMockFlushError(t *testing.T) {
if kerneltype.IsNextGen() {
t.Skip("DXF is always enabled on nextgen")
}
store := realtikvtest.CreateMockStoreAndSetup(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("drop database if exists addindexlit;")
@ -430,6 +455,9 @@ func TestAddIndexMockFlushError(t *testing.T) {
}
func TestAddIndexDiskQuotaTS(t *testing.T) {
if kerneltype.IsNextGen() {
t.Skip("DXF and fast-reorg is always enabled on nextgen, and we only support global sort in release")
}
disttestutil.ReduceCheckInterval(t)
store := realtikvtest.CreateMockStoreAndSetup(t)
tk := testkit.NewTestKit(t, store)
@ -472,7 +500,7 @@ func testAddIndexDiskQuotaTS(t *testing.T, tk *testkit.TestKit) {
func TestAddIndexAdvanceWatermarkFailed(t *testing.T) {
if kerneltype.IsNextGen() {
t.Skip("have overlapped ingest sst, skip")
t.Skip("DXF is always enabled on nextgen")
}
store := realtikvtest.CreateMockStoreAndSetup(t)
tk := testkit.NewTestKit(t, store)
@ -551,7 +579,7 @@ func TestAddIndexTempDirDataRemoved(t *testing.T) {
func TestAddIndexRemoteDuplicateCheck(t *testing.T) {
if kerneltype.IsNextGen() {
t.Skip("have overlapped ingest sst, skip")
t.Skip("DXF is always enabled on nextgen")
}
store := realtikvtest.CreateMockStoreAndSetup(t)
tk := testkit.NewTestKit(t, store)
@ -703,6 +731,9 @@ func TestAddIndexImportFailed(t *testing.T) {
}
func TestAddEmptyMultiValueIndex(t *testing.T) {
if kerneltype.IsNextGen() {
t.Skip("DXF is always enabled on nextgen")
}
store := realtikvtest.CreateMockStoreAndSetup(t)
tk := testkit.NewTestKit(t, store)
@ -729,6 +760,9 @@ func TestAddUniqueIndexDuplicatedError(t *testing.T) {
}
func TestFirstLitSlowStart(t *testing.T) {
if kerneltype.IsNextGen() {
t.Skip("DXF is always enabled on nextgen")
}
store := realtikvtest.CreateMockStoreAndSetup(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("drop database if exists addindexlit;")
@ -777,7 +811,9 @@ func TestConcFastReorg(t *testing.T) {
tk.MustExec("drop database if exists addindexlit;")
tk.MustExec("create database addindexlit;")
tk.MustExec("use addindexlit;")
tk.MustExec(`set global tidb_ddl_enable_fast_reorg=on;`)
if kerneltype.IsClassic() {
tk.MustExec(`set global tidb_ddl_enable_fast_reorg=on;`)
}
tblNum := 10
for i := range tblNum {
@ -836,6 +872,9 @@ func TestIssue55808(t *testing.T) {
}
func TestAddIndexBackfillLostTempIndexValues(t *testing.T) {
if kerneltype.IsNextGen() {
t.Skip("DXF is always enabled on nextgen")
}
store := realtikvtest.CreateMockStoreAndSetup(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("drop database if exists addindexlit;")
@ -893,6 +932,9 @@ func TestAddIndexBackfillLostTempIndexValues(t *testing.T) {
}
func TestAddIndexInsertSameOriginIndexValue(t *testing.T) {
if kerneltype.IsNextGen() {
t.Skip("DXF is always enabled on nextgen")
}
store := realtikvtest.CreateMockStoreAndSetup(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("drop database if exists addindexlit;")
@ -927,6 +969,9 @@ func TestAddIndexInsertSameOriginIndexValue(t *testing.T) {
}
func TestMergeTempIndexSplitConflictTxn(t *testing.T) {
if kerneltype.IsNextGen() {
t.Skip("DXF is always enabled on nextgen")
}
store := realtikvtest.CreateMockStoreAndSetup(t)
tk := testkit.NewTestKit(t, store)

View File

@ -25,6 +25,7 @@ import (
"github.com/ngaut/pools"
"github.com/pingcap/failpoint"
"github.com/pingcap/tidb/pkg/config"
"github.com/pingcap/tidb/pkg/config/kerneltype"
"github.com/pingcap/tidb/pkg/ddl"
"github.com/pingcap/tidb/pkg/ddl/copr"
"github.com/pingcap/tidb/pkg/ddl/ingest"
@ -355,7 +356,9 @@ func prepare(t *testing.T, tk *testkit.TestKit, dom *domain.Domain, regionCnt in
tk.MustExec("drop database if exists op;")
tk.MustExec("create database op;")
tk.MustExec("use op;")
tk.MustExec(`set global tidb_ddl_enable_fast_reorg=on;`)
if kerneltype.IsClassic() {
tk.MustExec(`set global tidb_ddl_enable_fast_reorg=on;`)
}
tk.MustExec("create table t(a int primary key, b int, index idx(b));")
for i := range regionCnt {

View File

@ -13,6 +13,7 @@ go_test(
shard_count = 7,
deps = [
"//pkg/config",
"//pkg/config/kerneltype",
"//pkg/ddl",
"//pkg/domain",
"//pkg/kv",

View File

@ -23,6 +23,7 @@ import (
"time"
"github.com/pingcap/tidb/pkg/config"
"github.com/pingcap/tidb/pkg/config/kerneltype"
"github.com/pingcap/tidb/pkg/ddl"
"github.com/pingcap/tidb/pkg/meta/model"
"github.com/pingcap/tidb/pkg/testkit"
@ -43,8 +44,10 @@ func TestMultiSchemaChangeTwoIndexes(t *testing.T) {
store := realtikvtest.CreateMockStoreAndSetup(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test;")
tk.MustExec("set @@global.tidb_ddl_enable_fast_reorg=on;")
tk.MustExec("set @@global.tidb_enable_dist_task=on;")
if kerneltype.IsClassic() {
tk.MustExec("set @@global.tidb_ddl_enable_fast_reorg=on;")
tk.MustExec("set @@global.tidb_enable_dist_task=on;")
}
createTables := []string{
"create table t (id int, b int, c int, primary key(id) clustered);",
@ -109,6 +112,9 @@ func TestMultiSchemaChangeTwoIndexes(t *testing.T) {
}
func TestFixAdminAlterDDLJobs(t *testing.T) {
if kerneltype.IsNextGen() {
t.Skip("DXF is always enabled on nextgen")
}
store := realtikvtest.CreateMockStoreAndSetup(t)
tk1 := testkit.NewTestKit(t, store)
tk1.MustExec("use test")

View File

@ -2275,6 +2275,9 @@ func Test1PCWithSchemaChange(t *testing.T) {
}
func TestPlanCacheSchemaChange(t *testing.T) {
if kerneltype.IsNextGen() {
t.Skip("fast reorg is always enabled on nextgen")
}
store := realtikvtest.CreateMockStoreAndSetup(t)
tmp := testkit.NewTestKit(t, store)
tmp.MustExec("set tidb_enable_prepared_plan_cache=ON")

View File

@ -10,6 +10,7 @@ go_library(
importpath = "github.com/pingcap/tidb/tests/realtikvtest/testutils",
visibility = ["//visibility:public"],
deps = [
"//pkg/config/kerneltype",
"//pkg/kv",
"//pkg/testkit",
"//pkg/util/logutil",

View File

@ -23,6 +23,7 @@ import (
"time"
"github.com/pingcap/failpoint"
"github.com/pingcap/tidb/pkg/config/kerneltype"
"github.com/pingcap/tidb/pkg/kv"
"github.com/pingcap/tidb/pkg/testkit"
"github.com/pingcap/tidb/pkg/util/logutil"
@ -91,7 +92,9 @@ func InitTest(t *testing.T) *SuiteContext {
tk.MustExec("drop database if exists addindex;")
tk.MustExec("create database addindex;")
tk.MustExec("use addindex;")
tk.MustExec(`set global tidb_ddl_enable_fast_reorg=on;`)
if kerneltype.IsClassic() {
tk.MustExec(`set global tidb_ddl_enable_fast_reorg=on;`)
}
ctx := newSuiteContext(t, tk, store)
createTable(tk)