From d169a57ba5a4e677196d12b7fef797561315b29b Mon Sep 17 00:00:00 2001 From: Shenghui Wu <793703860@qq.com> Date: Fri, 14 May 2021 11:55:38 +0800 Subject: [PATCH 1/5] util: fix wrong enum building for index range (#24632) --- expression/integration_test.go | 10 ++++++++++ util/ranger/points.go | 2 +- 2 files changed, 11 insertions(+), 1 deletion(-) diff --git a/expression/integration_test.go b/expression/integration_test.go index 8e0f70823f327..095e21119ab30 100644 --- a/expression/integration_test.go +++ b/expression/integration_test.go @@ -9344,4 +9344,14 @@ func (s *testIntegrationSuite) TestEnumIndex(c *C) { testkit.Rows( "OFJHCEKCQGT:MXI7P3[YO4N0DF=2XJWJ4Z9Z;HQ8TMUTZV8YLQAHWJ4BDZHR3A -30 ", "ZOHBSCRMZPOI`IVTSEZAIDAF7DS@1TT20AP9 -30 ")) + + // issue 24576 + tk.MustExec("use test") + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t(col1 enum('a','b','c'), col2 enum('a','b','c'), col3 int, index idx(col1,col2));") + tk.MustExec("insert into t values(1,1,1),(2,2,2),(3,3,3);") + tk.MustQuery("select /*+ use_index(t,idx) */ col3 from t where col2 between 'b' and 'b' and col1 is not null;").Check( + testkit.Rows("2")) + tk.MustQuery("select /*+ use_index(t,idx) */ col3 from t where col2 = 'b' and col1 is not null;").Check( + testkit.Rows("2")) } diff --git a/util/ranger/points.go b/util/ranger/points.go index 9c33ccef7feb3..46a4283dd3222 100644 --- a/util/ranger/points.go +++ b/util/ranger/points.go @@ -459,7 +459,7 @@ func handleEnumFromBinOp(sc *stmtctx.StatementContext, ft *types.FieldType, val tmpEnum := types.Enum{} for i := range ft.Elems { tmpEnum.Name = ft.Elems[i] - tmpEnum.Value = uint64(i) + tmpEnum.Value = uint64(i) + 1 d := types.NewMysqlEnumDatum(tmpEnum) if v, err := d.CompareDatum(sc, &val); err == nil { switch op { From 80a557eb4956da36f21ba1c81ad12595dcf716b1 Mon Sep 17 00:00:00 2001 From: Morgan Tocker Date: Thu, 13 May 2021 22:43:38 -0600 Subject: [PATCH 2/5] sessionctx: change innodb large prefix default (#24555) --- expression/integration_test.go | 6 ++++++ sessionctx/variable/noop.go | 2 +- 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/expression/integration_test.go b/expression/integration_test.go index 095e21119ab30..b1bded2a17719 100644 --- a/expression/integration_test.go +++ b/expression/integration_test.go @@ -9086,9 +9086,15 @@ func (s *testIntegrationSuite) TestEnumPushDown(c *C) { func (s *testIntegrationSuite) TestJiraSetInnoDBDefaultRowFormat(c *C) { // For issue #23541 // JIRA needs to be able to set this to be happy. + // See: https://nova.moe/run-jira-on-tidb/ tk := testkit.NewTestKit(c, s.store) tk.MustExec("set global innodb_default_row_format = dynamic") tk.MustExec("set global innodb_default_row_format = 'dynamic'") + tk.MustQuery("SHOW VARIABLES LIKE 'innodb_default_row_format'").Check(testkit.Rows("innodb_default_row_format dynamic")) + tk.MustQuery("SHOW VARIABLES LIKE 'character_set_server'").Check(testkit.Rows("character_set_server utf8mb4")) + tk.MustQuery("SHOW VARIABLES LIKE 'innodb_file_format'").Check(testkit.Rows("innodb_file_format Barracuda")) + tk.MustQuery("SHOW VARIABLES LIKE 'innodb_large_prefix'").Check(testkit.Rows("innodb_large_prefix ON")) + } func (s *testIntegrationSerialSuite) TestCollationForBinaryLiteral(c *C) { diff --git a/sessionctx/variable/noop.go b/sessionctx/variable/noop.go index c510d9c73ce3a..1ad37b512d807 100644 --- a/sessionctx/variable/noop.go +++ b/sessionctx/variable/noop.go @@ -312,7 +312,7 @@ var noopSysVars = []*SysVar{ {Scope: ScopeNone, Name: "datetime_format", Value: "%Y-%m-%d %H:%i:%s"}, {Scope: ScopeGlobal, Name: "log_syslog", Value: ""}, {Scope: ScopeGlobal | ScopeSession, Name: "transaction_alloc_block_size", Value: "8192"}, - {Scope: ScopeGlobal, Name: "innodb_large_prefix", Type: TypeBool, Value: Off}, + {Scope: ScopeGlobal, Name: "innodb_large_prefix", Type: TypeBool, Value: On}, {Scope: ScopeNone, Name: "performance_schema_max_cond_classes", Value: "80"}, {Scope: ScopeGlobal, Name: "innodb_io_capacity", Value: "200"}, {Scope: ScopeGlobal, Name: "max_binlog_cache_size", Value: "18446744073709547520"}, From ea7f0ca1ba80674a1bec449ca2c3d0f79d48b735 Mon Sep 17 00:00:00 2001 From: Song Gao Date: Fri, 14 May 2021 15:49:38 +0800 Subject: [PATCH 3/5] store: fix data race about KVStore.tikvClient (#24655) --- store/tikv/commit.go | 2 +- store/tikv/kv.go | 25 ++++++++++++++++--------- store/tikv/prewrite.go | 2 +- store/tikv/scan.go | 2 +- store/tikv/split_region.go | 2 +- 5 files changed, 20 insertions(+), 13 deletions(-) diff --git a/store/tikv/commit.go b/store/tikv/commit.go index 449081860c029..10c60d9f6d4bd 100644 --- a/store/tikv/commit.go +++ b/store/tikv/commit.go @@ -48,7 +48,7 @@ func (actionCommit) handleSingleBatch(c *twoPhaseCommitter, bo *Backoffer, batch CommitVersion: c.commitTS, }, pb.Context{Priority: c.priority, SyncLog: c.syncLog}) - sender := NewRegionRequestSender(c.store.regionCache, c.store.client) + sender := NewRegionRequestSender(c.store.regionCache, c.store.GetTiKVClient()) resp, err := sender.SendReq(bo, req, batch.region, ReadTimeoutShort) // If we fail to receive response for the request that commits primary key, it will be undetermined whether this diff --git a/store/tikv/kv.go b/store/tikv/kv.go index f61db4168ef7d..bbf8517a42a8c 100644 --- a/store/tikv/kv.go +++ b/store/tikv/kv.go @@ -66,10 +66,13 @@ var oracleUpdateInterval = 2000 // KVStore contains methods to interact with a TiKV cluster. type KVStore struct { - clusterID uint64 - uuid string - oracle oracle.Oracle - client Client + clusterID uint64 + uuid string + oracle oracle.Oracle + clientMu struct { + sync.RWMutex + client Client + } pdClient pd.Client regionCache *RegionCache lockResolver *LockResolver @@ -133,7 +136,6 @@ func NewKVStore(uuid string, pdClient pd.Client, spkv SafePointKV, client Client clusterID: pdClient.GetClusterID(context.TODO()), uuid: uuid, oracle: o, - client: reqCollapse{client}, pdClient: pdClient, regionCache: NewRegionCache(pdClient), kv: spkv, @@ -142,6 +144,7 @@ func NewKVStore(uuid string, pdClient pd.Client, spkv SafePointKV, client Client closed: make(chan struct{}), replicaReadSeed: rand.Uint32(), } + store.clientMu.client = reqCollapse{client} store.lockResolver = newLockResolver(store) go store.runSafePointChecker() @@ -205,7 +208,7 @@ func (s *KVStore) Close() error { s.pdClient.Close() close(s.closed) - if err := s.client.Close(); err != nil { + if err := s.GetTiKVClient().Close(); err != nil { return errors.Trace(err) } @@ -312,7 +315,7 @@ func (s *KVStore) SupportDeleteRange() (supported bool) { // SendReq sends a request to region. func (s *KVStore) SendReq(bo *Backoffer, req *tikvrpc.Request, regionID RegionVerID, timeout time.Duration) (*tikvrpc.Response, error) { - sender := NewRegionRequestSender(s.regionCache, s.client) + sender := NewRegionRequestSender(s.regionCache, s.GetTiKVClient()) return sender.SendReq(bo, req, regionID, timeout) } @@ -343,12 +346,16 @@ func (s *KVStore) SetOracle(oracle oracle.Oracle) { // SetTiKVClient resets the client instance. func (s *KVStore) SetTiKVClient(client Client) { - s.client = client + s.clientMu.Lock() + defer s.clientMu.Unlock() + s.clientMu.client = client } // GetTiKVClient gets the client instance. func (s *KVStore) GetTiKVClient() (client Client) { - return s.client + s.clientMu.RLock() + defer s.clientMu.RUnlock() + return s.clientMu.client } func (s *KVStore) getSafeTS(storeID uint64) uint64 { diff --git a/store/tikv/prewrite.go b/store/tikv/prewrite.go index 7097ba5dbcd3e..ffb47e1fb46fa 100644 --- a/store/tikv/prewrite.go +++ b/store/tikv/prewrite.go @@ -157,7 +157,7 @@ func (action actionPrewrite) handleSingleBatch(c *twoPhaseCommitter, bo *Backoff req := c.buildPrewriteRequest(batch, txnSize) for { - sender := NewRegionRequestSender(c.store.regionCache, c.store.client) + sender := NewRegionRequestSender(c.store.regionCache, c.store.GetTiKVClient()) resp, err := sender.SendReq(bo, req, batch.region, ReadTimeoutShort) // If we fail to receive response for async commit prewrite, it will be undetermined whether this diff --git a/store/tikv/scan.go b/store/tikv/scan.go index 6c43b7bdee7cd..035291a783aec 100644 --- a/store/tikv/scan.go +++ b/store/tikv/scan.go @@ -164,7 +164,7 @@ func (s *Scanner) getData(bo *Backoffer) error { zap.String("nextEndKey", kv.StrKey(s.nextEndKey)), zap.Bool("reverse", s.reverse), zap.Uint64("txnStartTS", s.startTS())) - sender := NewRegionRequestSender(s.snapshot.store.regionCache, s.snapshot.store.client) + sender := NewRegionRequestSender(s.snapshot.store.regionCache, s.snapshot.store.GetTiKVClient()) var reqEndKey, reqStartKey []byte var loc *KeyLocation var err error diff --git a/store/tikv/split_region.go b/store/tikv/split_region.go index 38ce24917d1cf..c33a89efc19be 100644 --- a/store/tikv/split_region.go +++ b/store/tikv/split_region.go @@ -123,7 +123,7 @@ func (s *KVStore) batchSendSingleRegion(bo *Backoffer, batch batch, scatter bool Priority: kvrpcpb.CommandPri_Normal, }) - sender := NewRegionRequestSender(s.regionCache, s.client) + sender := NewRegionRequestSender(s.regionCache, s.GetTiKVClient()) resp, err := sender.SendReq(bo, req, batch.regionID, ReadTimeoutShort) batchResp := singleBatchResp{resp: resp} From d9f28c7f765662a575e8a2bf6f139229cce1d360 Mon Sep 17 00:00:00 2001 From: Morgan Tocker Date: Fri, 14 May 2021 02:07:39 -0600 Subject: [PATCH 4/5] executor, privileges: Add dynamic privileges to SHOW PRIVILEGES (#24646) --- executor/executor_test.go | 9 ++++++++- executor/show.go | 4 ++++ privilege/privileges/privileges.go | 11 +++++++++++ privilege/privileges/privileges_test.go | 12 ++++++++++++ 4 files changed, 35 insertions(+), 1 deletion(-) diff --git a/executor/executor_test.go b/executor/executor_test.go index 65af164174e6f..d67bb6b48b8f4 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -350,7 +350,14 @@ func (s *testSuiteP1) TestShow(c *C) { "Trigger Tables To use triggers", "Create tablespace Server Admin To create/alter/drop tablespaces", "Update Tables To update existing rows", - "Usage Server Admin No privileges - allow connect only")) + "Usage Server Admin No privileges - allow connect only", + "BACKUP_ADMIN Server Admin ", + "SYSTEM_VARIABLES_ADMIN Server Admin ", + "ROLE_ADMIN Server Admin ", + "CONNECTION_ADMIN Server Admin ", + "RESTRICTED_TABLES_ADMIN Server Admin ", + "RESTRICTED_STATUS_ADMIN Server Admin ", + )) c.Assert(len(tk.MustQuery("show table status").Rows()), Equals, 1) } diff --git a/executor/show.go b/executor/show.go index 2bd9b786fcffb..ab08e5ba4cbf5 100644 --- a/executor/show.go +++ b/executor/show.go @@ -1412,6 +1412,10 @@ func (e *ShowExec) fetchShowPrivileges() error { e.appendRow([]interface{}{"Create tablespace", "Server Admin", "To create/alter/drop tablespaces"}) e.appendRow([]interface{}{"Update", "Tables", "To update existing rows"}) e.appendRow([]interface{}{"Usage", "Server Admin", "No privileges - allow connect only"}) + + for _, priv := range privileges.GetDynamicPrivileges() { + e.appendRow([]interface{}{priv, "Server Admin", ""}) + } return nil } diff --git a/privilege/privileges/privileges.go b/privilege/privileges/privileges.go index c5ec2f8394385..6ac58e04e44e4 100644 --- a/privilege/privileges/privileges.go +++ b/privilege/privileges/privileges.go @@ -535,3 +535,14 @@ func RegisterDynamicPrivilege(privNameInUpper string) error { dynamicPrivs = append(dynamicPrivs, privNameInUpper) return nil } + +// GetDynamicPrivileges returns the list of registered DYNAMIC privileges +// for use in meta data commands (i.e. SHOW PRIVILEGES) +func GetDynamicPrivileges() []string { + dynamicPrivLock.Lock() + defer dynamicPrivLock.Unlock() + + privCopy := make([]string, len(dynamicPrivs)) + copy(privCopy, dynamicPrivs) + return privCopy +} diff --git a/privilege/privileges/privileges_test.go b/privilege/privileges/privileges_test.go index 3038aad397076..c7a825a4d894d 100644 --- a/privilege/privileges/privileges_test.go +++ b/privilege/privileges/privileges_test.go @@ -1427,3 +1427,15 @@ func (s *testPrivilegeSuite) TestViewDefiner(c *C) { tk.MustExec("select * from test_view") tk.MustExec("select * from test_view2") } + +func (s *testPrivilegeSuite) TestDynamicPrivsRegistration(c *C) { + se := newSession(c, s.store, s.dbName) + pm := privilege.GetPrivilegeManager(se) + + count := len(privileges.GetDynamicPrivileges()) + + c.Assert(pm.IsDynamicPrivilege("ACDC_ADMIN"), IsFalse) + privileges.RegisterDynamicPrivilege("ACDC_ADMIN") + c.Assert(pm.IsDynamicPrivilege("ACDC_ADMIN"), IsTrue) + c.Assert(len(privileges.GetDynamicPrivileges()), Equals, count+1) +} From 031a9fa18b909c1501c99d6a8a98448beb94a962 Mon Sep 17 00:00:00 2001 From: xhe Date: Fri, 14 May 2021 17:05:38 +0800 Subject: [PATCH 5/5] ddl: refactor rule [4/6] (#24007) --- ddl/ddl_api.go | 14 +-- ddl/placement/errors.go | 6 + ddl/placement/rule.go | 132 +++++++++++++++++++++ ddl/placement/rule_test.go | 206 +++++++++++++++++++++++++++++++++ ddl/placement/types.go | 36 ------ ddl/placement/types_test.go | 12 -- ddl/placement/utils.go | 4 +- ddl/placement/utils_test.go | 14 +-- ddl/placement_rule_test.go | 36 +++--- ddl/placement_sql_test.go | 6 +- executor/infoschema_reader.go | 2 +- expression/integration_test.go | 2 +- infoschema/tables_test.go | 2 +- session/session_test.go | 2 +- 14 files changed, 385 insertions(+), 89 deletions(-) create mode 100644 ddl/placement/rule.go create mode 100644 ddl/placement/rule_test.go diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 528a6087638fc..a3f8bb7f9c622 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -5895,8 +5895,8 @@ func buildPlacementSpecReplicasAndConstraint(replicas uint64, cnstr string) ([]* } rules = append(rules, &placement.Rule{ - Count: int(replicas), - LabelConstraints: labelConstraints, + Count: int(replicas), + Constraints: labelConstraints, }) return rules, nil @@ -5925,8 +5925,8 @@ func buildPlacementSpecReplicasAndConstraint(replicas uint64, cnstr string) ([]* } rules = append(rules, &placement.Rule{ - Count: cnt, - LabelConstraints: labelConstraints, + Count: cnt, + Constraints: labelConstraints, }) } @@ -6051,14 +6051,14 @@ func (d *ddl) AlterTableAlterPartition(ctx sessionctx.Context, ident ast.Ident, newRules := bundle.Rules[:0] for i, rule := range bundle.Rules { // merge all empty constraints - if len(rule.LabelConstraints) == 0 { + 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.LabelConstraints.Add(placement.Constraint{ + if err := rule.Constraints.Add(placement.Constraint{ Op: placement.NotIn, Key: placement.EngineLabelKey, Values: []string{placement.EngineLabelTiFlash}, @@ -6083,7 +6083,7 @@ func (d *ddl) AlterTableAlterPartition(ctx sessionctx.Context, ident ast.Ident, Count: cnt, StartKeyHex: startKey, EndKeyHex: endKey, - LabelConstraints: []placement.Constraint{{ + Constraints: []placement.Constraint{{ Op: placement.NotIn, Key: placement.EngineLabelKey, Values: []string{placement.EngineLabelTiFlash}, diff --git a/ddl/placement/errors.go b/ddl/placement/errors.go index 19797022a609c..95fce4591c961 100644 --- a/ddl/placement/errors.go +++ b/ddl/placement/errors.go @@ -24,4 +24,10 @@ var ( ErrUnsupportedConstraint = errors.New("unsupported label constraint") // ErrConflictingConstraints is from constraints.go. ErrConflictingConstraints = errors.New("conflicting label constraints") + // ErrInvalidConstraintsMapcnt is from rule.go. + ErrInvalidConstraintsMapcnt = errors.New("label constraints in map syntax have invalid replicas") + // ErrInvalidConstraintsFormat is from rule.go. + ErrInvalidConstraintsFormat = errors.New("invalid label constraints format") + // ErrInvalidConstraintsRelicas is from rule.go. + ErrInvalidConstraintsRelicas = errors.New("label constraints with invalid REPLICAS") ) diff --git a/ddl/placement/rule.go b/ddl/placement/rule.go new file mode 100644 index 0000000000000..134bdd5a610f9 --- /dev/null +++ b/ddl/placement/rule.go @@ -0,0 +1,132 @@ +// 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 ( + "fmt" + "strings" + + "github.com/go-yaml/yaml" +) + +// PeerRoleType is the expected peer type of the placement rule. +type PeerRoleType string + +const ( + // Voter can either match a leader peer or follower peer. + Voter PeerRoleType = "voter" + // Leader matches a leader. + Leader PeerRoleType = "leader" + // Follower matches a follower. + Follower PeerRoleType = "follower" + // Learner matches a learner. + Learner PeerRoleType = "learner" +) + +// Rule is the core placement rule struct. Check https://github.com/tikv/pd/blob/master/server/schedule/placement/rule.go. +type Rule struct { + GroupID string `json:"group_id"` + ID string `json:"id"` + Index int `json:"index,omitempty"` + Override bool `json:"override,omitempty"` + StartKeyHex string `json:"start_key"` + EndKeyHex string `json:"end_key"` + Role PeerRoleType `json:"role"` + Count int `json:"count"` + Constraints Constraints `json:"label_constraints,omitempty"` + LocationLabels []string `json:"location_labels,omitempty"` + IsolationLevel string `json:"isolation_level,omitempty"` +} + +// NewRules constructs []*Rule from a yaml-compatible representation of +// array or map of constraints. It converts 'CONSTRAINTS' field in RFC +// https://github.com/pingcap/tidb/blob/master/docs/design/2020-06-24-placement-rules-in-sql.md to structs. +func NewRules(replicas uint64, cnstr string) ([]*Rule, error) { + rules := []*Rule{} + + cnstbytes := []byte(cnstr) + + constraints1 := []string{} + err1 := yaml.UnmarshalStrict(cnstbytes, &constraints1) + if err1 == nil { + // can not emit REPLICAS with an array or empty label + if replicas == 0 { + return rules, fmt.Errorf("%w: should be positive", ErrInvalidConstraintsRelicas) + } + + labelConstraints, err := NewConstraints(constraints1) + if err != nil { + return rules, err + } + + rules = append(rules, &Rule{ + Count: int(replicas), + Constraints: labelConstraints, + }) + + return rules, nil + } + + constraints2 := map[string]int{} + err2 := yaml.UnmarshalStrict(cnstbytes, &constraints2) + if err2 == nil { + ruleCnt := 0 + for labels, cnt := range constraints2 { + if cnt <= 0 { + return rules, fmt.Errorf("%w: count of labels '%s' should be positive, but got %d", ErrInvalidConstraintsMapcnt, labels, cnt) + } + ruleCnt += cnt + } + + if replicas == 0 { + replicas = uint64(ruleCnt) + } + + if int(replicas) < ruleCnt { + return rules, fmt.Errorf("%w: should be larger or equal to the number of total replicas, but REPLICAS=%d < total=%d", ErrInvalidConstraintsRelicas, replicas, ruleCnt) + } + + for labels, cnt := range constraints2 { + labelConstraints, err := NewConstraints(strings.Split(labels, ",")) + if err != nil { + return rules, err + } + + rules = append(rules, &Rule{ + Count: cnt, + Constraints: labelConstraints, + }) + } + + remain := int(replicas) - ruleCnt + if remain > 0 { + rules = append(rules, &Rule{ + Count: remain, + }) + } + + return rules, nil + } + + return nil, fmt.Errorf("%w: should be [constraint1, ...] (error %s), {constraint1: cnt1, ...} (error %s), or any yaml compatible representation", ErrInvalidConstraintsFormat, err1, err2) +} + +// Clone is used to duplicate a RuleOp for safe modification. +// Note that it is a shallow copy: LocationLabels and Constraints +// is not cloned. +func (r *Rule) Clone() *Rule { + n := &Rule{} + *n = *r + return n +} diff --git a/ddl/placement/rule_test.go b/ddl/placement/rule_test.go new file mode 100644 index 0000000000000..85dd492f348e7 --- /dev/null +++ b/ddl/placement/rule_test.go @@ -0,0 +1,206 @@ +// 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/json" + "errors" + + . "github.com/pingcap/check" +) + +var _ = Suite(&testRuleSuite{}) + +type testRuleSuite struct{} + +func (t *testRuleSuite) TestClone(c *C) { + rule := &Rule{ID: "434"} + newRule := rule.Clone() + newRule.ID = "121" + + c.Assert(rule, DeepEquals, &Rule{ID: "434"}) + c.Assert(newRule, DeepEquals, &Rule{ID: "121"}) +} + +func matchRule(r1 *Rule, t2 []*Rule) bool { + for _, r2 := range t2 { + if ok, _ := DeepEquals.Check([]interface{}{r1, r2}, nil); ok { + return true + } + } + return false +} + +func matchRules(t1, t2 []*Rule, prefix string, c *C) { + expected, err := json.Marshal(t1) + c.Assert(err, IsNil) + got, err := json.Marshal(t2) + c.Assert(err, IsNil) + comment := Commentf("%s, expected %s\nbut got %s", prefix, expected, got) + c.Assert(len(t1), Equals, len(t2), comment) + for _, r1 := range t1 { + c.Assert(matchRule(r1, t2), IsTrue, comment) + } +} + +func (t *testRuleSuite) TestNewRules(c *C) { + type TestCase struct { + name string + input string + replicas uint64 + output []*Rule + err error + } + tests := []TestCase{} + + tests = append(tests, TestCase{ + name: "empty constraints", + input: "", + replicas: 3, + output: []*Rule{ + { + Count: 3, + Constraints: Constraints{}, + }, + }, + }) + + tests = append(tests, TestCase{ + name: "zero replicas", + input: "", + replicas: 0, + err: ErrInvalidConstraintsRelicas, + }) + + labels, err := NewConstraints([]string{"+zone=sh", "+zone=sh"}) + c.Assert(err, IsNil) + tests = append(tests, TestCase{ + name: "normal array constraints", + input: `["+zone=sh", "+zone=sh"]`, + replicas: 3, + output: []*Rule{ + { + Count: 3, + Constraints: labels, + }, + }, + }) + + labels1, err := NewConstraints([]string{"+zone=sh", "-zone=bj"}) + c.Assert(err, IsNil) + labels2, err := NewConstraints([]string{"+zone=sh"}) + c.Assert(err, IsNil) + tests = append(tests, TestCase{ + name: "normal object constraints", + input: `{"+zone=sh,-zone=bj":2, "+zone=sh": 1}`, + replicas: 3, + output: []*Rule{ + { + Count: 2, + Constraints: labels1, + }, + { + Count: 1, + Constraints: labels2, + }, + }, + }) + + tests = append(tests, TestCase{ + name: "normal object constraints, with extra count", + input: "{'+zone=sh,-zone=bj':2, '+zone=sh': 1}", + replicas: 4, + output: []*Rule{ + { + Count: 2, + Constraints: labels1, + }, + { + Count: 1, + Constraints: labels2, + }, + { + Count: 1, + }, + }, + }) + + tests = append(tests, TestCase{ + name: "normal object constraints, without count", + input: "{'+zone=sh,-zone=bj':2, '+zone=sh': 1}", + output: []*Rule{ + { + Count: 2, + Constraints: labels1, + }, + { + Count: 1, + Constraints: labels2, + }, + }, + }) + + tests = append(tests, TestCase{ + name: "zero count in object constraints", + input: `{"+zone=sh,-zone=bj":0, "+zone=sh": 1}`, + replicas: 3, + err: ErrInvalidConstraintsMapcnt, + }) + + tests = append(tests, TestCase{ + name: "overlarge total count in object constraints", + input: `{"+ne=sh,-zone=bj":1, "+zone=sh": 4}`, + replicas: 3, + err: ErrInvalidConstraintsRelicas, + }) + + tests = append(tests, TestCase{ + name: "invalid array", + input: `["+ne=sh", "+zone=sh"`, + replicas: 3, + err: ErrInvalidConstraintsFormat, + }) + + tests = append(tests, TestCase{ + name: "invalid array constraints", + input: `["ne=sh", "+zone=sh"]`, + replicas: 3, + err: ErrInvalidConstraintFormat, + }) + + tests = append(tests, TestCase{ + name: "invalid map", + input: `{+ne=sh,-zone=bj:1, "+zone=sh": 4`, + replicas: 5, + err: ErrInvalidConstraintsFormat, + }) + + tests = append(tests, TestCase{ + name: "invalid map constraints", + input: `{"nesh,-zone=bj":1, "+zone=sh": 4}`, + replicas: 6, + err: ErrInvalidConstraintFormat, + }) + + for _, t := range tests { + comment := Commentf("%s", t.name) + output, err := NewRules(t.replicas, t.input) + if t.err == nil { + c.Assert(err, IsNil, comment) + matchRules(t.output, output, comment.CheckCommentString(), c) + } else { + c.Assert(errors.Is(err, t.err), IsTrue, comment) + } + } +} diff --git a/ddl/placement/types.go b/ddl/placement/types.go index 3bb9da96e3890..72093a2c19c78 100644 --- a/ddl/placement/types.go +++ b/ddl/placement/types.go @@ -22,42 +22,6 @@ import ( // After all, placement rules are communicated using an HTTP API. Loose // coupling is a good feature. -// PeerRoleType is the expected peer type of the placement rule. -type PeerRoleType string - -const ( - // Voter can either match a leader peer or follower peer. - Voter PeerRoleType = "voter" - // Leader matches a leader. - Leader PeerRoleType = "leader" - // Follower matches a follower. - Follower PeerRoleType = "follower" - // Learner matches a learner. - Learner PeerRoleType = "learner" -) - -// Rule is the placement rule. Check https://github.com/tikv/pd/blob/master/server/schedule/placement/rule.go. -type Rule struct { - GroupID string `json:"group_id"` - ID string `json:"id"` - Index int `json:"index,omitempty"` - Override bool `json:"override,omitempty"` - StartKeyHex string `json:"start_key"` - EndKeyHex string `json:"end_key"` - Role PeerRoleType `json:"role"` - Count int `json:"count"` - LabelConstraints Constraints `json:"label_constraints,omitempty"` - LocationLabels []string `json:"location_labels,omitempty"` - IsolationLevel string `json:"isolation_level,omitempty"` -} - -// Clone is used to duplicate a RuleOp for safe modification. -func (r *Rule) Clone() *Rule { - n := &Rule{} - *n = *r - return n -} - // Bundle is a group of all rules and configurations. It is used to support rule cache. type Bundle struct { ID string `json:"group_id"` diff --git a/ddl/placement/types_test.go b/ddl/placement/types_test.go index 77153cb29b692..93ed1a5a80f43 100644 --- a/ddl/placement/types_test.go +++ b/ddl/placement/types_test.go @@ -18,7 +18,6 @@ import ( ) var _ = Suite(&testBundleSuite{}) -var _ = Suite(&testRuleSuite{}) type testBundleSuite struct{} @@ -49,14 +48,3 @@ func (t *testBundleSuite) TestClone(c *C) { c.Assert(bundle, DeepEquals, &Bundle{ID: GroupID(1), Rules: []*Rule{{ID: "434"}}}) c.Assert(newBundle, DeepEquals, &Bundle{ID: GroupID(2), Rules: []*Rule{{ID: "121"}}}) } - -type testRuleSuite struct{} - -func (t *testRuleSuite) TestClone(c *C) { - rule := &Rule{ID: "434"} - newRule := rule.Clone() - newRule.ID = "121" - - c.Assert(rule, DeepEquals, &Rule{ID: "434"}) - c.Assert(newRule, DeepEquals, &Rule{ID: "121"}) -} diff --git a/ddl/placement/utils.go b/ddl/placement/utils.go index 16c0a424dde53..5b12f10e2d243 100644 --- a/ddl/placement/utils.go +++ b/ddl/placement/utils.go @@ -61,7 +61,7 @@ func BuildPlacementCopyBundle(oldBundle *Bundle, newID int64) *Bundle { func GetLeaderDCByBundle(bundle *Bundle, dcLabelKey string) (string, bool) { for _, rule := range bundle.Rules { if isValidLeaderRule(rule, dcLabelKey) { - return rule.LabelConstraints[0].Values[0], true + return rule.Constraints[0].Values[0], true } } return "", false @@ -69,7 +69,7 @@ func GetLeaderDCByBundle(bundle *Bundle, dcLabelKey string) (string, bool) { func isValidLeaderRule(rule *Rule, dcLabelKey string) bool { if rule.Role == Leader && rule.Count == 1 { - for _, con := range rule.LabelConstraints { + for _, con := range rule.Constraints { if con.Op == In && con.Key == dcLabelKey && len(con.Values) == 1 { return true } diff --git a/ddl/placement/utils_test.go b/ddl/placement/utils_test.go index 964382846485e..10941e0663455 100644 --- a/ddl/placement/utils_test.go +++ b/ddl/placement/utils_test.go @@ -58,7 +58,7 @@ func (t *testUtilsSuite) TestGetLeaderDCByBundle(c *C) { { ID: "12", Role: Leader, - LabelConstraints: []Constraint{ + Constraints: []Constraint{ { Key: "zone", Op: In, @@ -84,7 +84,7 @@ func (t *testUtilsSuite) TestGetLeaderDCByBundle(c *C) { { ID: "12", Role: Voter, - LabelConstraints: []Constraint{ + Constraints: []Constraint{ { Key: "zone", Op: In, @@ -110,7 +110,7 @@ func (t *testUtilsSuite) TestGetLeaderDCByBundle(c *C) { { ID: "11", Role: Leader, - LabelConstraints: []Constraint{ + Constraints: []Constraint{ { Key: "zone", Op: In, @@ -127,7 +127,7 @@ func (t *testUtilsSuite) TestGetLeaderDCByBundle(c *C) { { ID: "12", Role: Voter, - LabelConstraints: []Constraint{ + Constraints: []Constraint{ { Key: "zone", Op: In, @@ -153,7 +153,7 @@ func (t *testUtilsSuite) TestGetLeaderDCByBundle(c *C) { { ID: "11", Role: Leader, - LabelConstraints: []Constraint{ + Constraints: []Constraint{ { Key: "fake", Op: In, @@ -179,7 +179,7 @@ func (t *testUtilsSuite) TestGetLeaderDCByBundle(c *C) { { ID: "11", Role: Leader, - LabelConstraints: []Constraint{ + Constraints: []Constraint{ { Key: "zone", Op: NotIn, @@ -205,7 +205,7 @@ func (t *testUtilsSuite) TestGetLeaderDCByBundle(c *C) { { ID: "11", Role: Leader, - LabelConstraints: []Constraint{ + Constraints: []Constraint{ { Key: "zone", Op: In, diff --git a/ddl/placement_rule_test.go b/ddl/placement_rule_test.go index b051092a776e9..a9a916cb5a199 100644 --- a/ddl/placement_rule_test.go +++ b/ddl/placement_rule_test.go @@ -52,7 +52,7 @@ func (s *testPlacementSuite) TestPlacementBuild(c *C) { { Role: placement.Voter, Count: 3, - LabelConstraints: []placement.Constraint{ + Constraints: []placement.Constraint{ {Key: "zone", Op: "in", Values: []string{"sh"}}, }, }, @@ -67,9 +67,9 @@ func (s *testPlacementSuite) TestPlacementBuild(c *C) { Constraints: "", }}, output: []*placement.Rule{{ - Role: placement.Voter, - Count: 3, - LabelConstraints: []placement.Constraint{}, + Role: placement.Voter, + Count: 3, + Constraints: []placement.Constraint{}, }}, }, @@ -83,14 +83,14 @@ func (s *testPlacementSuite) TestPlacementBuild(c *C) { { Role: placement.Voter, Count: 1, - LabelConstraints: []placement.Constraint{ + Constraints: []placement.Constraint{ {Key: "zone", Op: "in", Values: []string{"sh"}}, }, }, { Role: placement.Voter, Count: 2, - LabelConstraints: []placement.Constraint{ + Constraints: []placement.Constraint{ {Key: "zone", Op: "in", Values: []string{"sh"}}, }, }, @@ -108,7 +108,7 @@ func (s *testPlacementSuite) TestPlacementBuild(c *C) { { Role: placement.Voter, Count: 3, - LabelConstraints: []placement.Constraint{ + Constraints: []placement.Constraint{ {Key: "zone", Op: "notIn", Values: []string{"sh"}}, {Key: "zone", Op: "notIn", Values: []string{"bj"}}, }, @@ -127,7 +127,7 @@ func (s *testPlacementSuite) TestPlacementBuild(c *C) { { Role: placement.Voter, Count: 3, - LabelConstraints: []placement.Constraint{ + Constraints: []placement.Constraint{ {Key: "zone", Op: "in", Values: []string{"sh"}}, {Key: "zone", Op: "notIn", Values: []string{"bj"}}, }, @@ -154,7 +154,7 @@ func (s *testPlacementSuite) TestPlacementBuild(c *C) { { Role: placement.Voter, Count: 3, - LabelConstraints: []placement.Constraint{ + Constraints: []placement.Constraint{ {Key: "zone", Op: "in", Values: []string{"sh"}}, {Key: "zone", Op: "notIn", Values: []string{"bj"}}, }, @@ -162,7 +162,7 @@ func (s *testPlacementSuite) TestPlacementBuild(c *C) { { Role: placement.Follower, Count: 2, - LabelConstraints: []placement.Constraint{ + Constraints: []placement.Constraint{ {Key: "zone", Op: "notIn", Values: []string{"sh"}}, {Key: "zone", Op: "in", Values: []string{"bj"}}, }, @@ -189,7 +189,7 @@ func (s *testPlacementSuite) TestPlacementBuild(c *C) { { Role: placement.Voter, Count: 2, - LabelConstraints: []placement.Constraint{ + Constraints: []placement.Constraint{ {Key: "zone", Op: "notIn", Values: []string{"sh"}}, {Key: "zone", Op: "in", Values: []string{"bj"}}, }, @@ -214,14 +214,14 @@ func (s *testPlacementSuite) TestPlacementBuild(c *C) { }, output: []*placement.Rule{ { - Role: placement.Voter, - Count: 1, - LabelConstraints: []placement.Constraint{{Key: "zone", Op: "notIn", Values: []string{"sh"}}}, + Role: placement.Voter, + Count: 1, + Constraints: []placement.Constraint{{Key: "zone", Op: "notIn", Values: []string{"sh"}}}, }, { - Role: placement.Voter, - Count: 1, - LabelConstraints: []placement.Constraint{{Key: "zone", Op: "in", Values: []string{"bj"}}}, + Role: placement.Voter, + Count: 1, + Constraints: []placement.Constraint{{Key: "zone", Op: "in", Values: []string{"bj"}}}, }, { Role: placement.Voter, @@ -306,7 +306,7 @@ func (s *testPlacementSuite) TestPlacementBuild(c *C) { { Role: placement.Voter, Count: 3, - LabelConstraints: []placement.Constraint{ + Constraints: []placement.Constraint{ {Key: "zone", Op: "in", Values: []string{"sh"}}, {Key: "zone", Op: "notIn", Values: []string{"bj"}}, }, diff --git a/ddl/placement_sql_test.go b/ddl/placement_sql_test.go index e77b0ba99d5cf..fb7158681714f 100644 --- a/ddl/placement_sql_test.go +++ b/ddl/placement_sql_test.go @@ -404,7 +404,7 @@ PARTITION BY RANGE (c) ( GroupID: groupID, Role: placement.Leader, Count: 1, - LabelConstraints: []placement.Constraint{ + Constraints: []placement.Constraint{ { Key: placement.DCLabelKey, Op: placement.In, @@ -423,7 +423,7 @@ PARTITION BY RANGE (c) ( GroupID: groupID, Role: placement.Follower, Count: 3, - LabelConstraints: []placement.Constraint{ + Constraints: []placement.Constraint{ { Key: placement.DCLabelKey, Op: placement.In, @@ -619,7 +619,7 @@ PARTITION BY RANGE (c) ( GroupID: groupID, Role: placement.Leader, Count: 1, - LabelConstraints: []placement.Constraint{ + Constraints: []placement.Constraint{ { Key: placement.DCLabelKey, Op: placement.In, diff --git a/executor/infoschema_reader.go b/executor/infoschema_reader.go index 4d56cc55accac..4f788a3d7bd1d 100644 --- a/executor/infoschema_reader.go +++ b/executor/infoschema_reader.go @@ -1915,7 +1915,7 @@ func (e *memtableRetriever) setDataForPlacementPolicy(ctx sessionctx.Context) er continue } for _, rule := range bundle.Rules { - constraint, err := rule.LabelConstraints.Restore() + constraint, err := rule.Constraints.Restore() if err != nil { return errors.Wrapf(err, "Restore rule %s in bundle %s failed", rule.ID, bundle.ID) } diff --git a/expression/integration_test.go b/expression/integration_test.go index b1bded2a17719..80e39b76ce746 100644 --- a/expression/integration_test.go +++ b/expression/integration_test.go @@ -8755,7 +8755,7 @@ PARTITION BY RANGE (c) ( GroupID: groupID, Role: placement.Leader, Count: 1, - LabelConstraints: []placement.Constraint{ + Constraints: []placement.Constraint{ { Key: placement.DCLabelKey, Op: placement.In, diff --git a/infoschema/tables_test.go b/infoschema/tables_test.go index 6cc24300c1be4..ebe4a0620256f 100644 --- a/infoschema/tables_test.go +++ b/infoschema/tables_test.go @@ -1449,7 +1449,7 @@ func (s *testTableSuite) TestPlacementPolicy(c *C) { ID: "0", Role: "voter", Count: 3, - LabelConstraints: []placement.Constraint{ + Constraints: []placement.Constraint{ { Key: "zone", Op: "in", diff --git a/session/session_test.go b/session/session_test.go index 3baee4f0ef6f1..a8861dfddd79c 100644 --- a/session/session_test.go +++ b/session/session_test.go @@ -3319,7 +3319,7 @@ PARTITION BY RANGE (c) ( GroupID: groupID, Role: placement.Leader, Count: 1, - LabelConstraints: []placement.Constraint{ + Constraints: []placement.Constraint{ { Key: placement.DCLabelKey, Op: placement.In,