diff --git a/ddl/partition.go b/ddl/partition.go index d024ef47b4b43..1a93cfdfb7f9b 100644 --- a/ddl/partition.go +++ b/ddl/partition.go @@ -239,10 +239,6 @@ func alterTablePartitionBundles(t *meta.Meta, tblInfo *model.TableInfo, addingDe p.Definitions = append(tblInfo.Partition.Definitions, addingDefinitions...) tblInfo.Partition = &p - if tblInfo.TiFlashReplica != nil && tblInfo.TiFlashReplica.Count > 0 && tableHasPlacementSettings(tblInfo) { - return nil, errors.Trace(dbterror.ErrIncompatibleTiFlashAndPlacement) - } - // bundle for table should be recomputed because it includes some default configs for partitions tblBundle, err := placement.NewTableBundle(t, tblInfo) if err != nil { diff --git a/ddl/placement_policy.go b/ddl/placement_policy.go index e38797fb22a9e..eedb4bbf09dcf 100644 --- a/ddl/placement_policy.go +++ b/ddl/placement_policy.go @@ -417,19 +417,3 @@ func checkPlacementPolicyNotUsedByTable(tblInfo *model.TableInfo, policy *model. return nil } - -func tableHasPlacementSettings(tblInfo *model.TableInfo) bool { - if tblInfo.PlacementPolicyRef != nil { - return true - } - - if tblInfo.Partition != nil { - for _, def := range tblInfo.Partition.Definitions { - if def.PlacementPolicyRef != nil { - return true - } - } - } - - return false -} diff --git a/ddl/placement_sql_test.go b/ddl/placement_sql_test.go index 42ef1e29986e8..83fa4cc09249e 100644 --- a/ddl/placement_sql_test.go +++ b/ddl/placement_sql_test.go @@ -21,11 +21,11 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/tidb/ddl" "github.com/pingcap/tidb/domain" + "github.com/pingcap/tidb/domain/infosync" mysql "github.com/pingcap/tidb/errno" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/testkit" - "github.com/pingcap/tidb/util/dbterror" "github.com/stretchr/testify/require" ) @@ -583,8 +583,23 @@ func TestPlacementMode(t *testing.T) { } +func checkTiflashReplicaSet(t *testing.T, do *domain.Domain, db, tb string, cnt uint64) { + tbl, err := do.InfoSchema().TableByName(model.NewCIStr(db), model.NewCIStr(tb)) + require.NoError(t, err) + + tiflashReplica := tbl.Meta().TiFlashReplica + if cnt == 0 { + require.Nil(t, tiflashReplica) + return + } + + CheckPlacementRule(infosync.GetMockTiFlash(), *infosync.MakeNewRule(tbl.Meta().ID, 1, nil)) + require.NotNil(t, tiflashReplica) + require.Equal(t, cnt, tiflashReplica.Count) +} + func TestPlacementTiflashCheck(t *testing.T) { - store, clean := testkit.CreateMockStore(t) + store, do, clean := testkit.CreateMockStoreAndDomain(t) defer clean() tk := testkit.NewTestKit(t, store) require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/infoschema/mockTiFlashStoreCount", `return(true)`)) @@ -593,39 +608,65 @@ func TestPlacementTiflashCheck(t *testing.T) { require.NoError(t, err) }() + tiflash := infosync.NewMockTiFlash() + infosync.SetMockTiFlash(tiflash) + defer func() { + tiflash.Lock() + tiflash.StatusServer.Close() + tiflash.Unlock() + }() + tk.MustExec("use test") tk.MustExec("drop placement policy if exists p1") + tk.MustExec("drop placement policy if exists p2") tk.MustExec("drop table if exists tp") tk.MustExec("create placement policy p1 primary_region='r1' regions='r1'") defer tk.MustExec("drop placement policy if exists p1") + tk.MustExec("create placement policy p2 primary_region='r2' regions='r1,r2'") + defer tk.MustExec("drop placement policy if exists p2") + tk.MustExec(`CREATE TABLE tp (id INT) PARTITION BY RANGE (id) ( PARTITION p0 VALUES LESS THAN (100), PARTITION p1 VALUES LESS THAN (1000) )`) defer tk.MustExec("drop table if exists tp") + checkTiflashReplicaSet(t, do, "test", "tp", 0) tk.MustExec("alter table tp set tiflash replica 1") - err := tk.ExecToErr("alter table tp placement policy p1") - require.True(t, dbterror.ErrIncompatibleTiFlashAndPlacement.Equal(err)) - err = tk.ExecToErr("alter table tp partition p0 placement policy p1") - require.True(t, dbterror.ErrIncompatibleTiFlashAndPlacement.Equal(err)) + tk.MustExec("alter table tp placement policy p1") + checkExistTableBundlesInPD(t, do, "test", "tp") + checkTiflashReplicaSet(t, do, "test", "tp", 1) tk.MustQuery("show create table tp").Check(testkit.Rows("" + "tp CREATE TABLE `tp` (\n" + " `id` int(11) DEFAULT NULL\n" + - ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin /*T![placement] PLACEMENT POLICY=`p1` */\n" + "PARTITION BY RANGE (`id`)\n" + "(PARTITION `p0` VALUES LESS THAN (100),\n" + " PARTITION `p1` VALUES LESS THAN (1000))")) + tk.MustExec("alter table tp partition p0 placement policy p2") + checkExistTableBundlesInPD(t, do, "test", "tp") + checkTiflashReplicaSet(t, do, "test", "tp", 1) + tk.MustQuery("show create table tp").Check(testkit.Rows("" + + "tp CREATE TABLE `tp` (\n" + + " `id` int(11) DEFAULT NULL\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin /*T![placement] PLACEMENT POLICY=`p1` */\n" + + "PARTITION BY RANGE (`id`)\n" + + "(PARTITION `p0` VALUES LESS THAN (100) /*T![placement] PLACEMENT POLICY=`p2` */,\n" + + " PARTITION `p1` VALUES LESS THAN (1000))")) + tk.MustExec("drop table tp") tk.MustExec(`CREATE TABLE tp (id INT) placement policy p1 PARTITION BY RANGE (id) ( PARTITION p0 VALUES LESS THAN (100), PARTITION p1 VALUES LESS THAN (1000) )`) - err = tk.ExecToErr("alter table tp set tiflash replica 1") - require.True(t, dbterror.ErrIncompatibleTiFlashAndPlacement.Equal(err)) + checkTiflashReplicaSet(t, do, "test", "tp", 0) + + tk.MustExec("alter table tp set tiflash replica 1") + checkTiflashReplicaSet(t, do, "test", "tp", 1) + checkExistTableBundlesInPD(t, do, "test", "tp") tk.MustQuery("show create table tp").Check(testkit.Rows("" + "tp CREATE TABLE `tp` (\n" + " `id` int(11) DEFAULT NULL\n" + @@ -639,8 +680,11 @@ func TestPlacementTiflashCheck(t *testing.T) { PARTITION p0 VALUES LESS THAN (100) placement policy p1 , PARTITION p1 VALUES LESS THAN (1000) )`) - err = tk.ExecToErr("alter table tp set tiflash replica 1") - require.True(t, dbterror.ErrIncompatibleTiFlashAndPlacement.Equal(err)) + checkTiflashReplicaSet(t, do, "test", "tp", 0) + + tk.MustExec("alter table tp set tiflash replica 1") + checkExistTableBundlesInPD(t, do, "test", "tp") + checkTiflashReplicaSet(t, do, "test", "tp", 1) tk.MustQuery("show create table tp").Check(testkit.Rows("" + "tp CREATE TABLE `tp` (\n" + " `id` int(11) DEFAULT NULL\n" + @@ -654,8 +698,11 @@ func TestPlacementTiflashCheck(t *testing.T) { PARTITION p0 VALUES LESS THAN (100), PARTITION p1 VALUES LESS THAN (1000) )`) - err = tk.ExecToErr("alter table tp set tiflash replica 1") - require.True(t, dbterror.ErrIncompatibleTiFlashAndPlacement.Equal(err)) + checkTiflashReplicaSet(t, do, "test", "tp", 0) + + tk.MustExec("alter table tp set tiflash replica 1") + checkExistTableBundlesInPD(t, do, "test", "tp") + checkTiflashReplicaSet(t, do, "test", "tp", 1) tk.MustQuery("show create table tp").Check(testkit.Rows("" + "tp CREATE TABLE `tp` (\n" + " `id` int(11) DEFAULT NULL\n" + @@ -669,8 +716,11 @@ func TestPlacementTiflashCheck(t *testing.T) { PARTITION p0 VALUES LESS THAN (100) PLACEMENT POLICY p1, PARTITION p1 VALUES LESS THAN (1000) )`) - err = tk.ExecToErr("alter table tp set tiflash replica 1") - require.True(t, dbterror.ErrIncompatibleTiFlashAndPlacement.Equal(err)) + checkTiflashReplicaSet(t, do, "test", "tp", 0) + + tk.MustExec("alter table tp set tiflash replica 1") + checkExistTableBundlesInPD(t, do, "test", "tp") + checkTiflashReplicaSet(t, do, "test", "tp", 1) tk.MustQuery("show create table tp").Check(testkit.Rows("" + "tp CREATE TABLE `tp` (\n" + " `id` int(11) DEFAULT NULL\n" + diff --git a/ddl/table.go b/ddl/table.go index be7e2fabb4922..6293dd6be72e3 100644 --- a/ddl/table.go +++ b/ddl/table.go @@ -1076,11 +1076,6 @@ func (w *worker) onSetTableFlashReplica(d *ddlCtx, t *meta.Meta, job *model.Job) return ver, errors.Trace(err) } - if replicaInfo.Count > 0 && tableHasPlacementSettings(tblInfo) { - job.State = model.JobStateCancelled - return ver, errors.Trace(dbterror.ErrIncompatibleTiFlashAndPlacement) - } - // Ban setting replica count for tables in system database. if tidb_util.IsMemOrSysDB(job.SchemaName) { return ver, errors.Trace(dbterror.ErrUnsupportedAlterReplicaForSysTable) @@ -1419,11 +1414,6 @@ func onAlterTablePartitionPlacement(d *ddlCtx, t *meta.Meta, job *model.Job) (ve return 0, err } - if tblInfo.TiFlashReplica != nil && tblInfo.TiFlashReplica.Count > 0 { - job.State = model.JobStateCancelled - return 0, errors.Trace(dbterror.ErrIncompatibleTiFlashAndPlacement) - } - ptInfo := tblInfo.GetPartitionInfo() var partitionDef *model.PartitionDefinition definitions := ptInfo.Definitions @@ -1489,11 +1479,6 @@ func onAlterTablePlacement(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, return 0, err } - if tblInfo.TiFlashReplica != nil && tblInfo.TiFlashReplica.Count > 0 { - job.State = model.JobStateCancelled - return 0, errors.Trace(dbterror.ErrIncompatibleTiFlashAndPlacement) - } - if _, err = checkPlacementPolicyRefValidAndCanNonValidJob(t, job, policyRefInfo); err != nil { return 0, errors.Trace(err) } diff --git a/util/dbterror/ddl_terror.go b/util/dbterror/ddl_terror.go index bcfe9b7a02730..293cc7da10ce2 100644 --- a/util/dbterror/ddl_terror.go +++ b/util/dbterror/ddl_terror.go @@ -365,8 +365,6 @@ var ( ErrDependentByFunctionalIndex = ClassDDL.NewStd(mysql.ErrDependentByFunctionalIndex) // ErrFunctionalIndexOnBlob when the expression of expression index returns blob or text. ErrFunctionalIndexOnBlob = ClassDDL.NewStd(mysql.ErrFunctionalIndexOnBlob) - // ErrIncompatibleTiFlashAndPlacement when placement and tiflash replica options are set at the same time - ErrIncompatibleTiFlashAndPlacement = ClassDDL.NewStdErr(mysql.ErrUnsupportedDDLOperation, parser_mysql.Message("Placement and tiflash replica options cannot be set at the same time", nil)) // ErrAutoConvert when auto convert happens ErrAutoConvert = ClassDDL.NewStd(mysql.ErrAutoConvert)