Skip to content

Commit

Permalink
ddl: support ALTER PLACEMENT clause (pingcap#19065)
Browse files Browse the repository at this point in the history
* ddl: check count here

* ddl: drop all rules that will be overrided

* ddl: change ID format

* ddl: local placement deifinition

* ddl: support `ALTER PLACEMENT` clause

* ddl: add test cases

* ddl: fix ID format

* ddl: copy comments too

* ddl: allow ignore replicas with dict constraints

* ddl: rename COUNT to REPLICAS

* ddl: clone method for RuleOp

* ddl: checkPlacementSpec => buildPlacementSpec

* ddl: remove TODO
It is possible to incrementally add replicas. Consider drop later. Let
us support it correctly first.

* ddl: ID format
Datbase/schema ID is added back. A common prefix will improve the
efficience of PD batch operation. Drop a whole db/table will only need
one loop for rules.

* ddl: move const/util to placement package

* ddl: fix test

* ddl: fix format error

* ddl: error check fix

* ddl: use an explicit condition flag

* ddl: adapt parser changes

* ddl: fix go.mod/simplify code path

* ddl: go mod tidy

* ddl: improve tests

* ddl: return new rules instead of appending

* ddl: return constraints instead of appending

* ddl: fix test

* ddl: one more test

* ddl: remove rules rollback, meaningless

* ddl: fix slice

* ddl: add period to comments

* Update ddl/ddl_api.go

Co-authored-by: tangenta <[email protected]>

* ddl: remove unused arguments

* infosync: do not request PD if no ruleOperations

* ddl: a new test suite

Co-authored-by: tangenta <[email protected]>
Co-authored-by: ti-srebot <[email protected]>
  • Loading branch information
3 people authored Aug 11, 2020
1 parent 2de3aed commit a016061
Show file tree
Hide file tree
Showing 11 changed files with 751 additions and 250 deletions.
171 changes: 80 additions & 91 deletions ddl/ddl_api.go
Original file line number Diff line number Diff line change
Expand Up @@ -36,8 +36,8 @@ import (
"github.com/pingcap/parser/model"
"github.com/pingcap/parser/mysql"
field_types "github.com/pingcap/parser/types"
"github.com/pingcap/pd/v4/server/schedule/placement"
"github.com/pingcap/tidb/config"
"github.com/pingcap/tidb/ddl/placement"
"github.com/pingcap/tidb/expression"
"github.com/pingcap/tidb/infoschema"
"github.com/pingcap/tidb/kv"
Expand All @@ -62,16 +62,6 @@ import (

const expressionIndexPrefix = "_V$"

const placementRuleDefaultGroupID = "TiDB_DDL"

const (
placementRuleIndexDefault int = iota
placementRuleIndexDatabase
placementRuleIndexTable
placementRuleIndexPartition
placementRuleIndexIndex
)

func (d *ddl) CreateSchema(ctx sessionctx.Context, schema model.CIStr, charsetInfo *ast.CharsetOpt) error {
dbInfo := &model.DBInfo{Name: schema}
if charsetInfo != nil {
Expand Down Expand Up @@ -5165,67 +5155,25 @@ func (d *ddl) AlterIndexVisibility(ctx sessionctx.Context, ident ast.Ident, inde
return errors.Trace(err)
}

func checkPlacementLabelConstraint(label string) (placement.LabelConstraint, error) {
r := placement.LabelConstraint{}

if len(label) < 4 {
return r, errors.Errorf("label constraint should be in format '{+|-}key=value', but got '%s'", label)
}

var op placement.LabelConstraintOp
switch label[0] {
case '+':
op = placement.In
case '-':
op = placement.NotIn
default:
return r, errors.Errorf("label constraint should be in format '{+|-}key=value', but got '%s'", label)
}

kv := strings.Split(label[1:], "=")
if len(kv) != 2 {
return r, errors.Errorf("label constraint should be in format '{+|-}key=value', but got '%s'", label)
}

key := strings.TrimSpace(kv[0])
if key == "" {
return r, errors.Errorf("label constraint should be in format '{+|-}key=value', but got '%s'", label)
}

val := strings.TrimSpace(kv[1])
if val == "" {
return r, errors.Errorf("label constraint should be in format '{+|-}key=value', but got '%s'", label)
}

r.Key = key
r.Op = op
r.Values = []string{val}
return r, nil
}

func checkPlacementLabelConstraints(rule *placement.Rule, labels []string) error {
for _, str := range labels {
label, err := checkPlacementLabelConstraint(strings.TrimSpace(str))
if err != nil {
return err
}
rule.LabelConstraints = append(rule.LabelConstraints, label)
}
return nil
}

func checkPlacementSpecConstraint(rules []*placement.Rule, rule *placement.Rule, cnstr string) ([]*placement.Rule, error) {
cnstr = strings.TrimSpace(cnstr)
func buildPlacementSpecReplicasAndConstraint(rule *placement.RuleOp, replicas uint64, cnstr string) ([]*placement.RuleOp, error) {
var err error
cnstr = strings.TrimSpace(cnstr)
rules := make([]*placement.RuleOp, 0, 1)
if len(cnstr) > 0 && cnstr[0] == '[' {
// can not emit REPLICAS with an array label
if replicas == 0 {
return rules, errors.Errorf("array CONSTRAINTS should be with a positive REPLICAS")
}
rule.Count = int(replicas)

constraints := []string{}

err = json.Unmarshal([]byte(cnstr), &constraints)
if err != nil {
return rules, err
}

err = checkPlacementLabelConstraints(rule, constraints)
rule.LabelConstraints, err = placement.CheckLabelConstraints(constraints)
if err != nil {
return rules, err
}
Expand All @@ -5238,66 +5186,107 @@ func checkPlacementSpecConstraint(rules []*placement.Rule, rule *placement.Rule,
return rules, err
}

ruleCnt := int(replicas)
for labels, cnt := range constraints {
newRule := &placement.Rule{}
*newRule = *rule
newRule := rule.Clone()
if cnt <= 0 {
err = errors.Errorf("count should be non-positive, but got %d", cnt)
err = errors.Errorf("count should be positive, but got %d", cnt)
break
}
// TODO: handle or remove rule.Count in later commits
rule.Count -= cnt

if replicas != 0 {
ruleCnt -= cnt
if ruleCnt < 0 {
err = errors.Errorf("REPLICAS should be larger or equal to the number of total replicas, but got %d", replicas)
break
}
}
newRule.Count = cnt
err = checkPlacementLabelConstraints(newRule, strings.Split(strings.TrimSpace(labels), ","))

newRule.LabelConstraints, err = placement.CheckLabelConstraints(strings.Split(strings.TrimSpace(labels), ","))
if err != nil {
break
}
rules = append(rules, newRule)
}
rule.Count = ruleCnt

if rule.Count > 0 {
rules = append(rules, rule)
}
} else {
err = errors.Errorf("constraint should be a JSON array or object, but got '%s'", cnstr)
}
return rules, err
}

func checkPlacementSpecs(specs []*ast.PlacementSpec) ([]*placement.Rule, error) {
rules := make([]*placement.Rule, 0, len(specs))
func buildPlacementSpecs(specs []*ast.PlacementSpec) ([]*placement.RuleOp, error) {
rules := make([]*placement.RuleOp, 0, len(specs))

var err error
var sb strings.Builder
restoreFlags := format.RestoreStringSingleQuotes | format.RestoreKeyWordLowercase | format.RestoreNameBackQuotes
restoreCtx := format.NewRestoreCtx(restoreFlags, &sb)

for _, spec := range specs {
rule := &placement.Rule{
GroupID: placementRuleDefaultGroupID,
Count: int(spec.Replicas),
Override: true,
rule := &placement.RuleOp{
Rule: &placement.Rule{
GroupID: placement.RuleDefaultGroupID,
Override: true,
},
}

switch spec.Tp {
case ast.PlacementAdd:
switch spec.Role {
case ast.PlacementRoleFollower:
rule.Role = placement.Follower
case ast.PlacementRoleLeader:
rule.Role = placement.Leader
case ast.PlacementRoleLearner:
rule.Role = placement.Learner
case ast.PlacementRoleVoter:
rule.Role = placement.Voter
default:
err = errors.Errorf("unknown action type: %d", spec.Tp)
err = errors.Errorf("unknown role: %d", spec.Role)
}

if err == nil {
switch spec.Role {
case ast.PlacementRoleFollower:
rule.Role = placement.Follower
case ast.PlacementRoleLeader:
rule.Role = placement.Leader
case ast.PlacementRoleLearner:
rule.Role = placement.Learner
case ast.PlacementRoleVoter:
rule.Role = placement.Voter
switch spec.Tp {
case ast.PlacementAdd:
rule.Action = placement.RuleOpAdd
case ast.PlacementAlter:
rule.Action = placement.RuleOpAdd

// alter will overwrite all things
// drop all rules that will be overridden
newRules := rules[:0]

for _, r := range rules {
if r.Role != rule.Role {
newRules = append(newRules, r)
}
}

rules = newRules

// delete previous definitions
rules = append(rules, &placement.RuleOp{
Action: placement.RuleOpDel,
DeleteByIDPrefix: true,
Rule: &placement.Rule{
GroupID: placement.RuleDefaultGroupID,
// ROLE is useless for PD, prevent two alter statements from overriding each other
Role: rule.Role,
},
})
default:
err = errors.Errorf("unknown role: %d", spec.Role)
err = errors.Errorf("unknown action type: %d", spec.Tp)
}
}

if err == nil {
rules, err = checkPlacementSpecConstraint(rules, rule, spec.Constraints)
var newRules []*placement.RuleOp
newRules, err = buildPlacementSpecReplicasAndConstraint(rule, spec.Replicas, spec.Constraints)
rules = append(rules, newRules...)
}

if err != nil {
Expand Down Expand Up @@ -5327,15 +5316,15 @@ func (d *ddl) AlterTablePartition(ctx sessionctx.Context, ident ast.Ident, spec
return errors.Trace(err)
}

rules, err := checkPlacementSpecs(spec.PlacementSpecs)
rules, err := buildPlacementSpecs(spec.PlacementSpecs)
if err != nil {
return errors.Trace(err)
}

startKey := hex.EncodeToString(codec.EncodeBytes(nil, tablecodec.GenTablePrefix(partitionID)))
endKey := hex.EncodeToString(codec.EncodeBytes(nil, tablecodec.GenTablePrefix(partitionID+1)))
for _, rule := range rules {
rule.Index = placementRuleIndexPartition
rule.Index = placement.RuleIndexPartition
rule.StartKeyHex = startKey
rule.EndKeyHex = endKey
}
Expand Down
10 changes: 7 additions & 3 deletions ddl/partition.go
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,7 @@ import (
"github.com/pingcap/parser/model"
"github.com/pingcap/parser/mysql"
"github.com/pingcap/parser/opcode"
"github.com/pingcap/pd/v4/server/schedule/placement"
"github.com/pingcap/tidb/ddl/placement"
"github.com/pingcap/tidb/ddl/util"
"github.com/pingcap/tidb/domain/infosync"
"github.com/pingcap/tidb/expression"
Expand Down Expand Up @@ -1451,7 +1451,7 @@ func truncateTableByReassignPartitionIDs(t *meta.Meta, tblInfo *model.TableInfo)

func onAlterTablePartition(t *meta.Meta, job *model.Job) (int64, error) {
var partitionID int64
var rules []*placement.Rule
var rules []*placement.RuleOp
err := job.DecodeArgs(&partitionID, &rules)
if err != nil {
job.State = model.JobStateCancelled
Expand All @@ -1470,7 +1470,11 @@ func onAlterTablePartition(t *meta.Meta, job *model.Job) (int64, error) {
}

for i, rule := range rules {
rule.ID = fmt.Sprintf("%d_%d_%d_%d_%d", job.SchemaID, tblInfo.ID, partitionID, job.ID, i)
if rule.Action == placement.RuleOpDel {
rule.ID = fmt.Sprintf("%d_t%d_p%d_%s", job.SchemaID, tblInfo.ID, partitionID, rule.Role)
} else {
rule.ID = fmt.Sprintf("%d_t%d_p%d_%s_%d_%d", job.SchemaID, tblInfo.ID, partitionID, rule.Role, job.ID, i)
}
}

ver, err := t.GetSchemaVersion()
Expand Down
31 changes: 31 additions & 0 deletions ddl/placement/const.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,31 @@
// 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

// RuleDefaultGroupID is the default GroupID for all placement rules, to
// indicate that it is from TiDB_DDL statements.
const RuleDefaultGroupID = "TiDB_DDL"

const (
// RuleIndexDefault is the default index for a rule, check Rule.Index.
RuleIndexDefault int = iota
// RuleIndexDatabase is the index for a rule of database.
RuleIndexDatabase
// RuleIndexTable is the index for a rule of table.
RuleIndexTable
// RuleIndexPartition is the index for a rule of partition.
RuleIndexPartition
// RuleIndexIndex is the index for a rule of index.
RuleIndexIndex
)
Loading

0 comments on commit a016061

Please sign in to comment.