From bdd2b6f2845caa7e0ff79dae7ab1010efee38bc0 Mon Sep 17 00:00:00 2001 From: D3Hunter Date: Fri, 31 Oct 2025 01:24:13 +0800 Subject: [PATCH] dxf/ddl: always enable DXF and fast reorg in nextgen (#64179) ref pingcap/tidb#61702 --- pkg/ddl/index_change_test.go | 29 ++++++++ pkg/ddl/index_modify_test.go | 6 ++ pkg/ddl/ingest/integration_test.go | 33 +++++++-- pkg/ddl/multi_schema_change_test.go | 7 +- pkg/ddl/tests/indexmerge/merge_test.go | 6 -- .../framework/integrationtests/BUILD.bazel | 2 +- .../integrationtests/framework_test.go | 12 ++++ .../importer/importer_testkit_test.go | 3 + pkg/executor/test/ddl/ddl_test.go | 3 + pkg/sessionctx/vardef/runtime.go | 3 +- pkg/sessionctx/vardef/runtime_test.go | 2 + pkg/telemetry/data_feature_usage_test.go | 1 - tests/realtikvtest/addindextest/BUILD.bazel | 1 + .../addindextest/add_index_test.go | 13 +++- .../addindextest1/disttask_test.go | 70 +++++++++++-------- .../addindextest2/alter_job_test.go | 7 +- .../addindextest2/global_sort_test.go | 42 ++++++++--- .../addindextest3/functional_test.go | 4 ++ .../realtikvtest/addindextest3/ingest_test.go | 67 +++++++++++++++--- .../addindextest3/operator_test.go | 5 +- tests/realtikvtest/addindextest4/BUILD.bazel | 1 + .../addindextest4/integration_test.go | 10 ++- .../pessimistictest/pessimistic_test.go | 3 + tests/realtikvtest/testutils/BUILD.bazel | 1 + tests/realtikvtest/testutils/common.go | 5 +- 25 files changed, 265 insertions(+), 71 deletions(-) diff --git a/pkg/ddl/index_change_test.go b/pkg/ddl/index_change_test.go index bdac906574..122bc3cf7e 100644 --- a/pkg/ddl/index_change_test.go +++ b/pkg/ddl/index_change_test.go @@ -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") +} diff --git a/pkg/ddl/index_modify_test.go b/pkg/ddl/index_modify_test.go index a6dc24eb8d..ceb38ee5af 100644 --- a/pkg/ddl/index_modify_test.go +++ b/pkg/ddl/index_modify_test.go @@ -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) diff --git a/pkg/ddl/ingest/integration_test.go b/pkg/ddl/ingest/integration_test.go index 078e71e0e0..2477d775f7 100644 --- a/pkg/ddl/ingest/integration_test.go +++ b/pkg/ddl/ingest/integration_test.go @@ -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);") diff --git a/pkg/ddl/multi_schema_change_test.go b/pkg/ddl/multi_schema_change_test.go index 3469d6a1fa..9184afdbc3 100644 --- a/pkg/ddl/multi_schema_change_test.go +++ b/pkg/ddl/multi_schema_change_test.go @@ -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") diff --git a/pkg/ddl/tests/indexmerge/merge_test.go b/pkg/ddl/tests/indexmerge/merge_test.go index 2cabc0c061..e9c9a40b8c 100644 --- a/pkg/ddl/tests/indexmerge/merge_test.go +++ b/pkg/ddl/tests/indexmerge/merge_test.go @@ -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 diff --git a/pkg/disttask/framework/integrationtests/BUILD.bazel b/pkg/disttask/framework/integrationtests/BUILD.bazel index 26e6dc6fef..711ccd0100 100644 --- a/pkg/disttask/framework/integrationtests/BUILD.bazel +++ b/pkg/disttask/framework/integrationtests/BUILD.bazel @@ -17,7 +17,7 @@ go_test( ], flaky = True, race = "off", - shard_count = 23, + shard_count = 24, deps = [ "//pkg/config", "//pkg/config/kerneltype", diff --git a/pkg/disttask/framework/integrationtests/framework_test.go b/pkg/disttask/framework/integrationtests/framework_test.go index cce746aa3e..b5f809cb7b 100644 --- a/pkg/disttask/framework/integrationtests/framework_test.go +++ b/pkg/disttask/framework/integrationtests/framework_test.go @@ -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") +} diff --git a/pkg/executor/importer/importer_testkit_test.go b/pkg/executor/importer/importer_testkit_test.go index 098fba9b65..2dd2bd6e4d 100644 --- a/pkg/executor/importer/importer_testkit_test.go +++ b/pkg/executor/importer/importer_testkit_test.go @@ -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) diff --git a/pkg/executor/test/ddl/ddl_test.go b/pkg/executor/test/ddl/ddl_test.go index 211507b1a0..69b103f01d 100644 --- a/pkg/executor/test/ddl/ddl_test.go +++ b/pkg/executor/test/ddl/ddl_test.go @@ -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") diff --git a/pkg/sessionctx/vardef/runtime.go b/pkg/sessionctx/vardef/runtime.go index 6a6bec52a5..d119df2040 100644 --- a/pkg/sessionctx/vardef/runtime.go +++ b/pkg/sessionctx/vardef/runtime.go @@ -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 diff --git a/pkg/sessionctx/vardef/runtime_test.go b/pkg/sessionctx/vardef/runtime_test.go index cb850e1ad6..bf23e8f484 100644 --- a/pkg/sessionctx/vardef/runtime_test.go +++ b/pkg/sessionctx/vardef/runtime_test.go @@ -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)) } diff --git a/pkg/telemetry/data_feature_usage_test.go b/pkg/telemetry/data_feature_usage_test.go index 65c7327002..b59c8d9f7a 100644 --- a/pkg/telemetry/data_feature_usage_test.go +++ b/pkg/telemetry/data_feature_usage_test.go @@ -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()) diff --git a/tests/realtikvtest/addindextest/BUILD.bazel b/tests/realtikvtest/addindextest/BUILD.bazel index 8029f8a58f..de8b96b887 100644 --- a/tests/realtikvtest/addindextest/BUILD.bazel +++ b/tests/realtikvtest/addindextest/BUILD.bazel @@ -14,6 +14,7 @@ go_test( shard_count = 30, deps = [ "//pkg/config", + "//pkg/config/kerneltype", "//pkg/disttask/framework/testutil", "//pkg/meta/model", "//pkg/testkit", diff --git a/tests/realtikvtest/addindextest/add_index_test.go b/tests/realtikvtest/addindextest/add_index_test.go index 217ea72838..f9d3604ead 100644 --- a/tests/realtikvtest/addindextest/add_index_test.go +++ b/tests/realtikvtest/addindextest/add_index_test.go @@ -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');") diff --git a/tests/realtikvtest/addindextest1/disttask_test.go b/tests/realtikvtest/addindextest1/disttask_test.go index 6d3676120f..ebf86b65bd 100644 --- a/tests/realtikvtest/addindextest1/disttask_test.go +++ b/tests/realtikvtest/addindextest1/disttask_test.go @@ -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) diff --git a/tests/realtikvtest/addindextest2/alter_job_test.go b/tests/realtikvtest/addindextest2/alter_job_test.go index dc6014401a..ceb4cdfbad 100644 --- a/tests/realtikvtest/addindextest2/alter_job_test.go +++ b/tests/realtikvtest/addindextest2/alter_job_test.go @@ -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 (), (), (), ()") diff --git a/tests/realtikvtest/addindextest2/global_sort_test.go b/tests/realtikvtest/addindextest2/global_sort_test.go index 2ca170c243..c005deab84 100644 --- a/tests/realtikvtest/addindextest2/global_sort_test.go +++ b/tests/realtikvtest/addindextest2/global_sort_test.go @@ -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 = '';") diff --git a/tests/realtikvtest/addindextest3/functional_test.go b/tests/realtikvtest/addindextest3/functional_test.go index 3c16650a2d..715a03fdfc 100644 --- a/tests/realtikvtest/addindextest3/functional_test.go +++ b/tests/realtikvtest/addindextest3/functional_test.go @@ -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) diff --git a/tests/realtikvtest/addindextest3/ingest_test.go b/tests/realtikvtest/addindextest3/ingest_test.go index 0c16d3458d..1bc24ae932 100644 --- a/tests/realtikvtest/addindextest3/ingest_test.go +++ b/tests/realtikvtest/addindextest3/ingest_test.go @@ -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) diff --git a/tests/realtikvtest/addindextest3/operator_test.go b/tests/realtikvtest/addindextest3/operator_test.go index 3a0c59f5aa..d2dfefd3a9 100644 --- a/tests/realtikvtest/addindextest3/operator_test.go +++ b/tests/realtikvtest/addindextest3/operator_test.go @@ -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 { diff --git a/tests/realtikvtest/addindextest4/BUILD.bazel b/tests/realtikvtest/addindextest4/BUILD.bazel index 483dce57f0..7d93ce6231 100644 --- a/tests/realtikvtest/addindextest4/BUILD.bazel +++ b/tests/realtikvtest/addindextest4/BUILD.bazel @@ -13,6 +13,7 @@ go_test( shard_count = 7, deps = [ "//pkg/config", + "//pkg/config/kerneltype", "//pkg/ddl", "//pkg/domain", "//pkg/kv", diff --git a/tests/realtikvtest/addindextest4/integration_test.go b/tests/realtikvtest/addindextest4/integration_test.go index 2a00f8ad8f..799e72532a 100644 --- a/tests/realtikvtest/addindextest4/integration_test.go +++ b/tests/realtikvtest/addindextest4/integration_test.go @@ -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") diff --git a/tests/realtikvtest/pessimistictest/pessimistic_test.go b/tests/realtikvtest/pessimistictest/pessimistic_test.go index 6170ede7b8..4fc0a9a229 100644 --- a/tests/realtikvtest/pessimistictest/pessimistic_test.go +++ b/tests/realtikvtest/pessimistictest/pessimistic_test.go @@ -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") diff --git a/tests/realtikvtest/testutils/BUILD.bazel b/tests/realtikvtest/testutils/BUILD.bazel index 38aea4184b..64fda2e50c 100644 --- a/tests/realtikvtest/testutils/BUILD.bazel +++ b/tests/realtikvtest/testutils/BUILD.bazel @@ -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", diff --git a/tests/realtikvtest/testutils/common.go b/tests/realtikvtest/testutils/common.go index 340e8474ef..8d91339aec 100644 --- a/tests/realtikvtest/testutils/common.go +++ b/tests/realtikvtest/testutils/common.go @@ -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)