dxf/ddl: always enable DXF and fast reorg in nextgen (#64179)
ref pingcap/tidb#61702
This commit is contained in:
@ -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")
|
||||
}
|
||||
|
||||
@ -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)
|
||||
|
||||
@ -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);")
|
||||
|
||||
@ -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")
|
||||
|
||||
@ -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
|
||||
|
||||
@ -17,7 +17,7 @@ go_test(
|
||||
],
|
||||
flaky = True,
|
||||
race = "off",
|
||||
shard_count = 23,
|
||||
shard_count = 24,
|
||||
deps = [
|
||||
"//pkg/config",
|
||||
"//pkg/config/kerneltype",
|
||||
|
||||
@ -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")
|
||||
}
|
||||
|
||||
@ -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)
|
||||
|
||||
|
||||
@ -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")
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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))
|
||||
}
|
||||
|
||||
@ -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())
|
||||
|
||||
@ -14,6 +14,7 @@ go_test(
|
||||
shard_count = 30,
|
||||
deps = [
|
||||
"//pkg/config",
|
||||
"//pkg/config/kerneltype",
|
||||
"//pkg/disttask/framework/testutil",
|
||||
"//pkg/meta/model",
|
||||
"//pkg/testkit",
|
||||
|
||||
@ -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');")
|
||||
|
||||
|
||||
@ -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)
|
||||
|
||||
@ -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 (), (), (), ()")
|
||||
|
||||
@ -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 = '';")
|
||||
|
||||
@ -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)
|
||||
|
||||
@ -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)
|
||||
|
||||
|
||||
@ -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 {
|
||||
|
||||
@ -13,6 +13,7 @@ go_test(
|
||||
shard_count = 7,
|
||||
deps = [
|
||||
"//pkg/config",
|
||||
"//pkg/config/kerneltype",
|
||||
"//pkg/ddl",
|
||||
"//pkg/domain",
|
||||
"//pkg/kv",
|
||||
|
||||
@ -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")
|
||||
|
||||
@ -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")
|
||||
|
||||
@ -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",
|
||||
|
||||
@ -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)
|
||||
|
||||
Reference in New Issue
Block a user