diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 9b670911c4b71..6b31fc717ab93 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -18,7 +18,6 @@ package ddl import ( - "encoding/hex" "fmt" "math" "strconv" @@ -28,7 +27,6 @@ import ( "unicode/utf8" "github.com/cznic/mathutil" - "github.com/go-yaml/yaml" "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/parser/ast" @@ -49,12 +47,10 @@ import ( "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/table/tables" - "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" driver "github.com/pingcap/tidb/types/parser_driver" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/chunk" - "github.com/pingcap/tidb/util/codec" "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/dbterror" "github.com/pingcap/tidb/util/domainutil" @@ -5917,150 +5913,6 @@ func (d *ddl) AlterIndexVisibility(ctx sessionctx.Context, ident ast.Ident, inde return errors.Trace(err) } -func buildPlacementSpecReplicasAndConstraint(replicas uint64, cnstr string) ([]*placement.Rule, error) { - rules := []*placement.Rule{} - - cnstbytes := []byte(cnstr) - - constraints1 := []string{} - err1 := yaml.UnmarshalStrict(cnstbytes, &constraints1) - if err1 == nil { - // can not emit REPLICAS with an array label - if replicas == 0 { - return rules, errors.Errorf("array CONSTRAINTS should be with a positive REPLICAS") - } - - labelConstraints, err := placement.NewConstraints(constraints1) - if err != nil { - return rules, err - } - - rules = append(rules, &placement.Rule{ - Count: int(replicas), - Constraints: labelConstraints, - }) - - return rules, nil - } - - constraints2 := map[string]int{} - err2 := yaml.UnmarshalStrict(cnstbytes, &constraints2) - if err2 == nil { - ruleCnt := int(replicas) - - for labels, cnt := range constraints2 { - if cnt <= 0 { - return rules, errors.Errorf("count should be positive, but got %d", cnt) - } - - if replicas != 0 { - ruleCnt -= cnt - if ruleCnt < 0 { - return rules, errors.Errorf("REPLICAS should be larger or equal to the number of total replicas, but got %d", replicas) - } - } - - labelConstraints, err := placement.NewConstraints(strings.Split(strings.TrimSpace(labels), ",")) - if err != nil { - return rules, err - } - - rules = append(rules, &placement.Rule{ - Count: cnt, - Constraints: labelConstraints, - }) - } - - if ruleCnt > 0 { - rules = append(rules, &placement.Rule{ - Count: ruleCnt, - }) - } - - return rules, nil - } - - return nil, errors.Errorf("constraint is neither an array of string, nor a string-to-number map, due to:\n%s\n%s", err1, err2) -} - -func buildPlacementSpecs(bundle *placement.Bundle, specs []*ast.PlacementSpec) (*placement.Bundle, error) { - var err error - var spec *ast.PlacementSpec - - for _, rspec := range specs { - spec = rspec - - var role placement.PeerRoleType - switch spec.Role { - case ast.PlacementRoleFollower: - role = placement.Follower - case ast.PlacementRoleLeader: - if spec.Replicas == 0 { - spec.Replicas = 1 - } - if spec.Replicas > 1 { - err = errors.Errorf("replicas can only be 1 when the role is leader") - } - role = placement.Leader - case ast.PlacementRoleLearner: - role = placement.Learner - case ast.PlacementRoleVoter: - role = placement.Voter - default: - err = errors.Errorf("ROLE is not specified") - } - if err != nil { - break - } - - if spec.Tp == ast.PlacementAlter || spec.Tp == ast.PlacementDrop { - origLen := len(bundle.Rules) - newRules := bundle.Rules[:0] - for _, r := range bundle.Rules { - if r.Role != role { - newRules = append(newRules, r) - } - } - bundle.Rules = newRules - - // alter == drop + add new rules - if spec.Tp == ast.PlacementDrop { - // error if no rules will be dropped - if len(bundle.Rules) == origLen { - err = errors.Errorf("no rule of role '%s' to drop", role) - break - } - continue - } - } - - var newRules []*placement.Rule - newRules, err = buildPlacementSpecReplicasAndConstraint(spec.Replicas, spec.Constraints) - if err != nil { - break - } - for _, r := range newRules { - r.Role = role - bundle.Rules = append(bundle.Rules, r) - } - } - - if err != nil { - var sb strings.Builder - sb.Reset() - - restoreCtx := format.NewRestoreCtx(format.RestoreStringSingleQuotes|format.RestoreKeyWordLowercase|format.RestoreNameBackQuotes, &sb) - - if e := spec.Restore(restoreCtx); e != nil { - return nil, ErrInvalidPlacementSpec.GenWithStackByArgs("", err) - } - - return nil, ErrInvalidPlacementSpec.GenWithStackByArgs(sb.String(), err) - } - - return bundle, nil -} - func (d *ddl) AlterTableAlterPartition(ctx sessionctx.Context, ident ast.Ident, spec *ast.AlterTableSpec) (err error) { schema, tb, err := d.getSchemaAndTableByIdent(ctx, ident) if err != nil { @@ -6077,61 +5929,29 @@ func (d *ddl) AlterTableAlterPartition(ctx sessionctx.Context, ident ast.Ident, return errors.Trace(err) } - oldBundle := infoschema.GetBundle(d.infoCache.GetLatest(), []int64{partitionID, meta.ID, schema.ID}) + bundle := infoschema.GetBundle(d.infoCache.GetLatest(), []int64{partitionID, meta.ID, schema.ID}) - oldBundle.ID = placement.GroupID(partitionID) + bundle.ID = placement.GroupID(partitionID) - bundle, err := buildPlacementSpecs(oldBundle, spec.PlacementSpecs) + err = bundle.ApplyPlacementSpec(spec.PlacementSpecs) if err != nil { - return errors.Trace(err) - } + var sb strings.Builder + sb.Reset() - extraCnt := map[placement.PeerRoleType]int{} - startKey := hex.EncodeToString(codec.EncodeBytes(nil, tablecodec.GenTableRecordPrefix(partitionID))) - endKey := hex.EncodeToString(codec.EncodeBytes(nil, tablecodec.GenTableRecordPrefix(partitionID+1))) - newRules := bundle.Rules[:0] - for i, rule := range bundle.Rules { - // merge all empty constraints - if len(rule.Constraints) == 0 { - extraCnt[rule.Role] += rule.Count - continue - } - // refer to tidb#22065. - // add -engine=tiflash to every rule to avoid schedules to tiflash instances. - // placement rules in SQL is not compatible with `set tiflash replica` yet - if err := rule.Constraints.Add(placement.Constraint{ - Op: placement.NotIn, - Key: placement.EngineLabelKey, - Values: []string{placement.EngineLabelTiFlash}, - }); err != nil { - return errors.Trace(err) + restoreCtx := format.NewRestoreCtx(format.RestoreStringSingleQuotes|format.RestoreKeyWordLowercase|format.RestoreNameBackQuotes, &sb) + + if e := spec.Restore(restoreCtx); e != nil { + return ErrInvalidPlacementSpec.GenWithStackByArgs("", err) } - rule.GroupID = bundle.ID - rule.ID = strconv.Itoa(i) - rule.StartKeyHex = startKey - rule.EndKeyHex = endKey - newRules = append(newRules, rule) + return ErrInvalidPlacementSpec.GenWithStackByArgs(sb.String(), err) } - for role, cnt := range extraCnt { - if cnt <= 0 { - continue - } - // refer to tidb#22065. - newRules = append(newRules, &placement.Rule{ - GroupID: bundle.ID, - ID: string(role), - Role: role, - Count: cnt, - StartKeyHex: startKey, - EndKeyHex: endKey, - Constraints: []placement.Constraint{{ - Op: placement.NotIn, - Key: placement.EngineLabelKey, - Values: []string{placement.EngineLabelTiFlash}, - }}, - }) + + err = bundle.Tidy() + if err != nil { + return errors.Trace(err) } - bundle.Rules = newRules + bundle.Reset(partitionID) + if len(bundle.Rules) == 0 { bundle.Index = 0 bundle.Override = false diff --git a/ddl/partition.go b/ddl/partition.go index 4e55ec1779e21..566d98505d948 100644 --- a/ddl/partition.go +++ b/ddl/partition.go @@ -915,7 +915,7 @@ func dropRuleBundles(d *ddlCtx, physicalTableIDs []int64) error { for _, ID := range physicalTableIDs { oldBundle, ok := d.infoCache.GetLatest().BundleByName(placement.GroupID(ID)) if ok && !oldBundle.IsEmpty() { - bundles = append(bundles, placement.BuildPlacementDropBundle(ID)) + bundles = append(bundles, placement.NewBundle(ID)) } } err := infosync.PutRuleBundles(context.TODO(), bundles) @@ -1097,8 +1097,8 @@ func onTruncateTablePartition(d *ddlCtx, t *meta.Meta, job *model.Job) (int64, e for i, oldID := range oldIDs { oldBundle, ok := d.infoCache.GetLatest().BundleByName(placement.GroupID(oldID)) if ok && !oldBundle.IsEmpty() { - bundles = append(bundles, placement.BuildPlacementDropBundle(oldID)) - bundles = append(bundles, placement.BuildPlacementCopyBundle(oldBundle, newPartitions[i].ID)) + bundles = append(bundles, placement.NewBundle(oldID)) + bundles = append(bundles, oldBundle.Clone().Reset(newPartitions[i].ID)) } } @@ -1300,14 +1300,14 @@ func (w *worker) onExchangeTablePartition(d *ddlCtx, t *meta.Meta, job *model.Jo ntBundle, ntOK := d.infoCache.GetLatest().BundleByName(placement.GroupID(nt.ID)) ntOK = ntOK && !ntBundle.IsEmpty() if ptOK && ntOK { - bundles = append(bundles, placement.BuildPlacementCopyBundle(ptBundle, nt.ID)) - bundles = append(bundles, placement.BuildPlacementCopyBundle(ntBundle, partDef.ID)) + bundles = append(bundles, ptBundle.Clone().Reset(nt.ID)) + bundles = append(bundles, ntBundle.Clone().Reset(partDef.ID)) } else if ptOK { - bundles = append(bundles, placement.BuildPlacementDropBundle(partDef.ID)) - bundles = append(bundles, placement.BuildPlacementCopyBundle(ptBundle, nt.ID)) + bundles = append(bundles, placement.NewBundle(partDef.ID)) + bundles = append(bundles, ptBundle.Clone().Reset(nt.ID)) } else if ntOK { - bundles = append(bundles, placement.BuildPlacementDropBundle(nt.ID)) - bundles = append(bundles, placement.BuildPlacementCopyBundle(ntBundle, partDef.ID)) + bundles = append(bundles, placement.NewBundle(nt.ID)) + bundles = append(bundles, ntBundle.Clone().Reset(partDef.ID)) } err = infosync.PutRuleBundles(context.TODO(), bundles) if err != nil { diff --git a/ddl/placement/bundle.go b/ddl/placement/bundle.go new file mode 100644 index 0000000000000..1bb7539d79ad0 --- /dev/null +++ b/ddl/placement/bundle.go @@ -0,0 +1,203 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package placement + +import ( + "encoding/hex" + "encoding/json" + "errors" + "fmt" + "strconv" + + "github.com/pingcap/failpoint" + "github.com/pingcap/parser/ast" + "github.com/pingcap/tidb/tablecodec" + "github.com/pingcap/tidb/util/codec" +) + +// Refer to https://github.com/tikv/pd/issues/2701 . +// IMO, it is indeed not bad to have a copy of definition. +// After all, placement rules are communicated using an HTTP API. Loose +// coupling is a good feature. + +// Bundle is a group of all rules and configurations. It is used to support rule cache. +type Bundle struct { + ID string `json:"group_id"` + Index int `json:"group_index"` + Override bool `json:"group_override"` + Rules []*Rule `json:"rules"` +} + +// NewBundle will create a bundle with the provided ID. +// Note that you should never pass negative id. +func NewBundle(id int64) *Bundle { + return &Bundle{ + ID: GroupID(id), + } +} + +// ApplyPlacementSpec will apply actions defined in PlacementSpec to the bundle. +func (b *Bundle) ApplyPlacementSpec(specs []*ast.PlacementSpec) error { + for _, spec := range specs { + var role PeerRoleType + switch spec.Role { + case ast.PlacementRoleFollower: + role = Follower + case ast.PlacementRoleLeader: + if spec.Replicas == 0 { + spec.Replicas = 1 + } + if spec.Replicas > 1 { + return ErrLeaderReplicasMustOne + } + role = Leader + case ast.PlacementRoleLearner: + role = Learner + case ast.PlacementRoleVoter: + role = Voter + default: + return ErrMissingRoleField + } + + if spec.Tp == ast.PlacementAlter || spec.Tp == ast.PlacementDrop { + origLen := len(b.Rules) + newRules := b.Rules[:0] + for _, r := range b.Rules { + if r.Role != role { + newRules = append(newRules, r) + } + } + b.Rules = newRules + + // alter == drop + add new rules + if spec.Tp == ast.PlacementDrop { + // error if no rules will be dropped + if len(b.Rules) == origLen { + return fmt.Errorf("%w: %s", ErrNoRulesToDrop, role) + } + continue + } + } + + var newRules []*Rule + newRules, err := NewRules(spec.Replicas, spec.Constraints) + if err != nil { + return err + } + for _, r := range newRules { + r.Role = role + b.Rules = append(b.Rules, r) + } + } + + return nil +} + +// String implements fmt.Stringer. +func (b *Bundle) String() string { + t, err := json.Marshal(b) + failpoint.Inject("MockMarshalFailure", func(val failpoint.Value) { + if _, ok := val.(bool); ok { + err = errors.New("test") + } + }) + if err != nil { + return "" + } + return string(t) +} + +// Tidy will post optimize Rules, trying to generate rules that suits PD. +func (b *Bundle) Tidy() error { + extraCnt := map[PeerRoleType]int{} + newRules := b.Rules[:0] + for i, rule := range b.Rules { + // useless Rule + if rule.Count <= 0 { + continue + } + // merge all empty constraints + if len(rule.Constraints) == 0 { + extraCnt[rule.Role] += rule.Count + continue + } + // refer to tidb#22065. + // add -engine=tiflash to every rule to avoid schedules to tiflash instances. + // placement rules in SQL is not compatible with `set tiflash replica` yet + err := rule.Constraints.Add(Constraint{ + Op: NotIn, + Key: EngineLabelKey, + Values: []string{EngineLabelTiFlash}, + }) + if err != nil { + return err + } + // Constraints.Add() will automatically avoid duplication + // if -engine=tiflash is added and there is only one constraint + // then it must be -engine=tiflash + // it is seen as an empty constraint, so merge it + if len(rule.Constraints) == 1 { + extraCnt[rule.Role] += rule.Count + continue + } + rule.ID = strconv.Itoa(i) + newRules = append(newRules, rule) + } + for role, cnt := range extraCnt { + // add -engine=tiflash, refer to tidb#22065. + newRules = append(newRules, &Rule{ + ID: string(role), + Role: role, + Count: cnt, + Constraints: []Constraint{{ + Op: NotIn, + Key: EngineLabelKey, + Values: []string{EngineLabelTiFlash}, + }}, + }) + } + b.Rules = newRules + return nil +} + +// Reset resets the bundle ID and keyrange of all rules. +func (b *Bundle) Reset(newID int64) *Bundle { + b.ID = GroupID(newID) + startKey := hex.EncodeToString(codec.EncodeBytes(nil, tablecodec.GenTableRecordPrefix(newID))) + endKey := hex.EncodeToString(codec.EncodeBytes(nil, tablecodec.GenTableRecordPrefix(newID+1))) + for _, rule := range b.Rules { + rule.GroupID = b.ID + rule.StartKeyHex = startKey + rule.EndKeyHex = endKey + } + return b +} + +// Clone is used to duplicate a bundle. +func (b *Bundle) Clone() *Bundle { + newBundle := &Bundle{} + *newBundle = *b + if len(b.Rules) > 0 { + newBundle.Rules = make([]*Rule, 0, len(b.Rules)) + for i := range b.Rules { + newBundle.Rules = append(newBundle.Rules, b.Rules[i].Clone()) + } + } + return newBundle +} + +// IsEmpty is used to check if a bundle is empty. +func (b *Bundle) IsEmpty() bool { + return len(b.Rules) == 0 && b.Index == 0 && !b.Override +} diff --git a/ddl/placement/bundle_test.go b/ddl/placement/bundle_test.go new file mode 100644 index 0000000000000..c63fbae74e670 --- /dev/null +++ b/ddl/placement/bundle_test.go @@ -0,0 +1,378 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package placement + +import ( + "encoding/hex" + "errors" + + . "github.com/pingcap/check" + "github.com/pingcap/failpoint" + "github.com/pingcap/parser/ast" + "github.com/pingcap/tidb/tablecodec" + "github.com/pingcap/tidb/util/codec" +) + +var _ = Suite(&testBundleSuite{}) + +type testBundleSuite struct{} + +func (s *testBundleSuite) TestEmpty(c *C) { + bundle := &Bundle{ID: GroupID(1)} + c.Assert(bundle.IsEmpty(), IsTrue) + + bundle = &Bundle{ID: GroupID(1), Index: 1} + c.Assert(bundle.IsEmpty(), IsFalse) + + bundle = &Bundle{ID: GroupID(1), Override: true} + c.Assert(bundle.IsEmpty(), IsFalse) + + bundle = &Bundle{ID: GroupID(1), Rules: []*Rule{{ID: "434"}}} + c.Assert(bundle.IsEmpty(), IsFalse) + + bundle = &Bundle{ID: GroupID(1), Index: 1, Override: true} + c.Assert(bundle.IsEmpty(), IsFalse) +} + +func (s *testBundleSuite) TestClone(c *C) { + bundle := &Bundle{ID: GroupID(1), Rules: []*Rule{{ID: "434"}}} + + newBundle := bundle.Clone() + newBundle.ID = GroupID(2) + newBundle.Rules[0] = &Rule{ID: "121"} + + c.Assert(bundle, DeepEquals, &Bundle{ID: GroupID(1), Rules: []*Rule{{ID: "434"}}}) + c.Assert(newBundle, DeepEquals, &Bundle{ID: GroupID(2), Rules: []*Rule{{ID: "121"}}}) +} + +func (s *testBundleSuite) TestApplyPlacmentSpec(c *C) { + type TestCase struct { + name string + input []*ast.PlacementSpec + output []*Rule + err error + } + var tests []TestCase + + tests = append(tests, TestCase{ + name: "empty", + input: []*ast.PlacementSpec{}, + output: []*Rule{}, + }) + + rules, err := NewRules(3, `["+zone=sh", "+zone=sh"]`) + c.Assert(err, IsNil) + c.Assert(rules, HasLen, 1) + rules[0].Role = Voter + tests = append(tests, TestCase{ + name: "add voter array", + input: []*ast.PlacementSpec{{ + Role: ast.PlacementRoleVoter, + Tp: ast.PlacementAdd, + Replicas: 3, + Constraints: `["+zone=sh", "+zone=sh"]`, + }}, + output: rules, + }) + + rules, err = NewRules(3, `["+zone=sh", "+zone=sh"]`) + c.Assert(err, IsNil) + c.Assert(rules, HasLen, 1) + rules[0].Role = Learner + tests = append(tests, TestCase{ + name: "add learner array", + input: []*ast.PlacementSpec{{ + Role: ast.PlacementRoleLearner, + Tp: ast.PlacementAdd, + Replicas: 3, + Constraints: `["+zone=sh", "+zone=sh"]`, + }}, + output: rules, + }) + + rules, err = NewRules(3, `["+zone=sh", "+zone=sh"]`) + c.Assert(err, IsNil) + c.Assert(rules, HasLen, 1) + rules[0].Role = Follower + tests = append(tests, TestCase{ + name: "add follower array", + input: []*ast.PlacementSpec{{ + Role: ast.PlacementRoleFollower, + Tp: ast.PlacementAdd, + Replicas: 3, + Constraints: `["+zone=sh", "+zone=sh"]`, + }}, + output: rules, + }) + + tests = append(tests, TestCase{ + name: "add invalid constraints", + input: []*ast.PlacementSpec{{ + Role: ast.PlacementRoleVoter, + Tp: ast.PlacementAdd, + Replicas: 3, + Constraints: "ne", + }}, + err: ErrInvalidConstraintsFormat, + }) + + tests = append(tests, TestCase{ + name: "add empty role", + input: []*ast.PlacementSpec{{ + Tp: ast.PlacementAdd, + Replicas: 3, + Constraints: "", + }}, + err: ErrMissingRoleField, + }) + + tests = append(tests, TestCase{ + name: "add multiple leaders", + input: []*ast.PlacementSpec{{ + Role: ast.PlacementRoleLeader, + Tp: ast.PlacementAdd, + Replicas: 3, + Constraints: "", + }}, + err: ErrLeaderReplicasMustOne, + }) + + rules, err = NewRules(1, "") + c.Assert(err, IsNil) + c.Assert(rules, HasLen, 1) + rules[0].Role = Leader + tests = append(tests, TestCase{ + name: "omit leader field", + input: []*ast.PlacementSpec{{ + Role: ast.PlacementRoleLeader, + Tp: ast.PlacementAdd, + Constraints: "", + }}, + output: rules, + }) + + rules, err = NewRules(3, `["-zone=sh","+zone=bj"]`) + c.Assert(err, IsNil) + c.Assert(rules, HasLen, 1) + rules[0].Role = Follower + tests = append(tests, TestCase{ + name: "drop", + input: []*ast.PlacementSpec{ + { + Role: ast.PlacementRoleFollower, + Tp: ast.PlacementAdd, + Replicas: 3, + Constraints: `["- zone=sh", "+zone = bj"]`, + }, + { + Role: ast.PlacementRoleVoter, + Tp: ast.PlacementAdd, + Replicas: 3, + Constraints: `["+ zone=sh", "-zone = bj"]`, + }, + { + Role: ast.PlacementRoleVoter, + Tp: ast.PlacementDrop, + }, + }, + output: rules, + }) + + tests = append(tests, TestCase{ + name: "drop unexisted", + input: []*ast.PlacementSpec{{ + Role: ast.PlacementRoleLeader, + Tp: ast.PlacementDrop, + Constraints: "", + }}, + err: ErrNoRulesToDrop, + }) + + rules1, err := NewRules(3, `["-zone=sh","+zone=bj"]`) + c.Assert(err, IsNil) + c.Assert(rules1, HasLen, 1) + rules1[0].Role = Follower + rules2, err := NewRules(3, `["+zone=sh","-zone=bj"]`) + c.Assert(err, IsNil) + c.Assert(rules2, HasLen, 1) + rules2[0].Role = Voter + tests = append(tests, TestCase{ + name: "alter", + input: []*ast.PlacementSpec{ + { + Role: ast.PlacementRoleFollower, + Tp: ast.PlacementAdd, + Replicas: 3, + Constraints: `["- zone=sh", "+zone = bj"]`, + }, + { + Role: ast.PlacementRoleVoter, + Tp: ast.PlacementAdd, + Replicas: 3, + Constraints: `["- zone=sh", "+zone = bj"]`, + }, + { + Role: ast.PlacementRoleVoter, + Tp: ast.PlacementAlter, + Replicas: 3, + Constraints: `["+ zone=sh", "-zone = bj"]`, + }, + }, + output: append(rules1, rules2...), + }) + + for _, t := range tests { + comment := Commentf("%s", t.name) + bundle := &Bundle{} + err := bundle.ApplyPlacementSpec(t.input) + if t.err == nil { + c.Assert(err, IsNil) + matchRules(t.output, bundle.Rules, comment.CheckCommentString(), c) + } else { + c.Assert(errors.Is(err, t.err), IsTrue, comment) + } + } +} + +func (s *testBundleSuite) TestString(c *C) { + bundle := &Bundle{ + ID: GroupID(1), + } + + rules1, err := NewRules(3, `["+zone=sh", "+zone=sh"]`) + c.Assert(err, IsNil) + rules2, err := NewRules(4, `["-zone=sh", "+zone=bj"]`) + c.Assert(err, IsNil) + bundle.Rules = append(rules1, rules2...) + + c.Assert(bundle.String(), Equals, `{"group_id":"TiDB_DDL_1","group_index":0,"group_override":false,"rules":[{"group_id":"","id":"","start_key":"","end_key":"","role":"","count":3,"label_constraints":[{"key":"zone","op":"in","values":["sh"]}]},{"group_id":"","id":"","start_key":"","end_key":"","role":"","count":4,"label_constraints":[{"key":"zone","op":"notIn","values":["sh"]},{"key":"zone","op":"in","values":["bj"]}]}]}`) + + c.Assert(failpoint.Enable("github.com/pingcap/tidb/ddl/placement/MockMarshalFailure", `return(true)`), IsNil) + defer func() { + c.Assert(failpoint.Disable("github.com/pingcap/tidb/ddl/placement/MockMarshalFailure"), IsNil) + }() + c.Assert(bundle.String(), Equals, "") +} + +func (s *testBundleSuite) TestNew(c *C) { + c.Assert(NewBundle(3), DeepEquals, &Bundle{ID: GroupID(3)}) + c.Assert(NewBundle(-1), DeepEquals, &Bundle{ID: GroupID(-1)}) +} + +func (s *testBundleSuite) TestReset(c *C) { + bundle := &Bundle{ + ID: GroupID(1), + } + + rules, err := NewRules(3, `["+zone=sh", "+zone=sh"]`) + c.Assert(err, IsNil) + bundle.Rules = rules + + bundle.Reset(3) + c.Assert(bundle.ID, Equals, GroupID(3)) + c.Assert(bundle.Rules, HasLen, 1) + c.Assert(bundle.Rules[0].GroupID, Equals, bundle.ID) + + startKey := hex.EncodeToString(codec.EncodeBytes(nil, tablecodec.GenTableRecordPrefix(3))) + c.Assert(bundle.Rules[0].StartKeyHex, Equals, startKey) + + endKey := hex.EncodeToString(codec.EncodeBytes(nil, tablecodec.GenTableRecordPrefix(4))) + c.Assert(bundle.Rules[0].EndKeyHex, Equals, endKey) +} + +func (s *testBundleSuite) TestTidy(c *C) { + bundle := &Bundle{ + ID: GroupID(1), + } + + rules0, err := NewRules(1, `["+zone=sh", "+zone=sh"]`) + c.Assert(err, IsNil) + c.Assert(rules0, HasLen, 1) + rules0[0].Count = 0 + rules1, err := NewRules(4, `["-zone=sh", "+zone=bj"]`) + c.Assert(err, IsNil) + c.Assert(rules1, HasLen, 1) + rules2, err := NewRules(4, `["-zone=sh", "+zone=bj"]`) + c.Assert(err, IsNil) + bundle.Rules = append(bundle.Rules, rules0...) + bundle.Rules = append(bundle.Rules, rules1...) + bundle.Rules = append(bundle.Rules, rules2...) + + err = bundle.Tidy() + c.Assert(err, IsNil) + c.Assert(bundle.Rules, HasLen, 2) + c.Assert(bundle.Rules[0].ID, Equals, "1") + c.Assert(bundle.Rules[0].Constraints, HasLen, 3) + c.Assert(bundle.Rules[0].Constraints[2], DeepEquals, Constraint{ + Op: NotIn, + Key: EngineLabelKey, + Values: []string{EngineLabelTiFlash}, + }) + c.Assert(bundle.Rules[1].ID, Equals, "2") + + // merge + rules3, err := NewRules(4, "") + c.Assert(err, IsNil) + c.Assert(rules3, HasLen, 1) + rules3[0].Role = Follower + + rules4, err := NewRules(5, "") + c.Assert(err, IsNil) + c.Assert(rules4, HasLen, 1) + rules4[0].Role = Follower + + rules0[0].Role = Voter + bundle.Rules = append(bundle.Rules, rules0...) + bundle.Rules = append(bundle.Rules, rules3...) + bundle.Rules = append(bundle.Rules, rules4...) + + chkfunc := func() { + c.Assert(err, IsNil) + c.Assert(bundle.Rules, HasLen, 3) + c.Assert(bundle.Rules[0].ID, Equals, "0") + c.Assert(bundle.Rules[1].ID, Equals, "1") + c.Assert(bundle.Rules[2].ID, Equals, "follower") + c.Assert(bundle.Rules[2].Count, Equals, 9) + c.Assert(bundle.Rules[2].Constraints, DeepEquals, Constraints{ + { + Op: NotIn, + Key: EngineLabelKey, + Values: []string{EngineLabelTiFlash}, + }, + }) + } + err = bundle.Tidy() + chkfunc() + + // tidy again + // it should be stable + err = bundle.Tidy() + chkfunc() + + // tidy again + // it should be stable + bundle2 := bundle.Clone() + err = bundle2.Tidy() + c.Assert(err, IsNil) + c.Assert(bundle2, DeepEquals, bundle) + + bundle.Rules[2].Constraints = append(bundle.Rules[2].Constraints, Constraint{ + Op: In, + Key: EngineLabelKey, + Values: []string{EngineLabelTiFlash}, + }) + c.Log(bundle.Rules[2]) + err = bundle.Tidy() + c.Assert(errors.Is(err, ErrConflictingConstraints), IsTrue) +} diff --git a/ddl/placement/errors.go b/ddl/placement/errors.go index 95fce4591c961..26b60621d0f44 100644 --- a/ddl/placement/errors.go +++ b/ddl/placement/errors.go @@ -30,4 +30,14 @@ var ( ErrInvalidConstraintsFormat = errors.New("invalid label constraints format") // ErrInvalidConstraintsRelicas is from rule.go. ErrInvalidConstraintsRelicas = errors.New("label constraints with invalid REPLICAS") + // ErrInvalidBundleID is from bundle.go. + ErrInvalidBundleID = errors.New("invalid bundle ID") + // ErrInvalidBundleIDFormat is from bundle.go. + ErrInvalidBundleIDFormat = errors.New("invalid bundle ID format") + // ErrLeaderReplicasMustOne is from bundle.go. + ErrLeaderReplicasMustOne = errors.New("REPLICAS must be 1 if ROLE=leader") + // ErrMissingRoleField is from bundle.go. + ErrMissingRoleField = errors.New("the ROLE field is not specified") + // ErrNoRulesToDrop is from bundle.go. + ErrNoRulesToDrop = errors.New("no rule of such role to drop") ) diff --git a/ddl/placement/types.go b/ddl/placement/types.go deleted file mode 100644 index 72093a2c19c78..0000000000000 --- a/ddl/placement/types.go +++ /dev/null @@ -1,91 +0,0 @@ -// Copyright 2020 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package placement - -import ( - "encoding/json" -) - -// Refer to https://github.com/tikv/pd/issues/2701 . -// IMO, it is indeed not bad to have a copy of definition. -// After all, placement rules are communicated using an HTTP API. Loose -// coupling is a good feature. - -// Bundle is a group of all rules and configurations. It is used to support rule cache. -type Bundle struct { - ID string `json:"group_id"` - Index int `json:"group_index"` - Override bool `json:"group_override"` - Rules []*Rule `json:"rules"` -} - -func (b *Bundle) String() string { - t, err := json.Marshal(b) - if err != nil { - return "" - } - return string(t) -} - -// Clone is used to duplicate a bundle. -func (b *Bundle) Clone() *Bundle { - newBundle := &Bundle{} - *newBundle = *b - if len(b.Rules) > 0 { - newBundle.Rules = make([]*Rule, 0, len(b.Rules)) - for i := range b.Rules { - newBundle.Rules = append(newBundle.Rules, b.Rules[i].Clone()) - } - } - return newBundle -} - -// IsEmpty is used to check if a bundle is empty. -func (b *Bundle) IsEmpty() bool { - return len(b.Rules) == 0 && b.Index == 0 && !b.Override -} - -// RuleOpType indicates the operation type. -type RuleOpType string - -const ( - // RuleOpAdd a placement rule, only need to specify the field *Rule. - RuleOpAdd RuleOpType = "add" - // RuleOpDel a placement rule, only need to specify the field `GroupID`, `ID`, `MatchID`. - RuleOpDel RuleOpType = "del" -) - -// RuleOp is for batching placement rule actions. -type RuleOp struct { - *Rule - Action RuleOpType `json:"action"` - DeleteByIDPrefix bool `json:"delete_by_id_prefix"` -} - -// Clone is used to clone a RuleOp that is safe to modify, without affecting the old RuleOp. -func (op *RuleOp) Clone() *RuleOp { - newOp := &RuleOp{} - *newOp = *op - newOp.Rule = &Rule{} - *newOp.Rule = *op.Rule - return newOp -} - -func (op *RuleOp) String() string { - b, err := json.Marshal(op) - if err != nil { - return "" - } - return string(b) -} diff --git a/ddl/placement/types_test.go b/ddl/placement/types_test.go deleted file mode 100644 index 93ed1a5a80f43..0000000000000 --- a/ddl/placement/types_test.go +++ /dev/null @@ -1,50 +0,0 @@ -// Copyright 2020 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package placement - -import ( - . "github.com/pingcap/check" -) - -var _ = Suite(&testBundleSuite{}) - -type testBundleSuite struct{} - -func (t *testBundleSuite) TestEmpty(c *C) { - bundle := &Bundle{ID: GroupID(1)} - c.Assert(bundle.IsEmpty(), IsTrue) - - bundle = &Bundle{ID: GroupID(1), Index: 1} - c.Assert(bundle.IsEmpty(), IsFalse) - - bundle = &Bundle{ID: GroupID(1), Override: true} - c.Assert(bundle.IsEmpty(), IsFalse) - - bundle = &Bundle{ID: GroupID(1), Rules: []*Rule{{ID: "434"}}} - c.Assert(bundle.IsEmpty(), IsFalse) - - bundle = &Bundle{ID: GroupID(1), Index: 1, Override: true} - c.Assert(bundle.IsEmpty(), IsFalse) -} - -func (t *testBundleSuite) TestClone(c *C) { - bundle := &Bundle{ID: GroupID(1), Rules: []*Rule{{ID: "434"}}} - - newBundle := bundle.Clone() - newBundle.ID = GroupID(2) - newBundle.Rules[0] = &Rule{ID: "121"} - - c.Assert(bundle, DeepEquals, &Bundle{ID: GroupID(1), Rules: []*Rule{{ID: "434"}}}) - c.Assert(newBundle, DeepEquals, &Bundle{ID: GroupID(2), Rules: []*Rule{{ID: "121"}}}) -} diff --git a/ddl/placement_rule_test.go b/ddl/placement_rule_test.go deleted file mode 100644 index a9a916cb5a199..0000000000000 --- a/ddl/placement_rule_test.go +++ /dev/null @@ -1,409 +0,0 @@ -// Copyright 2020 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package ddl - -import ( - "encoding/hex" - "encoding/json" - - . "github.com/pingcap/check" - "github.com/pingcap/parser/ast" - "github.com/pingcap/tidb/ddl/placement" - "github.com/pingcap/tidb/tablecodec" - "github.com/pingcap/tidb/util/codec" -) - -var _ = Suite(&testPlacementSuite{}) - -type testPlacementSuite struct { -} - -func (s *testPlacementSuite) TestPlacementBuild(c *C) { - tests := []struct { - input []*ast.PlacementSpec - bundle *placement.Bundle - output []*placement.Rule - err string - }{ - { - input: []*ast.PlacementSpec{}, - output: []*placement.Rule{}, - }, - - { - input: []*ast.PlacementSpec{{ - Role: ast.PlacementRoleVoter, - Tp: ast.PlacementAdd, - Replicas: 3, - Constraints: `["+zone=sh", "+zone=sh"]`, - }}, - output: []*placement.Rule{ - { - Role: placement.Voter, - Count: 3, - Constraints: []placement.Constraint{ - {Key: "zone", Op: "in", Values: []string{"sh"}}, - }, - }, - }, - }, - - { - input: []*ast.PlacementSpec{{ - Role: ast.PlacementRoleVoter, - Tp: ast.PlacementAdd, - Replicas: 3, - Constraints: "", - }}, - output: []*placement.Rule{{ - Role: placement.Voter, - Count: 3, - Constraints: []placement.Constraint{}, - }}, - }, - - { - input: []*ast.PlacementSpec{{ - Role: ast.PlacementRoleVoter, - Tp: ast.PlacementAdd, - Constraints: `{"+zone=sh,+zone=sh": 2, "+zone=sh": 1}`, - }}, - output: []*placement.Rule{ - { - Role: placement.Voter, - Count: 1, - Constraints: []placement.Constraint{ - {Key: "zone", Op: "in", Values: []string{"sh"}}, - }, - }, - { - Role: placement.Voter, - Count: 2, - Constraints: []placement.Constraint{ - {Key: "zone", Op: "in", Values: []string{"sh"}}, - }, - }, - }, - }, - - { - input: []*ast.PlacementSpec{{ - Role: ast.PlacementRoleVoter, - Tp: ast.PlacementAdd, - Replicas: 3, - Constraints: `["- zone=sh", "-zone = bj"]`, - }}, - output: []*placement.Rule{ - { - Role: placement.Voter, - Count: 3, - Constraints: []placement.Constraint{ - {Key: "zone", Op: "notIn", Values: []string{"sh"}}, - {Key: "zone", Op: "notIn", Values: []string{"bj"}}, - }, - }, - }, - }, - - { - input: []*ast.PlacementSpec{{ - Role: ast.PlacementRoleVoter, - Tp: ast.PlacementAdd, - Replicas: 3, - Constraints: `["+ zone=sh", "-zone = bj"]`, - }}, - output: []*placement.Rule{ - { - Role: placement.Voter, - Count: 3, - Constraints: []placement.Constraint{ - {Key: "zone", Op: "in", Values: []string{"sh"}}, - {Key: "zone", Op: "notIn", Values: []string{"bj"}}, - }, - }, - }, - }, - - { - input: []*ast.PlacementSpec{ - { - Role: ast.PlacementRoleVoter, - Tp: ast.PlacementAdd, - Replicas: 3, - Constraints: `["+ zone=sh", "-zone = bj"]`, - }, - { - Role: ast.PlacementRoleFollower, - Tp: ast.PlacementAdd, - Replicas: 2, - Constraints: `["- zone=sh", "+zone = bj"]`, - }, - }, - output: []*placement.Rule{ - { - Role: placement.Voter, - Count: 3, - Constraints: []placement.Constraint{ - {Key: "zone", Op: "in", Values: []string{"sh"}}, - {Key: "zone", Op: "notIn", Values: []string{"bj"}}, - }, - }, - { - Role: placement.Follower, - Count: 2, - Constraints: []placement.Constraint{ - {Key: "zone", Op: "notIn", Values: []string{"sh"}}, - {Key: "zone", Op: "in", Values: []string{"bj"}}, - }, - }, - }, - }, - - { - input: []*ast.PlacementSpec{ - { - Role: ast.PlacementRoleVoter, - Tp: ast.PlacementAdd, - Replicas: 3, - Constraints: `["+ zone=sh", "-zone = bj"]`, - }, - { - Role: ast.PlacementRoleVoter, - Tp: ast.PlacementAlter, - Replicas: 2, - Constraints: `["- zone=sh", "+zone = bj"]`, - }, - }, - output: []*placement.Rule{ - { - Role: placement.Voter, - Count: 2, - Constraints: []placement.Constraint{ - {Key: "zone", Op: "notIn", Values: []string{"sh"}}, - {Key: "zone", Op: "in", Values: []string{"bj"}}, - }, - }, - }, - }, - - { - input: []*ast.PlacementSpec{ - { - Role: ast.PlacementRoleVoter, - Tp: ast.PlacementAdd, - Replicas: 3, - Constraints: `["+ zone=sh", "-zone = bj"]`, - }, - { - Role: ast.PlacementRoleVoter, - Tp: ast.PlacementAlter, - Replicas: 3, - Constraints: `{"- zone=sh":1, "+zone = bj":1}`, - }, - }, - output: []*placement.Rule{ - { - Role: placement.Voter, - Count: 1, - Constraints: []placement.Constraint{{Key: "zone", Op: "notIn", Values: []string{"sh"}}}, - }, - { - Role: placement.Voter, - Count: 1, - Constraints: []placement.Constraint{{Key: "zone", Op: "in", Values: []string{"bj"}}}, - }, - { - Role: placement.Voter, - Count: 1, - }, - }, - }, - - { - input: []*ast.PlacementSpec{ - { - Role: ast.PlacementRoleVoter, - Tp: ast.PlacementAdd, - Replicas: 3, - Constraints: `["+ zone=sh", "-zone = bj"]`, - }, - { - Role: ast.PlacementRoleVoter, - Tp: ast.PlacementDrop, - }, - }, - output: []*placement.Rule{}, - }, - - { - input: []*ast.PlacementSpec{ - { - Role: ast.PlacementRoleLearner, - Tp: ast.PlacementDrop, - }, - }, - bundle: &placement.Bundle{Rules: []*placement.Rule{ - {Role: placement.Learner}, - {Role: placement.Voter}, - {Role: placement.Learner}, - {Role: placement.Voter}, - }}, - output: []*placement.Rule{ - {Role: placement.Voter}, - {Role: placement.Voter}, - }, - }, - - { - input: []*ast.PlacementSpec{ - { - Role: ast.PlacementRoleLearner, - Tp: ast.PlacementDrop, - }, - { - Role: ast.PlacementRoleVoter, - Tp: ast.PlacementDrop, - }, - }, - bundle: &placement.Bundle{Rules: []*placement.Rule{ - {Role: placement.Learner}, - {Role: placement.Voter}, - {Role: placement.Learner}, - {Role: placement.Voter}, - }}, - output: []*placement.Rule{}, - }, - - { - input: []*ast.PlacementSpec{ - { - Role: ast.PlacementRoleLearner, - Tp: ast.PlacementDrop, - }, - }, - err: ".*no rule of role 'learner' to drop.*", - }, - - { - input: []*ast.PlacementSpec{{ - Role: ast.PlacementRoleVoter, - Tp: ast.PlacementAdd, - Replicas: 3, - Constraints: `['+ zone=sh', '-zone = bj']`, - }}, - output: []*placement.Rule{ - { - Role: placement.Voter, - Count: 3, - Constraints: []placement.Constraint{ - {Key: "zone", Op: "in", Values: []string{"sh"}}, - {Key: "zone", Op: "notIn", Values: []string{"bj"}}, - }, - }, - }, - }, - } - for i, t := range tests { - var bundle *placement.Bundle - if t.bundle == nil { - bundle = &placement.Bundle{Rules: []*placement.Rule{}} - } else { - bundle = t.bundle - } - out, err := buildPlacementSpecs(bundle, t.input) - if t.err == "" { - c.Assert(err, IsNil) - expected, err := json.Marshal(t.output) - c.Assert(err, IsNil) - got, err := json.Marshal(out.Rules) - c.Assert(err, IsNil) - comment := Commentf("%d test\nexpected %s\nbut got %s", i, expected, got) - c.Assert(len(t.output), Equals, len(out.Rules), comment) - for _, r1 := range t.output { - found := false - for _, r2 := range out.Rules { - if ok, _ := DeepEquals.Check([]interface{}{r1, r2}, nil); ok { - found = true - break - } - } - c.Assert(found, IsTrue, comment) - } - } else { - c.Assert(err, ErrorMatches, t.err) - } - } -} - -func (s *testPlacementSuite) TestPlacementBuildDrop(c *C) { - tests := []struct { - input int64 - output *placement.Bundle - }{ - { - input: 2, - output: &placement.Bundle{ID: placement.GroupID(2)}, - }, - { - input: 1, - output: &placement.Bundle{ID: placement.GroupID(1)}, - }, - } - for _, t := range tests { - out := placement.BuildPlacementDropBundle(t.input) - c.Assert(t.output, DeepEquals, out) - } -} - -func (s *testPlacementSuite) TestPlacementBuildTruncate(c *C) { - bundle := &placement.Bundle{ - ID: placement.GroupID(-1), - Rules: []*placement.Rule{{GroupID: placement.GroupID(-1)}}, - } - - tests := []struct { - input int64 - output *placement.Bundle - }{ - { - input: 1, - output: &placement.Bundle{ - ID: placement.GroupID(1), - Rules: []*placement.Rule{{ - GroupID: placement.GroupID(1), - StartKeyHex: hex.EncodeToString(codec.EncodeBytes(nil, tablecodec.GenTableRecordPrefix(1))), - EndKeyHex: hex.EncodeToString(codec.EncodeBytes(nil, tablecodec.GenTableRecordPrefix(2))), - }}, - }, - }, - { - input: 2, - output: &placement.Bundle{ - ID: placement.GroupID(2), - Rules: []*placement.Rule{{ - GroupID: placement.GroupID(2), - StartKeyHex: hex.EncodeToString(codec.EncodeBytes(nil, tablecodec.GenTableRecordPrefix(2))), - EndKeyHex: hex.EncodeToString(codec.EncodeBytes(nil, tablecodec.GenTableRecordPrefix(3))), - }}, - }, - }, - } - for _, t := range tests { - out := placement.BuildPlacementCopyBundle(bundle, t.input) - c.Assert(t.output, DeepEquals, out) - c.Assert(bundle.ID, Equals, placement.GroupID(-1)) - c.Assert(bundle.Rules, HasLen, 1) - c.Assert(bundle.Rules[0].GroupID, Equals, placement.GroupID(-1)) - } -} diff --git a/ddl/placement_sql_test.go b/ddl/placement_sql_test.go index 0396bc9554074..f5ce6c1328d6e 100644 --- a/ddl/placement_sql_test.go +++ b/ddl/placement_sql_test.go @@ -78,6 +78,12 @@ add placement policy replicas=3`) c.Assert(err, IsNil) + _, err = tk.Exec(`alter table t1 alter partition p0 +add placement policy + constraints="{'+zone=sh': 1}" + role=follower`) + c.Assert(err, IsNil) + _, err = tk.Exec(`alter table t1 alter partition p0 add placement policy constraints='{"+ zone = sh ": 1}' @@ -129,7 +135,7 @@ drop placement policy _, err = tk.Exec(`alter table t1 alter partition p0 drop placement policy role=follower`) - c.Assert(err, ErrorMatches, ".*no rule of role 'follower' to drop.*") + c.Assert(err, ErrorMatches, ".*no rule of such role to drop.*") _, err = tk.Exec(`alter table t1 alter partition p0 add placement policy @@ -142,7 +148,7 @@ add placement policy add placement policy constraints='{"+ zone = sh, -zone = bj ": 1}' replicas=3`) - c.Assert(err, ErrorMatches, ".*ROLE is not specified.*") + c.Assert(err, ErrorMatches, ".*the ROLE field is not specified.*") // multiple statements _, err = tk.Exec(`alter table t1 alter partition p0 @@ -202,7 +208,7 @@ drop placement policy role=leader, drop placement policy role=leader`) - c.Assert(err, ErrorMatches, ".*no rule of role 'leader' to drop.*") + c.Assert(err, ErrorMatches, ".*no rule of such role to drop.*") s.dom.InfoSchema().SetBundle(bundle) _, err = tk.Exec(`alter table t1 alter partition p0 @@ -219,14 +225,14 @@ drop placement policy add placement policy role=follower constraints='[]'`) - c.Assert(err, ErrorMatches, ".*array CONSTRAINTS should be with a positive REPLICAS.*") + c.Assert(err, ErrorMatches, ".*label constraints with invalid REPLICAS: should be positive.*") _, err = tk.Exec(`alter table t1 alter partition p0 add placement policy constraints=',,,' role=follower replicas=3`) - c.Assert(err, ErrorMatches, "(?s).*constraint is neither an array of string, nor a string-to-number map.*") + c.Assert(err, ErrorMatches, "(?s).*invalid label constraints format: .* or any yaml compatible representation.*") _, err = tk.Exec(`alter table t1 alter partition p0 add placement policy @@ -239,14 +245,14 @@ add placement policy constraints='[,,,' role=follower replicas=3`) - c.Assert(err, ErrorMatches, "(?s).*constraint is neither an array of string, nor a string-to-number map.*") + c.Assert(err, ErrorMatches, "(?s).*invalid label constraints format: .* or any yaml compatible representation.*") _, err = tk.Exec(`alter table t1 alter partition p0 add placement policy constraints='{,,,' role=follower replicas=3`) - c.Assert(err, ErrorMatches, "(?s).*constraint is neither an array of string, nor a string-to-number map.*") + c.Assert(err, ErrorMatches, "(?s).*invalid label constraints format: .* or any yaml compatible representation.*") _, err = tk.Exec(`alter table t1 alter partition p0 add placement policy @@ -273,7 +279,7 @@ add placement policy constraints='{"+ zone = sh, -zone = bj ": -1}' role=follower replicas=3`) - c.Assert(err, ErrorMatches, ".*count should be positive.*") + c.Assert(err, ErrorMatches, ".*label constraints in map syntax have invalid replicas: count of labels.*") _, err = tk.Exec(`alter table t1 alter partition p0 add placement policy @@ -298,7 +304,7 @@ add placement policy _, err = tk.Exec(`alter table t_part_pk_id alter partition p0 add placement policy constraints='["+host=store1"]' role=leader;`) c.Assert(err, IsNil) _, err = tk.Exec(`alter table t_part_pk_id alter partition p0 add placement policy constraints='["+host=store1"]' role=leader replicas=3;`) - c.Assert(err, ErrorMatches, ".*replicas can only be 1 when the role is leader") + c.Assert(err, ErrorMatches, ".*REPLICAS must be 1 if ROLE=leader.*") tk.MustExec("drop table t_part_pk_id") } diff --git a/ddl/schema.go b/ddl/schema.go index a4b14a49bdbc3..9fc43fd917af7 100644 --- a/ddl/schema.go +++ b/ddl/schema.go @@ -167,7 +167,7 @@ func onDropSchema(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ error) for _, ID := range append(oldIDs, dbInfo.ID) { oldBundle, ok := d.infoCache.GetLatest().BundleByName(placement.GroupID(ID)) if ok && !oldBundle.IsEmpty() { - bundles = append(bundles, placement.BuildPlacementDropBundle(ID)) + bundles = append(bundles, placement.NewBundle(ID)) } } err := infosync.PutRuleBundles(context.TODO(), bundles) diff --git a/ddl/table.go b/ddl/table.go index 6c113fc855b23..8e3843b0b77e3 100644 --- a/ddl/table.go +++ b/ddl/table.go @@ -492,7 +492,7 @@ func onTruncateTable(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ erro bundles := make([]*placement.Bundle, 0, len(oldPartitionIDs)+1) if oldBundle, ok := is.BundleByName(placement.GroupID(tableID)); ok { - bundles = append(bundles, placement.BuildPlacementCopyBundle(oldBundle, newTableID)) + bundles = append(bundles, oldBundle.Clone().Reset(newTableID)) } if pi := tblInfo.GetPartitionInfo(); pi != nil { @@ -504,7 +504,7 @@ func onTruncateTable(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ erro if oldBundle, ok := is.BundleByName(placement.GroupID(oldPartitionIDs[i])); ok && !oldBundle.IsEmpty() { oldIDs = append(oldIDs, oldPartitionIDs[i]) newIDs = append(newIDs, newID) - bundles = append(bundles, placement.BuildPlacementCopyBundle(oldBundle, newID)) + bundles = append(bundles, oldBundle.Clone().Reset(newID)) } } job.CtxVars = []interface{}{oldIDs, newIDs} diff --git a/store/gcworker/gc_worker.go b/store/gcworker/gc_worker.go index 1dbb3cde03880..155c84cc8c303 100644 --- a/store/gcworker/gc_worker.go +++ b/store/gcworker/gc_worker.go @@ -1897,7 +1897,7 @@ func (w *GCWorker) doGCPlacementRules(dr util.DelRangeTask) (pid int64, err erro return } // Notify PD to drop the placement rules, even if there may be no placement rules. - bundles := []*placement.Bundle{placement.BuildPlacementDropBundle(pid)} + bundles := []*placement.Bundle{placement.NewBundle(pid)} err = infosync.PutRuleBundles(context.TODO(), bundles) return }