Skip to content

Commit

Permalink
Merge branch 'master' into hanfei/tiflash-concur-factor
Browse files Browse the repository at this point in the history
  • Loading branch information
fzhedu authored Apr 23, 2021
2 parents 5190daa + 5c338e2 commit 429f12b
Show file tree
Hide file tree
Showing 150 changed files with 2,653 additions and 1,549 deletions.
1 change: 1 addition & 0 deletions Makefile
Original file line number Diff line number Diff line change
Expand Up @@ -54,6 +54,7 @@ check-static: tools/bin/golangci-lint
--enable=ineffassign \
--enable=typecheck \
--enable=varcheck \
--enable=unused \
--enable=structcheck \
--enable=deadcode \
$$($(PACKAGE_DIRECTORIES))
Expand Down
1 change: 1 addition & 0 deletions config/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -208,6 +208,7 @@ func (c *Config) getTiKVConfig() *tikvcfg.Config {
OpenTracingEnable: c.OpenTracing.Enable,
Path: c.Path,
EnableForwarding: c.EnableForwarding,
TxnScope: c.Labels["zone"],
}
}

Expand Down
19 changes: 2 additions & 17 deletions config/config_util.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,7 @@ import (

"github.com/BurntSushi/toml"
"github.com/pingcap/errors"
"github.com/pingcap/failpoint"
tikvcfg "github.com/pingcap/tidb/store/tikv/config"
)

// CloneConf deeply clones this config.
Expand Down Expand Up @@ -156,22 +156,7 @@ func flatten(flatMap map[string]interface{}, nested interface{}, prefix string)
}
}

const (
globalTxnScope = "global"
)

// GetTxnScopeFromConfig extracts @@txn_scope value from config
func GetTxnScopeFromConfig() (bool, string) {
failpoint.Inject("injectTxnScope", func(val failpoint.Value) {
v := val.(string)
if len(v) > 0 {
failpoint.Return(false, v)
}
failpoint.Return(true, globalTxnScope)
})
v, ok := GetGlobalConfig().Labels["zone"]
if ok && len(v) > 0 {
return false, v
}
return true, globalTxnScope
return tikvcfg.GetTxnScopeFromConfig()
}
19 changes: 0 additions & 19 deletions config/config_util_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,6 @@ import (

"github.com/BurntSushi/toml"
. "github.com/pingcap/check"
"github.com/pingcap/failpoint"
)

func (s *testConfigSuite) TestCloneConf(c *C) {
Expand Down Expand Up @@ -169,21 +168,3 @@ engines = ["tikv", "tiflash", "tidb"]
c.Assert(toJSONStr(flatMap["log.format"]), Equals, `"text"`)
c.Assert(toJSONStr(flatMap["isolation-read.engines"]), Equals, `["tikv","tiflash","tidb"]`)
}

func (s *testConfigSuite) TestTxnScopeValue(c *C) {
failpoint.Enable("github.com/pingcap/tidb/config/injectTxnScope", `return("bj")`)
isGlobal, v := GetTxnScopeFromConfig()
c.Assert(isGlobal, IsFalse)
c.Assert(v, Equals, "bj")
failpoint.Disable("github.com/pingcap/tidb/config/injectTxnScope")
failpoint.Enable("github.com/pingcap/tidb/config/injectTxnScope", `return("")`)
isGlobal, v = GetTxnScopeFromConfig()
c.Assert(isGlobal, IsTrue)
c.Assert(v, Equals, "global")
failpoint.Disable("github.com/pingcap/tidb/config/injectTxnScope")
failpoint.Enable("github.com/pingcap/tidb/config/injectTxnScope", `return("global")`)
isGlobal, v = GetTxnScopeFromConfig()
c.Assert(isGlobal, IsFalse)
c.Assert(v, Equals, "global")
failpoint.Disable("github.com/pingcap/tidb/config/injectTxnScope")
}
7 changes: 5 additions & 2 deletions ddl/db_integration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -43,6 +43,7 @@ import (
"github.com/pingcap/tidb/store/tikv/mockstore/cluster"
"github.com/pingcap/tidb/store/tikv/oracle"
"github.com/pingcap/tidb/table"
"github.com/pingcap/tidb/tablecodec"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/collate"
"github.com/pingcap/tidb/util/israce"
Expand Down Expand Up @@ -1202,7 +1203,8 @@ func (s *testIntegrationSuite5) TestBackwardCompatibility(c *C) {
c.Assert(err, IsNil)

// Split the table.
s.cluster.SplitTable(tbl.Meta().ID, 100)
tableStart := tablecodec.GenTableRecordPrefix(tbl.Meta().ID)
s.cluster.SplitKeys(tableStart, tableStart.PrefixNext(), 100)

unique := false
indexName := model.NewCIStr("idx_b")
Expand Down Expand Up @@ -1280,7 +1282,8 @@ func (s *testIntegrationSuite3) TestMultiRegionGetTableEndHandle(c *C) {
testCtx := newTestMaxTableRowIDContext(c, d, tbl)

// Split the table.
s.cluster.SplitTable(tblID, 100)
tableStart := tablecodec.GenTableRecordPrefix(tblID)
s.cluster.SplitKeys(tableStart, tableStart.PrefixNext(), 100)

maxHandle, emptyTable := getMaxTableHandle(testCtx, s.store)
c.Assert(emptyTable, IsFalse)
Expand Down
20 changes: 20 additions & 0 deletions ddl/db_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -6663,6 +6663,26 @@ func (s *testSerialSuite) TestTruncateAllPartitions(c *C) {
tk1.MustQuery("select count(*) from partition_table;").Check(testkit.Rows("0"))
}

func (s *testSerialSuite) TestIssue23872(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test;")
tk.MustExec("drop table if exists test_create_table;")
defer tk.MustExec("drop table if exists test_create_table;")
tk.MustExec("create table test_create_table(id smallint,id1 int, primary key (id));")
rs, err := tk.Exec("select * from test_create_table;")
c.Assert(err, IsNil)
cols := rs.Fields()
expectFlag := uint16(mysql.NotNullFlag | mysql.PriKeyFlag | mysql.NoDefaultValueFlag)
c.Assert(cols[0].Column.Flag, Equals, uint(expectFlag))
tk.MustExec("create table t(a int default 1, primary key(a));")
defer tk.MustExec("drop table if exists t;")
rs1, err := tk.Exec("select * from t;")
c.Assert(err, IsNil)
cols1 := rs1.Fields()
expectFlag1 := uint16(mysql.NotNullFlag | mysql.PriKeyFlag)
c.Assert(cols1[0].Column.Flag, Equals, uint(expectFlag1))
}

// Close issue #23321.
// See https://github.com/pingcap/tidb/issues/23321
func (s *testSerialDBSuite) TestJsonUnmarshalErrWhenPanicInCancellingPath(c *C) {
Expand Down
11 changes: 7 additions & 4 deletions ddl/ddl_api.go
Original file line number Diff line number Diff line change
Expand Up @@ -259,6 +259,7 @@ func setColumnFlagWithConstraint(colMap map[string]*table.Column, v *ast.Constra
c.Flag |= mysql.PriKeyFlag
// Primary key can not be NULL.
c.Flag |= mysql.NotNullFlag
setNoDefaultValueFlag(c, c.DefaultValue != nil)
}
case ast.ConstraintUniq, ast.ConstraintUniqIndex, ast.ConstraintUniqKey:
for i, key := range v.Keys {
Expand Down Expand Up @@ -5893,7 +5894,7 @@ func buildPlacementSpecReplicasAndConstraint(replicas uint64, cnstr string) ([]*
return rules, errors.Errorf("array CONSTRAINTS should be with a positive REPLICAS")
}

labelConstraints, err := placement.CheckLabelConstraints(constraints1)
labelConstraints, err := placement.NewConstraints(constraints1)
if err != nil {
return rules, err
}
Expand Down Expand Up @@ -5923,7 +5924,7 @@ func buildPlacementSpecReplicasAndConstraint(replicas uint64, cnstr string) ([]*
}
}

labelConstraints, err := placement.CheckLabelConstraints(strings.Split(strings.TrimSpace(labels), ","))
labelConstraints, err := placement.NewConstraints(strings.Split(strings.TrimSpace(labels), ","))
if err != nil {
return rules, err
}
Expand Down Expand Up @@ -6062,11 +6063,13 @@ func (d *ddl) AlterTableAlterPartition(ctx sessionctx.Context, ident ast.Ident,
// 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
rule.LabelConstraints = append(rule.LabelConstraints, placement.Constraint{
if err := rule.LabelConstraints.Add(placement.Constraint{
Op: placement.NotIn,
Key: placement.EngineLabelKey,
Values: []string{placement.EngineLabelTiFlash},
})
}); err != nil {
return errors.Trace(err)
}
rule.GroupID = bundle.ID
rule.ID = strconv.Itoa(i)
rule.StartKeyHex = startKey
Expand Down
7 changes: 5 additions & 2 deletions ddl/failtest/fail_db_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,7 @@ import (
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/store/mockstore"
"github.com/pingcap/tidb/store/tikv/mockstore/cluster"
"github.com/pingcap/tidb/tablecodec"
"github.com/pingcap/tidb/util/logutil"
"github.com/pingcap/tidb/util/testkit"
"github.com/pingcap/tidb/util/testleak"
Expand Down Expand Up @@ -237,7 +238,8 @@ func (s *testFailDBSuite) TestAddIndexFailed(c *C) {
tblID := tbl.Meta().ID

// Split the table.
s.cluster.SplitTable(tblID, 100)
tableStart := tablecodec.GenTableRecordPrefix(tblID)
s.cluster.SplitKeys(tableStart, tableStart.PrefixNext(), 100)

tk.MustExec("alter table t add index idx_b(b)")
tk.MustExec("admin check index t idx_b")
Expand Down Expand Up @@ -375,7 +377,8 @@ func (s *testFailDBSuite) TestAddIndexWorkerNum(c *C) {

splitCount := 100
// Split table to multi region.
s.cluster.SplitTable(tbl.Meta().ID, splitCount)
tableStart := tablecodec.GenTableRecordPrefix(tbl.Meta().ID)
s.cluster.SplitKeys(tableStart, tableStart.PrefixNext(), splitCount)

err = ddlutil.LoadDDLReorgVars(tk.Se)
c.Assert(err, IsNil)
Expand Down
88 changes: 88 additions & 0 deletions ddl/placement/constraints.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,88 @@
// 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"
)

// Constraints is a slice of constraints.
type Constraints []Constraint

// NewConstraints will check labels, and build Constraints for rule.
func NewConstraints(labels []string) (Constraints, error) {
constraints := make(Constraints, 0, len(labels))
for _, str := range labels {
label, err := NewConstraint(strings.TrimSpace(str))
if err != nil {
return constraints, err
}

err = constraints.Add(label)
if err != nil {
return constraints, err
}
}
return constraints, nil
}

// Restore converts label constraints to a string.
func (constraints *Constraints) Restore() (string, error) {
var sb strings.Builder
for i, constraint := range *constraints {
if i > 0 {
sb.WriteByte(',')
}
sb.WriteByte('"')
conStr, err := constraint.Restore()
if err != nil {
return "", err
}
sb.WriteString(conStr)
sb.WriteByte('"')
}
return sb.String(), nil
}

// Add will add a new label constraint, with validation of all constraints.
// Note that Add does not validate one single constraint.
func (constraints *Constraints) Add(label Constraint) error {
pass := true

for _, cnst := range *constraints {
res := label.CompatibleWith(&cnst)
if res == ConstraintCompatible {
continue
}
if res == ConstraintDuplicated {
pass = false
continue
}
s1, err := label.Restore()
if err != nil {
s1 = err.Error()
}
s2, err := cnst.Restore()
if err != nil {
s2 = err.Error()
}
return fmt.Errorf("%w: '%s' and '%s'", ErrConflictingConstraints, s1, s2)
}

if pass {
*constraints = append(*constraints, label)
}
return nil
}
Loading

0 comments on commit 429f12b

Please sign in to comment.