diff --git a/ddl/db_test.go b/ddl/db_test.go index deecc27b8974b..309ca8a03e736 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -79,7 +79,7 @@ var _ = Suite(&testDBSuite2{&testDBSuite{}}) var _ = Suite(&testDBSuite3{&testDBSuite{}}) var _ = Suite(&testDBSuite4{&testDBSuite{}}) var _ = Suite(&testDBSuite5{&testDBSuite{}}) -var _ = Suite(&testDBSuite6{&testDBSuite{}}) +var _ = SerialSuites(&testDBSuite6{&testDBSuite{}}) var _ = Suite(&testDBSuite7{&testDBSuite{}}) var _ = Suite(&testDBSuite8{&testDBSuite{}}) var _ = SerialSuites(&testSerialDBSuite{&testDBSuite{}}) diff --git a/ddl/partition.go b/ddl/partition.go index b9d0d72d91864..6f6d3b8b0ff59 100644 --- a/ddl/partition.go +++ b/ddl/partition.go @@ -88,6 +88,7 @@ func (w *worker) onAddTablePartition(d *ddlCtx, t *meta.Meta, job *model.Job) (v return ver, nil } + // notice: addingDefinitions is empty when job is in state model.StateNone tblInfo, partInfo, addingDefinitions, err := checkAddPartition(t, job) if err != nil { return ver, err @@ -117,14 +118,21 @@ func (w *worker) onAddTablePartition(d *ddlCtx, t *meta.Meta, job *model.Job) (v return ver, errors.Trace(err) } + // move the adding definition into tableInfo. + updateAddingPartitionInfo(partInfo, tblInfo) + ver, err = updateVersionAndTableInfoWithCheck(t, job, tblInfo, true) + if err != nil { + return ver, errors.Trace(err) + } + // modify placement settings - for _, def := range addingDefinitions { + for _, def := range tblInfo.Partition.AddingDefinitions { if _, err = checkPlacementPolicyRefValidAndCanNonValidJob(t, job, def.PlacementPolicyRef); err != nil { return ver, errors.Trace(err) } } - bundles, err := alterTablePartitionBundles(t, tblInfo, addingDefinitions) + bundles, err := alterTablePartitionBundles(t, tblInfo, tblInfo.Partition.AddingDefinitions) if err != nil { job.State = model.JobStateCancelled return ver, errors.Trace(err) @@ -135,12 +143,6 @@ func (w *worker) onAddTablePartition(d *ddlCtx, t *meta.Meta, job *model.Job) (v return ver, errors.Wrapf(err, "failed to notify PD the placement rules") } - // move the adding definition into tableInfo. - updateAddingPartitionInfo(partInfo, tblInfo) - ver, err = updateVersionAndTableInfoWithCheck(t, job, tblInfo, true) - if err != nil { - return ver, errors.Trace(err) - } // none -> replica only job.SchemaState = model.StateReplicaOnly case model.StateReplicaOnly: diff --git a/ddl/placement_policy.go b/ddl/placement_policy.go index 65a445ca6263e..3ccba4ef346f6 100644 --- a/ddl/placement_policy.go +++ b/ddl/placement_policy.go @@ -331,7 +331,7 @@ func getPlacementPolicyDependedObjectsIDs(t *meta.Meta, policy *model.PolicyInfo } if tblInfo.Partition != nil { for _, part := range tblInfo.Partition.Definitions { - if part.PlacementPolicyRef != nil && part.PlacementPolicyRef.ID == part.ID { + if part.PlacementPolicyRef != nil && part.PlacementPolicyRef.ID == policy.ID { partIDs = append(partIDs, part.ID) } } diff --git a/ddl/placement_policy_test.go b/ddl/placement_policy_test.go index f4c4eb54e69b3..c8626121515d9 100644 --- a/ddl/placement_policy_test.go +++ b/ddl/placement_policy_test.go @@ -16,6 +16,7 @@ package ddl_test import ( "context" + "encoding/json" "fmt" "math" "strconv" @@ -48,7 +49,83 @@ func clearAllBundles(c *C) { c.Assert(err, IsNil) } +func checkExistTableBundlesInPD(c *C, do *domain.Domain, dbName string, tbName string) { + tblInfo, err := do.InfoSchema().TableByName(model.NewCIStr(dbName), model.NewCIStr(tbName)) + c.Assert(err, IsNil) + + c.Assert(kv.RunInNewTxn(context.TODO(), do.Store(), false, func(ctx context.Context, txn kv.Transaction) error { + t := meta.NewMeta(txn) + checkTableBundlesInPD(c, t, tblInfo.Meta()) + return nil + }), IsNil) +} + +func checkAllBundlesNotChange(c *C, bundles []*placement.Bundle) { + currentBundles, err := infosync.GetAllRuleBundles(context.TODO()) + c.Assert(err, IsNil) + + bundlesMap := make(map[string]*placement.Bundle) + for _, bundle := range currentBundles { + bundlesMap[bundle.ID] = bundle + } + c.Assert(len(bundlesMap), Equals, len(currentBundles)) + c.Assert(len(currentBundles), Equals, len(bundles)) + + for _, bundle := range bundles { + got, ok := bundlesMap[bundle.ID] + c.Assert(ok, IsTrue) + + expectedJSON, err := json.Marshal(bundle) + c.Assert(err, IsNil) + + gotJSON, err := json.Marshal(got) + c.Assert(err, IsNil) + c.Assert(string(gotJSON), Equals, string(expectedJSON)) + } +} + +func checkTableBundlesInPD(c *C, t *meta.Meta, tblInfo *model.TableInfo) { + checks := make([]*struct { + ID string + bundle *placement.Bundle + }, 0) + + bundle, err := placement.NewTableBundle(t, tblInfo) + c.Assert(err, IsNil) + checks = append(checks, &struct { + ID string + bundle *placement.Bundle + }{ID: placement.GroupID(tblInfo.ID), bundle: bundle}) + + if tblInfo.Partition != nil { + for _, def := range tblInfo.Partition.Definitions { + bundle, err := placement.NewPartitionBundle(t, def) + c.Assert(err, IsNil) + checks = append(checks, &struct { + ID string + bundle *placement.Bundle + }{ID: placement.GroupID(def.ID), bundle: bundle}) + } + } + + for _, check := range checks { + got, err := infosync.GetRuleBundle(context.TODO(), check.ID) + c.Assert(err, IsNil) + if check.bundle == nil { + c.Assert(got.IsEmpty(), IsTrue) + } else { + expectedJSON, err := json.Marshal(check.bundle) + c.Assert(err, IsNil) + + gotJSON, err := json.Marshal(got) + c.Assert(err, IsNil) + c.Assert(string(gotJSON), Equals, string(expectedJSON)) + } + } +} + func (s *testDBSuite6) TestPlacementPolicy(c *C) { + clearAllBundles(c) tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop placement policy if exists x") @@ -114,6 +191,10 @@ func (s *testDBSuite6) TestPlacementPolicy(c *C) { "REGIONS=\"cn-east-1,cn-east-2\" ") tk.MustQuery("show warnings").Check(testkit.Rows("Note 8238 Placement policy 'X' already exists")) + bundles, err := infosync.GetAllRuleBundles(context.TODO()) + c.Assert(err, IsNil) + c.Assert(0, Equals, len(bundles)) + tk.MustExec("drop placement policy x") tk.MustGetErrCode("drop placement policy x", mysql.ErrPlacementPolicyNotExists) tk.MustExec("drop placement policy if exists x") @@ -365,12 +446,21 @@ func (s *testDBSuite6) TestCreateOrReplacePlacementPolicy(c *C) { } func (s *testDBSuite6) TestAlterPlacementPolicy(c *C) { + clearAllBundles(c) tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop placement policy if exists x") + tk.MustExec("drop table if exists tp") tk.MustExec("create placement policy x primary_region=\"cn-east-1\" regions=\"cn-east-1,cn-east\"") defer tk.MustExec("drop placement policy if exists x") + // create a table ref to policy x, testing for alter policy will update PD bundles + tk.MustExec(`CREATE TABLE tp (id INT) placement policy x PARTITION BY RANGE (id) ( + PARTITION p0 VALUES LESS THAN (100), + PARTITION p1 VALUES LESS THAN (1000) placement policy x + );`) + defer tk.MustExec("drop table if exists tp") + policy, ok := tk.Se.GetInfoSchema().(infoschema.InfoSchema).PolicyByName(model.NewCIStr("x")) c.Assert(ok, IsTrue) @@ -378,6 +468,7 @@ func (s *testDBSuite6) TestAlterPlacementPolicy(c *C) { tk.MustExec("alter placement policy x PRIMARY_REGION=\"bj\" REGIONS=\"bj,sh\"") tk.MustQuery("show placement where target='POLICY x'").Check(testkit.Rows("POLICY x PRIMARY_REGION=\"bj\" REGIONS=\"bj,sh\" NULL")) tk.MustQuery("select * from information_schema.placement_rules where policy_name = 'x'").Check(testkit.Rows(strconv.FormatInt(policy.ID, 10) + " def x bj bj,sh 0 0")) + checkExistTableBundlesInPD(c, s.dom, "test", "tp") tk.MustExec("alter placement policy x " + "PRIMARY_REGION=\"bj\" " + @@ -385,6 +476,7 @@ func (s *testDBSuite6) TestAlterPlacementPolicy(c *C) { "SCHEDULE=\"EVEN\"") tk.MustQuery("show placement where target='POLICY x'").Check(testkit.Rows("POLICY x PRIMARY_REGION=\"bj\" REGIONS=\"bj\" SCHEDULE=\"EVEN\" NULL")) tk.MustQuery("select * from INFORMATION_SCHEMA.PLACEMENT_RULES WHERE POLICY_NAME='x'").Check(testkit.Rows(strconv.FormatInt(policy.ID, 10) + " def x bj bj EVEN 0 0")) + checkExistTableBundlesInPD(c, s.dom, "test", "tp") tk.MustExec("alter placement policy x " + "LEADER_CONSTRAINTS=\"[+region=us-east-1]\" " + @@ -396,6 +488,7 @@ func (s *testDBSuite6) TestAlterPlacementPolicy(c *C) { tk.MustQuery("SELECT POLICY_NAME,LEADER_CONSTRAINTS,FOLLOWER_CONSTRAINTS,FOLLOWERS FROM information_schema.PLACEMENT_RULES WHERE POLICY_NAME = 'x'").Check( testkit.Rows("x [+region=us-east-1] [+region=us-east-2] 3"), ) + checkExistTableBundlesInPD(c, s.dom, "test", "tp") tk.MustExec("alter placement policy x " + "VOTER_CONSTRAINTS=\"[+region=bj]\" " + @@ -412,8 +505,10 @@ func (s *testDBSuite6) TestAlterPlacementPolicy(c *C) { "SCHEDULE,FOLLOWERS,LEARNERS FROM INFORMATION_SCHEMA.placement_rules WHERE POLICY_NAME='x'").Check( testkit.Rows("def x [+disk=ssd] [+region=sh] 0 3"), ) + checkExistTableBundlesInPD(c, s.dom, "test", "tp") // test alter not exist policies + tk.MustExec("drop table tp") tk.MustExec("drop placement policy x") tk.MustGetErrCode("alter placement policy x REGIONS=\"bj,sh\"", mysql.ErrPlacementPolicyNotExists) tk.MustGetErrCode("alter placement policy x2 REGIONS=\"bj,sh\"", mysql.ErrPlacementPolicyNotExists) @@ -421,6 +516,7 @@ func (s *testDBSuite6) TestAlterPlacementPolicy(c *C) { } func (s *testDBSuite6) TestCreateTableWithPlacementPolicy(c *C) { + clearAllBundles(c) tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t,t_range_p,t_hash_p,t_list_p") @@ -439,6 +535,7 @@ func (s *testDBSuite6) TestCreateTableWithPlacementPolicy(c *C) { "FOLLOWERS=2 ") defer tk.MustExec("DROP TABLE IF EXISTS t") tk.MustQuery("SELECT TABLE_CATALOG, TABLE_SCHEMA, TABLE_NAME, TIDB_PLACEMENT_POLICY_NAME, TIDB_DIRECT_PLACEMENT FROM information_schema.Tables WHERE TABLE_SCHEMA='test' AND TABLE_NAME = 't'").Check(testkit.Rows(`def test t PRIMARY_REGION="cn-east-1" REGIONS="cn-east-1, cn-east-2" FOLLOWERS=2`)) + checkExistTableBundlesInPD(c, s.dom, "test", "t") tbl := testGetTableByName(c, tk.Se, "test", "t") c.Assert(tbl, NotNil) @@ -763,6 +860,7 @@ func (s *testDBSuite6) TestPolicyCacheAndPolicyDependency(c *C) { } func (s *testDBSuite6) TestAlterTablePartitionWithPlacementPolicy(c *C) { + clearAllBundles(c) tk := testkit.NewTestKit(c, s.store) defer func() { tk.MustExec("drop table if exists t1") @@ -782,6 +880,7 @@ func (s *testDBSuite6) TestAlterTablePartitionWithPlacementPolicy(c *C) { "PRIMARY_REGION,REGIONS,CONSTRAINTS,LEADER_CONSTRAINTS,FOLLOWER_CONSTRAINTS,LEARNER_CONSTRAINTS," + "SCHEDULE,FOLLOWERS,LEARNERS FROM INFORMATION_SCHEMA.placement_rules WHERE table_NAME='t1'").Check( testkit.Rows()) + checkExistTableBundlesInPD(c, s.dom, "test", "t1") tk.MustExec("alter table t1 partition p0 " + "PRIMARY_REGION=\"cn-east-1\" " + @@ -795,6 +894,7 @@ func (s *testDBSuite6) TestAlterTablePartitionWithPlacementPolicy(c *C) { ptDef := testGetPartitionDefinitionsByName(c, tk.Se, "test", "t1", "p0") c.Assert(ptDef.PlacementPolicyRef, IsNil) c.Assert(ptDef.DirectPlacementOpts, NotNil) + checkExistTableBundlesInPD(c, s.dom, "test", "t1") checkFunc := func(policySetting *model.PlacementSettings) { c.Assert(policySetting.PrimaryRegion, Equals, "cn-east-1") @@ -832,6 +932,7 @@ func (s *testDBSuite6) TestAlterTablePartitionWithPlacementPolicy(c *C) { tk.MustExec("alter table t1 partition p0 " + "PLACEMENT POLICY=\"x\"") tk.MustQuery("SELECT TABLE_CATALOG, TABLE_SCHEMA, TABLE_NAME, PARTITION_NAME, TIDB_PLACEMENT_POLICY_NAME, TIDB_DIRECT_PLACEMENT FROM information_schema.Partitions WHERE TABLE_SCHEMA='test' AND TABLE_NAME = 't1' AND PARTITION_NAME = 'p0'").Check(testkit.Rows(`def test t1 p0 x `)) + checkExistTableBundlesInPD(c, s.dom, "test", "t1") ptDef = testGetPartitionDefinitionsByName(c, tk.Se, "test", "t1", "p0") c.Assert(ptDef, NotNil) @@ -848,6 +949,7 @@ func (s *testDBSuite6) TestAlterTablePartitionWithPlacementPolicy(c *C) { ptDef = testGetPartitionDefinitionsByName(c, tk.Se, "test", "t1", "p0") c.Assert(ptDef, NotNil) c.Assert(ptDef.DirectPlacementOpts, NotNil) + checkExistTableBundlesInPD(c, s.dom, "test", "t1") checkFunc = func(policySetting *model.PlacementSettings) { c.Assert(policySetting.PrimaryRegion, Equals, "cn-east-1") @@ -883,6 +985,7 @@ func testGetPartitionDefinitionsByName(c *C, ctx sessionctx.Context, db string, } func (s *testDBSuite6) TestPolicyInheritance(c *C) { + clearAllBundles(c) tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t, t0") @@ -897,12 +1000,14 @@ func (s *testDBSuite6) TestPolicyInheritance(c *C) { tk.MustQuery("show create table t").Check(testkit.Rows("t CREATE TABLE `t` (\n" + " `a` int(11) DEFAULT NULL\n" + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin /*T![placement] CONSTRAINTS=\"[+zone=hangzhou]\" */")) + checkExistTableBundlesInPD(c, s.dom, "mydb", "t") tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int) constraints=\"[+zone=suzhou]\"") tk.MustQuery("show create table t").Check(testkit.Rows("t CREATE TABLE `t` (\n" + " `a` int(11) DEFAULT NULL\n" + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin /*T![placement] CONSTRAINTS=\"[+zone=suzhou]\" */")) + checkExistTableBundlesInPD(c, s.dom, "mydb", "t") tk.MustExec("drop table if exists t") // test create table like should not inherit database's placement rules. @@ -910,10 +1015,12 @@ func (s *testDBSuite6) TestPolicyInheritance(c *C) { tk.MustQuery("show create table t0").Check(testkit.Rows("t0 CREATE TABLE `t0` (\n" + " `a` int(11) DEFAULT NULL\n" + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin")) + checkExistTableBundlesInPD(c, s.dom, "mydb", "t0") tk.MustExec("create table t1 like t0") tk.MustQuery("show create table t1").Check(testkit.Rows("t1 CREATE TABLE `t1` (\n" + " `a` int(11) DEFAULT NULL\n" + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin")) + checkExistTableBundlesInPD(c, s.dom, "mydb", "t1") tk.MustExec("drop table if exists t0, t") // table will inherit db's placement rules, which is shared by all partition as default one. @@ -924,6 +1031,7 @@ func (s *testDBSuite6) TestPolicyInheritance(c *C) { "PARTITION BY RANGE (`a`)\n" + "(PARTITION `p0` VALUES LESS THAN (100),\n" + " PARTITION `p1` VALUES LESS THAN (200))")) + checkExistTableBundlesInPD(c, s.dom, "mydb", "t") tk.MustExec("drop table if exists t") // partition's specified placement rules will override the default one. @@ -934,6 +1042,7 @@ func (s *testDBSuite6) TestPolicyInheritance(c *C) { "PARTITION BY RANGE (`a`)\n" + "(PARTITION `p0` VALUES LESS THAN (100) /*T![placement] CONSTRAINTS=\"[+zone=suzhou]\" */,\n" + " PARTITION `p1` VALUES LESS THAN (200))")) + checkExistTableBundlesInPD(c, s.dom, "mydb", "t") tk.MustExec("drop table if exists t") // test partition override table's placement rules. @@ -945,6 +1054,7 @@ func (s *testDBSuite6) TestPolicyInheritance(c *C) { "PARTITION BY RANGE (`a`)\n" + "(PARTITION `p0` VALUES LESS THAN (100) /*T![placement] CONSTRAINTS=\"[+zone=changzhou]\" */,\n" + " PARTITION `p1` VALUES LESS THAN (200))")) + checkExistTableBundlesInPD(c, s.dom, "mydb", "t") } func (s *testDBSuite6) TestDatabasePlacement(c *C) { @@ -1103,6 +1213,7 @@ func (s *testDBSuite6) TestDropTableGCPlacement(c *C) { } func (s *testDBSuite6) TestAlterTablePlacement(c *C) { + clearAllBundles(c) tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists tp") @@ -1126,6 +1237,7 @@ func (s *testDBSuite6) TestAlterTablePlacement(c *C) { "PARTITION BY RANGE (`id`)\n" + "(PARTITION `p0` VALUES LESS THAN (100),\n" + " PARTITION `p1` VALUES LESS THAN (1000))")) + checkExistTableBundlesInPD(c, s.dom, "test", "tp") // alter with policy tk.MustExec("alter table tp placement policy p1") @@ -1141,6 +1253,7 @@ func (s *testDBSuite6) TestAlterTablePlacement(c *C) { c.Assert(err, IsNil) c.Assert(tb.Meta().PlacementPolicyRef.ID, Equals, policy.ID) c.Assert(tb.Meta().DirectPlacementOpts, IsNil) + checkExistTableBundlesInPD(c, s.dom, "test", "tp") // alter with direct placement tk.MustExec("alter table tp primary_region='r2' regions='r1,r2'") @@ -1151,6 +1264,7 @@ func (s *testDBSuite6) TestAlterTablePlacement(c *C) { "PARTITION BY RANGE (`id`)\n" + "(PARTITION `p0` VALUES LESS THAN (100),\n" + " PARTITION `p1` VALUES LESS THAN (1000))")) + checkExistTableBundlesInPD(c, s.dom, "test", "tp") // reset with placement policy 'default' tk.MustExec("alter table tp placement policy default") @@ -1161,6 +1275,7 @@ func (s *testDBSuite6) TestAlterTablePlacement(c *C) { "PARTITION BY RANGE (`id`)\n" + "(PARTITION `p0` VALUES LESS THAN (100),\n" + " PARTITION `p1` VALUES LESS THAN (1000))")) + checkExistTableBundlesInPD(c, s.dom, "test", "tp") // error invalid policy err = tk.ExecToErr("alter table tp placement policy px") @@ -1182,6 +1297,7 @@ func (s *testDBSuite6) TestAlterTablePlacement(c *C) { "PARTITION BY RANGE (`id`)\n" + "(PARTITION `p0` VALUES LESS THAN (100),\n" + " PARTITION `p1` VALUES LESS THAN (1000))")) + checkExistTableBundlesInPD(c, s.dom, "test", "tp") } func (s *testDBSuite6) TestDropTablePartitionGCPlacement(c *C) { @@ -1246,6 +1362,7 @@ func (s *testDBSuite6) TestDropTablePartitionGCPlacement(c *C) { } func (s *testDBSuite6) TestAlterTablePartitionPlacement(c *C) { + clearAllBundles(c) tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists tp") @@ -1273,6 +1390,7 @@ func (s *testDBSuite6) TestAlterTablePartitionPlacement(c *C) { "PARTITION BY RANGE (`id`)\n" + "(PARTITION `p0` VALUES LESS THAN (100),\n" + " PARTITION `p1` VALUES LESS THAN (1000))")) + checkExistTableBundlesInPD(c, s.dom, "test", "tp") // alter with policy tk.MustExec("alter table tp partition p0 placement policy p1") @@ -1288,6 +1406,7 @@ func (s *testDBSuite6) TestAlterTablePartitionPlacement(c *C) { c.Assert(err, IsNil) c.Assert(tb.Meta().Partition.Definitions[0].PlacementPolicyRef.ID, Equals, policy.ID) c.Assert(tb.Meta().Partition.Definitions[0].DirectPlacementOpts, IsNil) + checkExistTableBundlesInPD(c, s.dom, "test", "tp") // alter with direct placement tk.MustExec("alter table tp partition p1 primary_region='r2' regions='r1,r2'") @@ -1298,6 +1417,7 @@ func (s *testDBSuite6) TestAlterTablePartitionPlacement(c *C) { "PARTITION BY RANGE (`id`)\n" + "(PARTITION `p0` VALUES LESS THAN (100) /*T![placement] PLACEMENT POLICY=`p1` */,\n" + " PARTITION `p1` VALUES LESS THAN (1000) /*T![placement] PRIMARY_REGION=\"r2\" REGIONS=\"r1,r2\" */)")) + checkExistTableBundlesInPD(c, s.dom, "test", "tp") tk.MustExec("alter table tp partition p1 primary_region='r3' regions='r3,r4'") tk.MustQuery("show create table tp").Check(testkit.Rows("" + @@ -1307,6 +1427,7 @@ func (s *testDBSuite6) TestAlterTablePartitionPlacement(c *C) { "PARTITION BY RANGE (`id`)\n" + "(PARTITION `p0` VALUES LESS THAN (100) /*T![placement] PLACEMENT POLICY=`p1` */,\n" + " PARTITION `p1` VALUES LESS THAN (1000) /*T![placement] PRIMARY_REGION=\"r3\" REGIONS=\"r3,r4\" */)")) + checkExistTableBundlesInPD(c, s.dom, "test", "tp") // reset with placement policy 'default' tk.MustExec("alter table tp partition p1 placement policy default") @@ -1317,6 +1438,7 @@ func (s *testDBSuite6) TestAlterTablePartitionPlacement(c *C) { "PARTITION BY RANGE (`id`)\n" + "(PARTITION `p0` VALUES LESS THAN (100) /*T![placement] PLACEMENT POLICY=`p1` */,\n" + " PARTITION `p1` VALUES LESS THAN (1000))")) + checkExistTableBundlesInPD(c, s.dom, "test", "tp") tk.MustExec("alter table tp partition p0 placement policy default") tk.MustQuery("show create table tp").Check(testkit.Rows("" + @@ -1326,6 +1448,7 @@ func (s *testDBSuite6) TestAlterTablePartitionPlacement(c *C) { "PARTITION BY RANGE (`id`)\n" + "(PARTITION `p0` VALUES LESS THAN (100),\n" + " PARTITION `p1` VALUES LESS THAN (1000))")) + checkExistTableBundlesInPD(c, s.dom, "test", "tp") // error invalid policy err = tk.ExecToErr("alter table tp partition p1 placement policy px") @@ -1351,9 +1474,11 @@ func (s *testDBSuite6) TestAlterTablePartitionPlacement(c *C) { "PARTITION BY RANGE (`id`)\n" + "(PARTITION `p0` VALUES LESS THAN (100),\n" + " PARTITION `p1` VALUES LESS THAN (1000))")) + checkExistTableBundlesInPD(c, s.dom, "test", "tp") } func (s *testDBSuite6) TestAddPartitionWithPlacement(c *C) { + clearAllBundles(c) tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists tp") @@ -1377,6 +1502,7 @@ func (s *testDBSuite6) TestAddPartitionWithPlacement(c *C) { "PARTITION BY RANGE (`id`)\n" + "(PARTITION `p0` VALUES LESS THAN (100),\n" + " PARTITION `p1` VALUES LESS THAN (1000))")) + checkExistTableBundlesInPD(c, s.dom, "test", "tp") // Add partitions tk.MustExec(`alter table tp add partition ( @@ -1394,6 +1520,7 @@ func (s *testDBSuite6) TestAddPartitionWithPlacement(c *C) { " PARTITION `p2` VALUES LESS THAN (10000) /*T![placement] PLACEMENT POLICY=`p1` */,\n" + " PARTITION `p3` VALUES LESS THAN (100000) /*T![placement] PRIMARY_REGION=\"r1\" REGIONS=\"r1,r2\" */,\n" + " PARTITION `p4` VALUES LESS THAN (1000000))")) + checkExistTableBundlesInPD(c, s.dom, "test", "tp") tb, err := tk.Se.GetInfoSchema().(infoschema.InfoSchema).TableByName(model.NewCIStr("test"), model.NewCIStr("tp")) c.Assert(err, IsNil) @@ -1422,6 +1549,7 @@ func (s *testDBSuite6) TestAddPartitionWithPlacement(c *C) { " PARTITION `p2` VALUES LESS THAN (10000) /*T![placement] PLACEMENT POLICY=`p1` */,\n" + " PARTITION `p3` VALUES LESS THAN (100000) /*T![placement] PRIMARY_REGION=\"r1\" REGIONS=\"r1,r2\" */,\n" + " PARTITION `p4` VALUES LESS THAN (1000000))")) + checkExistTableBundlesInPD(c, s.dom, "test", "tp") } func (s *testDBSuite6) TestTruncateTableWithPlacement(c *C) { @@ -1682,6 +1810,7 @@ func (s *testDBSuite6) TestTruncatePartitionGCWithPlacement(c *C) { } func (s *testDBSuite6) TestExchangePartitionWithPlacement(c *C) { + clearAllBundles(c) tk := testkit.NewTestKit(c, s.store) tk.MustExec("set @@tidb_enable_exchange_partition=1") tk.MustExec("use test") @@ -1754,6 +1883,7 @@ func (s *testDBSuite6) TestExchangePartitionWithPlacement(c *C) { c.Assert(t1.Meta().ID, Equals, par0ID) c.Assert(t1.Meta().DirectPlacementOpts, IsNil) c.Assert(t1.Meta().PlacementPolicyRef.ID, Equals, policy1.ID) + checkExistTableBundlesInPD(c, s.dom, "test", "tp") // exchange par0, t2 tk.MustExec("alter table tp exchange partition p0 with table t2") @@ -1780,6 +1910,7 @@ func (s *testDBSuite6) TestExchangePartitionWithPlacement(c *C) { c.Assert(t2.Meta().ID, Equals, t1ID) c.Assert(t2.Meta().DirectPlacementOpts, IsNil) c.Assert(t2.Meta().PlacementPolicyRef, IsNil) + checkExistTableBundlesInPD(c, s.dom, "test", "tp") // exchange par1, t1 tk.MustExec("alter table tp exchange partition p1 with table t1") @@ -1806,6 +1937,7 @@ func (s *testDBSuite6) TestExchangePartitionWithPlacement(c *C) { c.Assert(t1.Meta().ID, Equals, par1ID) c.Assert(t1.Meta().DirectPlacementOpts, IsNil) c.Assert(t1.Meta().PlacementPolicyRef.ID, Equals, policy1.ID) + checkExistTableBundlesInPD(c, s.dom, "test", "tp") // exchange par2, t2 tk.MustExec("alter table tp exchange partition p2 with table t2") @@ -1833,4 +1965,93 @@ func (s *testDBSuite6) TestExchangePartitionWithPlacement(c *C) { c.Assert(t2.Meta().ID, Equals, par2ID) c.Assert(t2.Meta().DirectPlacementOpts, IsNil) c.Assert(t2.Meta().PlacementPolicyRef, IsNil) + checkExistTableBundlesInPD(c, s.dom, "test", "tp") +} + +func (s *testDBSuite6) TestPDFail(c *C) { + defer func() { + c.Assert(failpoint.Disable("github.com/pingcap/tidb/domain/infosync/putRuleBundlesError"), IsNil) + }() + + clearAllBundles(c) + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop placement policy if exists p1") + tk.MustExec("drop table if exists t1, t2, tp") + + tk.MustExec("create placement policy p1 primary_region=\"cn-east-1\" regions=\"cn-east-1,cn-east\"") + defer tk.MustExec("drop placement policy if exists p1") + + tk.MustExec("create table t1(id int)") + defer tk.MustExec("drop table if exists t1") + + 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) placement policy p1 + );`) + defer tk.MustExec("drop table if exists tp") + existBundles, err := infosync.GetAllRuleBundles(context.TODO()) + c.Assert(err, IsNil) + + c.Assert(failpoint.Enable("github.com/pingcap/tidb/domain/infosync/putRuleBundlesError", "return(true)"), IsNil) + + // alter policy + err = tk.ExecToErr("alter placement policy p1 primary_region='rx' regions='rx'") + c.Assert(infosync.ErrHTTPServiceError.Equal(err), IsTrue) + tk.MustQuery("show create placement policy p1").Check(testkit.Rows("p1 CREATE PLACEMENT POLICY `p1` PRIMARY_REGION=\"cn-east-1\" REGIONS=\"cn-east-1,cn-east\"")) + checkAllBundlesNotChange(c, existBundles) + + // create table + err = tk.ExecToErr("create table t2 (id int) placement policy p1") + c.Assert(infosync.ErrHTTPServiceError.Equal(err), IsTrue) + err = tk.ExecToErr("show create table t2") + c.Assert(infoschema.ErrTableNotExists.Equal(err), IsTrue) + checkAllBundlesNotChange(c, existBundles) + + // alter table + err = tk.ExecToErr("alter table t1 placement policy p1") + c.Assert(infosync.ErrHTTPServiceError.Equal(err), IsTrue) + tk.MustQuery("show create table t1").Check(testkit.Rows("t1 CREATE TABLE `t1` (\n" + + " `id` int(11) DEFAULT NULL\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin")) + checkAllBundlesNotChange(c, existBundles) + + // add partition + err = tk.ExecToErr("alter table tp add partition (" + + "partition p2 values less than (10000) placement policy p1," + + "partition p3 values less than (100000) primary_region=\"r1\" regions=\"r1,r2\"" + + ")") + c.Assert(infosync.ErrHTTPServiceError.Equal(err), IsTrue) + 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),\n" + + " PARTITION `p1` VALUES LESS THAN (1000) /*T![placement] PLACEMENT POLICY=`p1` */)")) + checkAllBundlesNotChange(c, existBundles) + + // alter partition + err = tk.ExecToErr(`alter table tp PARTITION p1 primary_region="r2" regions="r2,r3"`) + c.Assert(infosync.ErrHTTPServiceError.Equal(err), IsTrue) + 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),\n" + + " PARTITION `p1` VALUES LESS THAN (1000) /*T![placement] PLACEMENT POLICY=`p1` */)")) + checkAllBundlesNotChange(c, existBundles) + + // exchange partition + tk.MustExec("alter table tp exchange partition p1 with table t1") + c.Assert(infosync.ErrHTTPServiceError.Equal(err), IsTrue) + tk.MustQuery("show create table t1").Check(testkit.Rows("t1 CREATE TABLE `t1` (\n" + + " `id` int(11) DEFAULT NULL\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin")) + 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),\n" + + " PARTITION `p1` VALUES LESS THAN (1000) /*T![placement] PLACEMENT POLICY=`p1` */)")) + checkAllBundlesNotChange(c, existBundles) }