diff --git a/Makefile b/Makefile index 5683ab4c9361f..dd0e5f521acab 100644 --- a/Makefile +++ b/Makefile @@ -54,6 +54,7 @@ check-static: tools/bin/golangci-lint --enable=ineffassign \ --enable=typecheck \ --enable=varcheck \ + --enable=unused \ --enable=structcheck \ --enable=deadcode \ $$($(PACKAGE_DIRECTORIES)) diff --git a/config/config.go b/config/config.go index 15dbaabb6bfe7..7d6560b5d6783 100644 --- a/config/config.go +++ b/config/config.go @@ -208,6 +208,7 @@ func (c *Config) getTiKVConfig() *tikvcfg.Config { OpenTracingEnable: c.OpenTracing.Enable, Path: c.Path, EnableForwarding: c.EnableForwarding, + TxnScope: c.Labels["zone"], } } diff --git a/config/config_util.go b/config/config_util.go index ce54255369507..e115808426fa9 100644 --- a/config/config_util.go +++ b/config/config_util.go @@ -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. @@ -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() } diff --git a/config/config_util_test.go b/config/config_util_test.go index 3ed621758f757..7972fcf706000 100644 --- a/config/config_util_test.go +++ b/config/config_util_test.go @@ -23,7 +23,6 @@ import ( "github.com/BurntSushi/toml" . "github.com/pingcap/check" - "github.com/pingcap/failpoint" ) func (s *testConfigSuite) TestCloneConf(c *C) { @@ -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") -} diff --git a/ddl/db_integration_test.go b/ddl/db_integration_test.go index d73ef6c0e6850..fe6bca7dc4563 100644 --- a/ddl/db_integration_test.go +++ b/ddl/db_integration_test.go @@ -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" @@ -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") @@ -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) diff --git a/ddl/db_test.go b/ddl/db_test.go index d58340878c780..7970b14989aaf 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -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) { diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 7e4e03f96316e..6520bc77dbba9 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -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 { @@ -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 } @@ -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 } @@ -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 diff --git a/ddl/failtest/fail_db_test.go b/ddl/failtest/fail_db_test.go index 805ee67154c54..78d6ad9decfd6 100644 --- a/ddl/failtest/fail_db_test.go +++ b/ddl/failtest/fail_db_test.go @@ -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" @@ -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") @@ -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) diff --git a/ddl/placement/constraints.go b/ddl/placement/constraints.go new file mode 100644 index 0000000000000..d759bad89a701 --- /dev/null +++ b/ddl/placement/constraints.go @@ -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 +} diff --git a/ddl/placement/constraints_test.go b/ddl/placement/constraints_test.go new file mode 100644 index 0000000000000..678ab2ee997ed --- /dev/null +++ b/ddl/placement/constraints_test.go @@ -0,0 +1,169 @@ +// 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 ( + "errors" + + . "github.com/pingcap/check" +) + +var _ = Suite(&testConstraintsSuite{}) + +type testConstraintsSuite struct{} + +func (t *testConstraintsSuite) TestNew(c *C) { + _, err := NewConstraints(nil) + c.Assert(err, IsNil) + + _, err = NewConstraints([]string{}) + c.Assert(err, IsNil) + + _, err = NewConstraints([]string{"+zonesh"}) + c.Assert(errors.Is(err, ErrInvalidConstraintFormat), IsTrue) + + _, err = NewConstraints([]string{"+zone=sh", "-zone=sh"}) + c.Assert(errors.Is(err, ErrConflictingConstraints), IsTrue) +} + +func (t *testConstraintsSuite) TestAdd(c *C) { + type TestCase struct { + name string + labels Constraints + label Constraint + err error + } + var tests []TestCase + + labels, err := NewConstraints([]string{"+zone=sh"}) + c.Assert(err, IsNil) + label, err := NewConstraint("-zone=sh") + c.Assert(err, IsNil) + tests = append(tests, TestCase{ + "always false match", + labels, label, + ErrConflictingConstraints, + }) + + labels, err = NewConstraints([]string{"+zone=sh"}) + c.Assert(err, IsNil) + label, err = NewConstraint("+zone=sh") + c.Assert(err, IsNil) + tests = append(tests, TestCase{ + "duplicated constraints, skip", + labels, label, + nil, + }) + + tests = append(tests, TestCase{ + "duplicated constraints should not stop conflicting constraints check", + append(labels, Constraint{ + Op: NotIn, + Key: "zone", + Values: []string{"sh"}, + }), label, + ErrConflictingConstraints, + }) + + labels, err = NewConstraints([]string{"+zone=sh"}) + c.Assert(err, IsNil) + tests = append(tests, TestCase{ + "invalid label in operand", + labels, Constraint{Op: "["}, + nil, + }) + + tests = append(tests, TestCase{ + "invalid label in operator", + Constraints{{Op: "["}}, label, + nil, + }) + + tests = append(tests, TestCase{ + "invalid label in both, same key", + Constraints{{Op: "[", Key: "dc"}}, Constraint{Op: "]", Key: "dc"}, + ErrConflictingConstraints, + }) + + labels, err = NewConstraints([]string{"+zone=sh"}) + c.Assert(err, IsNil) + label, err = NewConstraint("-zone=bj") + c.Assert(err, IsNil) + tests = append(tests, TestCase{ + "normal", + labels, label, + nil, + }) + + for _, t := range tests { + err := t.labels.Add(t.label) + comment := Commentf("%s: %v", t.name, err) + if t.err == nil { + c.Assert(err, IsNil, comment) + c.Assert(t.labels[len(t.labels)-1], DeepEquals, t.label, comment) + } else { + c.Assert(errors.Is(err, t.err), IsTrue, comment) + } + } +} + +func (t *testConstraintsSuite) TestRestore(c *C) { + type TestCase struct { + name string + input Constraints + output string + err error + } + var tests []TestCase + + tests = append(tests, TestCase{ + "normal1", + Constraints{}, + "", + nil, + }) + + input1, err := NewConstraint("+zone=bj") + c.Assert(err, IsNil) + input2, err := NewConstraint("-zone=sh") + c.Assert(err, IsNil) + tests = append(tests, TestCase{ + "normal2", + Constraints{input1, input2}, + `"+zone=bj","-zone=sh"`, + nil, + }) + + tests = append(tests, TestCase{ + "error", + Constraints{{ + Op: "[", + Key: "dc", + Values: []string{"dc1"}, + }}, + "", + ErrInvalidConstraintFormat, + }) + + for _, t := range tests { + res, err := t.input.Restore() + comment := Commentf("%s: %v", t.name, err) + if t.err == nil { + c.Assert(err, IsNil, comment) + c.Assert(res, Equals, t.output, comment) + } else { + c.Assert(errors.Is(err, t.err), IsTrue, comment) + } + } +} diff --git a/ddl/placement/errors.go b/ddl/placement/errors.go index ea079ce0dbdde..19797022a609c 100644 --- a/ddl/placement/errors.go +++ b/ddl/placement/errors.go @@ -22,4 +22,6 @@ var ( ErrInvalidConstraintFormat = errors.New("label constraint should be in format '{+|-}key=value'") // ErrUnsupportedConstraint is from constraint.go. ErrUnsupportedConstraint = errors.New("unsupported label constraint") + // ErrConflictingConstraints is from constraints.go. + ErrConflictingConstraints = errors.New("conflicting label constraints") ) diff --git a/ddl/placement/types.go b/ddl/placement/types.go index f03c84c5694f8..3bb9da96e3890 100644 --- a/ddl/placement/types.go +++ b/ddl/placement/types.go @@ -46,7 +46,7 @@ type Rule struct { EndKeyHex string `json:"end_key"` Role PeerRoleType `json:"role"` Count int `json:"count"` - LabelConstraints []Constraint `json:"label_constraints,omitempty"` + LabelConstraints Constraints `json:"label_constraints,omitempty"` LocationLabels []string `json:"location_labels,omitempty"` IsolationLevel string `json:"isolation_level,omitempty"` } diff --git a/ddl/placement/utils.go b/ddl/placement/utils.go index ca6467f4daa8c..16c0a424dde53 100644 --- a/ddl/placement/utils.go +++ b/ddl/placement/utils.go @@ -23,49 +23,6 @@ import ( "github.com/pingcap/tidb/util/codec" ) -// CheckLabelConstraints will check labels, and build Constraints for rule. -func CheckLabelConstraints(labels []string) ([]Constraint, error) { - constraints := make([]Constraint, 0, len(labels)) - for _, str := range labels { - label, err := NewConstraint(strings.TrimSpace(str)) - if err != nil { - return constraints, err - } - - pass := true - - for _, cnst := range constraints { - if label.Key == cnst.Key { - sameOp := label.Op == cnst.Op - sameVal := label.Values[0] == cnst.Values[0] - // no following cases: - // 1. duplicated constraint - // 2. no instance can meet: +dc=sh, -dc=sh - // 3. can not match multiple instances: +dc=sh, +dc=bj - if sameOp && sameVal { - pass = false - break - } else if (!sameOp && sameVal) || (sameOp && !sameVal && label.Op == In) { - s1, err := label.Restore() - if err != nil { - s1 = err.Error() - } - s2, err := cnst.Restore() - if err != nil { - s2 = err.Error() - } - return constraints, errors.Errorf("conflicting constraints '%s' and '%s'", s1, s2) - } - } - } - - if pass { - constraints = append(constraints, label) - } - } - return constraints, nil -} - // ObjectIDFromGroupID extracts the db/table/partition ID from the group ID func ObjectIDFromGroupID(groupID string) (int64, error) { // If the rule doesn't come from TiDB, skip it. @@ -79,24 +36,6 @@ func ObjectIDFromGroupID(groupID string) (int64, error) { return id, nil } -// RestoreLabelConstraintList converts the label constraints to a readable string. -func RestoreLabelConstraintList(constraints []Constraint) (string, error) { - var sb strings.Builder - for i, constraint := range constraints { - sb.WriteByte('"') - conStr, err := constraint.Restore() - if err != nil { - return "", err - } - sb.WriteString(conStr) - sb.WriteByte('"') - if i < len(constraints)-1 { - sb.WriteByte(',') - } - } - return sb.String(), nil -} - // BuildPlacementDropBundle builds the bundle to drop placement rules. func BuildPlacementDropBundle(partitionID int64) *Bundle { return &Bundle{ diff --git a/ddl/placement/utils_test.go b/ddl/placement/utils_test.go index 0e2f85a2064f7..964382846485e 100644 --- a/ddl/placement/utils_test.go +++ b/ddl/placement/utils_test.go @@ -21,82 +21,6 @@ var _ = Suite(&testUtilsSuite{}) type testUtilsSuite struct{} -func (t *testUtilsSuite) TestRestoreConstraints(c *C) { - testCases := []struct { - constraints []Constraint - expectedResult string - expectErr bool - }{ - { - constraints: []Constraint{}, - expectedResult: ``, - }, - { - constraints: []Constraint{ - { - Key: "zone", - Op: "in", - Values: []string{"bj"}, - }, - }, - expectedResult: `"+zone=bj"`, - }, - { - constraints: []Constraint{ - { - Key: "zone", - Op: "notIn", - Values: []string{"bj"}, - }, - }, - expectedResult: `"-zone=bj"`, - }, - { - constraints: []Constraint{ - { - Key: "zone", - Op: "exists", - Values: []string{"bj"}, - }, - }, - expectErr: true, - }, - { - constraints: []Constraint{ - { - Key: "zone", - Op: "in", - Values: []string{"bj", "sh"}, - }, - }, - expectErr: true, - }, - { - constraints: []Constraint{ - { - Key: "zone", - Op: "in", - Values: []string{"bj"}, - }, - { - Key: "disk", - Op: "in", - Values: []string{"ssd"}, - }, - }, - expectedResult: `"+zone=bj","+disk=ssd"`, - }, - } - for _, testCase := range testCases { - rs, err := RestoreLabelConstraintList(testCase.constraints) - if testCase.expectErr { - c.Assert(err, NotNil) - } else { - c.Assert(rs, Equals, testCase.expectedResult) - } - } -} - func (t *testUtilsSuite) TestObjectIDFromGroupID(c *C) { testCases := []struct { bundleID string diff --git a/ddl/placement_rule_test.go b/ddl/placement_rule_test.go index 671955786c977..b051092a776e9 100644 --- a/ddl/placement_rule_test.go +++ b/ddl/placement_rule_test.go @@ -59,26 +59,6 @@ func (s *testPlacementSuite) TestPlacementBuild(c *C) { }, }, - { - input: []*ast.PlacementSpec{{ - Role: ast.PlacementRoleVoter, - Tp: ast.PlacementAdd, - Replicas: 3, - Constraints: `["+zone=sh", "+engine=tiflash"]`, - }}, - err: ".*unsupported label constraint.*", - }, - - { - input: []*ast.PlacementSpec{{ - Role: ast.PlacementRoleVoter, - Tp: ast.PlacementAdd, - Replicas: 3, - Constraints: `["+zone=sh", "+engine=TiFlash"]`, - }}, - err: ".*unsupported label constraint.*", - }, - { input: []*ast.PlacementSpec{{ Role: ast.PlacementRoleVoter, @@ -93,26 +73,6 @@ func (s *testPlacementSuite) TestPlacementBuild(c *C) { }}, }, - { - input: []*ast.PlacementSpec{{ - Role: ast.PlacementRoleVoter, - Tp: ast.PlacementAdd, - Replicas: 3, - Constraints: `["+zone=sh", "-zone=sh"]`, - }}, - err: ".*conflicting constraints.*", - }, - - { - input: []*ast.PlacementSpec{{ - Role: ast.PlacementRoleVoter, - Tp: ast.PlacementAdd, - Replicas: 3, - Constraints: `["+zone=sh", "+zone=bj"]`, - }}, - err: ".*conflicting constraints.*", - }, - { input: []*ast.PlacementSpec{{ Role: ast.PlacementRoleVoter, @@ -137,24 +97,6 @@ func (s *testPlacementSuite) TestPlacementBuild(c *C) { }, }, - { - input: []*ast.PlacementSpec{{ - Role: ast.PlacementRoleVoter, - Tp: ast.PlacementAdd, - Constraints: `{"+zone=sh,-zone=sh": 2, "+zone=sh": 1}`, - }}, - err: ".*conflicting constraints.*", - }, - - { - input: []*ast.PlacementSpec{{ - Role: ast.PlacementRoleVoter, - Tp: ast.PlacementAdd, - Constraints: `{"+zone=sh,+zone=bj": 2, "+zone=sh": 1}`, - }}, - err: ".*conflicting constraints.*", - }, - { input: []*ast.PlacementSpec{{ Role: ast.PlacementRoleVoter, diff --git a/ddl/placement_sql_test.go b/ddl/placement_sql_test.go index 6ce013b8323f2..e77b0ba99d5cf 100644 --- a/ddl/placement_sql_test.go +++ b/ddl/placement_sql_test.go @@ -129,7 +129,7 @@ drop placement policy _, err = tk.Exec(`alter table t1 alter partition p0 drop placement policy role=follower`) - c.Assert(err, NotNil) + c.Assert(err, ErrorMatches, ".*no rule of role 'follower' to drop.*") _, err = tk.Exec(`alter table t1 alter partition p0 add placement policy @@ -202,7 +202,7 @@ drop placement policy role=leader, drop placement policy role=leader`) - c.Assert(err, NotNil) + c.Assert(err, ErrorMatches, ".*no rule of role 'leader' to drop.*") s.dom.InfoSchema().SetBundle(bundle) _, err = tk.Exec(`alter table t1 alter partition p0 @@ -261,67 +261,6 @@ add placement policy role=leader`) c.Assert(err, ErrorMatches, ".*should be larger or equal to the number of total replicas.*") - // checkPlacementSpecConstraint - _, err = tk.Exec(`alter table t1 alter partition p0 -add placement policy - constraints='[",,,"]' - role=follower - replicas=3`) - c.Assert(err, ErrorMatches, ".*label constraint should be in format.*") - - _, err = tk.Exec(`alter table t1 alter partition p0 -add placement policy - constraints='["+ "]' - role=follower - replicas=3`) - c.Assert(err, ErrorMatches, ".*label constraint should be in format.*") - - // unknown operation - _, err = tk.Exec(`alter table t1 alter partition p0 -add placement policy - constraints='["0000"]' - role=follower - replicas=3`) - c.Assert(err, ErrorMatches, ".*label constraint should be in format.*") - - // without = - _, err = tk.Exec(`alter table t1 alter partition p0 -add placement policy - constraints='["+000"]' - role=follower - replicas=3`) - c.Assert(err, ErrorMatches, ".*label constraint should be in format.*") - - // empty key - _, err = tk.Exec(`alter table t1 alter partition p0 -add placement policy - constraints='["+ =zone1"]' - role=follower - replicas=3`) - c.Assert(err, ErrorMatches, ".*label constraint should be in format.*") - - _, err = tk.Exec(`alter table t1 alter partition p0 -add placement policy - constraints='["+ = z"]' - role=follower - replicas=3`) - c.Assert(err, ErrorMatches, ".*label constraint should be in format.*") - - // empty value - _, err = tk.Exec(`alter table t1 alter partition p0 -add placement policy - constraints='["+zone="]' - role=follower - replicas=3`) - c.Assert(err, ErrorMatches, ".*label constraint should be in format.*") - - _, err = tk.Exec(`alter table t1 alter partition p0 -add placement policy - constraints='["+z = "]' - role=follower - replicas=3`) - c.Assert(err, ErrorMatches, ".*label constraint should be in format.*") - _, err = tk.Exec(`alter table t1 alter partition p add placement policy constraints='["+zone=sh"]' @@ -343,14 +282,6 @@ add placement policy replicas=0`) c.Assert(err, ErrorMatches, ".*Invalid placement option REPLICAS, it is not allowed to be 0.*") - // ban tiflash - _, err = tk.Exec(`alter table t1 alter partition p0 -add placement policy - constraints='["+zone=sh", "+engine=tiflash"]' - role=follower - replicas=3`) - c.Assert(err, ErrorMatches, ".*unsupported label.*") - // invalid partition tk.MustExec("drop table if exists t1") tk.MustExec("create table t1 (c int)") @@ -566,7 +497,7 @@ PARTITION BY RANGE (c) ( for _, testcase := range testCases { c.Log(testcase.name) - failpoint.Enable("github.com/pingcap/tidb/config/injectTxnScope", + failpoint.Enable("github.com/pingcap/tidb/store/tikv/config/injectTxnScope", fmt.Sprintf(`return("%v")`, testcase.zone)) se, err := session.CreateSession4Test(s.store) c.Check(err, IsNil) @@ -587,7 +518,7 @@ PARTITION BY RANGE (c) ( c.Assert(err, NotNil) c.Assert(err.Error(), Matches, testcase.err.Error()) } - failpoint.Disable("github.com/pingcap/tidb/config/injectTxnScope") + failpoint.Disable("github.com/pingcap/tidb/store/tikv/config/injectTxnScope") } } @@ -698,8 +629,8 @@ PARTITION BY RANGE (c) ( }, }, } - failpoint.Enable("github.com/pingcap/tidb/config/injectTxnScope", `return("bj")`) - defer failpoint.Disable("github.com/pingcap/tidb/config/injectTxnScope") + failpoint.Enable("github.com/pingcap/tidb/store/tikv/config/injectTxnScope", `return("bj")`) + defer failpoint.Disable("github.com/pingcap/tidb/store/tikv/config/injectTxnScope") dbInfo := testGetSchemaByName(c, tk.Se, "test") tk2 := testkit.NewTestKit(c, s.store) var chkErr error diff --git a/ddl/serial_test.go b/ddl/serial_test.go index 637fbc46bb037..e532bfc2352af 100644 --- a/ddl/serial_test.go +++ b/ddl/serial_test.go @@ -42,6 +42,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/admin" "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/gcutil" @@ -211,7 +212,8 @@ func (s *testIntegrationSuite9) 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) @@ -347,7 +349,8 @@ func (s *testIntegrationSuite9) TestMultiRegionGetTableEndCommonHandle(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) diff --git a/distsql/distsql.go b/distsql/distsql.go index 83236ce4dc5f4..b6bb0038ca712 100644 --- a/distsql/distsql.go +++ b/distsql/distsql.go @@ -134,7 +134,7 @@ func SelectWithRuntimeStats(ctx context.Context, sctx sessionctx.Context, kvReq } // Analyze do a analyze request. -func Analyze(ctx context.Context, client kv.Client, kvReq *kv.Request, vars *kv.Variables, +func Analyze(ctx context.Context, client kv.Client, kvReq *kv.Request, vars interface{}, isRestrict bool, sessionMemTracker *memory.Tracker) (SelectResult, error) { resp := client.Send(ctx, kvReq, vars, sessionMemTracker, false) if resp == nil { @@ -156,7 +156,7 @@ func Analyze(ctx context.Context, client kv.Client, kvReq *kv.Request, vars *kv. } // Checksum sends a checksum request. -func Checksum(ctx context.Context, client kv.Client, kvReq *kv.Request, vars *kv.Variables) (SelectResult, error) { +func Checksum(ctx context.Context, client kv.Client, kvReq *kv.Request, vars interface{}) (SelectResult, error) { // FIXME: As BR have dependency of `Checksum` and TiDB also introduced BR as dependency, Currently we can't edit // Checksum function signature. The two-way dependence should be removed in future. resp := client.Send(ctx, kvReq, vars, nil, false) diff --git a/distsql/distsql_test.go b/distsql/distsql_test.go index 906ecdb7af71b..7f2b5c34ab3b6 100644 --- a/distsql/distsql_test.go +++ b/distsql/distsql_test.go @@ -28,6 +28,7 @@ import ( "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/store/tikv" + tikvstore "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/tikvrpc" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" @@ -327,7 +328,7 @@ func (s *testSuite) TestAnalyze(c *C) { Build() c.Assert(err, IsNil) - response, err := Analyze(context.TODO(), s.sctx.GetClient(), request, kv.DefaultVars, true, s.sctx.GetSessionVars().StmtCtx.MemTracker) + response, err := Analyze(context.TODO(), s.sctx.GetClient(), request, tikvstore.DefaultVars, true, s.sctx.GetSessionVars().StmtCtx.MemTracker) c.Assert(err, IsNil) result, ok := response.(*selectResult) @@ -350,7 +351,7 @@ func (s *testSuite) TestChecksum(c *C) { Build() c.Assert(err, IsNil) - response, err := Checksum(context.TODO(), s.sctx.GetClient(), request, kv.DefaultVars) + response, err := Checksum(context.TODO(), s.sctx.GetClient(), request, tikvstore.DefaultVars) c.Assert(err, IsNil) result, ok := response.(*selectResult) diff --git a/docs/design/2018-10-08-online-DDL.md b/docs/design/2018-10-08-online-DDL.md index 87f746a0d0796..fcaccb26b8cc0 100644 --- a/docs/design/2018-10-08-online-DDL.md +++ b/docs/design/2018-10-08-online-DDL.md @@ -17,7 +17,7 @@ To simplify the design, the entire system allows only one node to make schema ch * **owner**: You can take it as a role. The information (including information of the node currently elected for this role) is stored in the Placement Driver (PD). Only one node in the entire system can be elected as the owner role. Each node can be elected to be this role, and workers will have the right to process jobs only when they are elected as the owner. * **workers**: Each node has 2 workers to process jobs. They will take the job from the corresponding job queue and process it. Workers will periodically check whether there are tasks that need to be processed. If the server receives the request while the owner role is on the same server, the start job module also tells the corresponding worker to process the job. * **meta data**: To simplify the design, we bring in system databases and system tables to record some metadatas in the process of asynchronous schema changes. -* **state**: According to the change process of F1’s asynchronous schema, some states are introduced in this process, which are bound to column, index, table, and database. These states includes none, delete only, write only, write reorganization, and public. The former order is generally the order in which the operations are created. The state of the delete operation is the reverse of the creating order. Change the write reorganization to delete reorganization. Although they are all reorganization states, the visible level is different. So they are divided into two status flags. +* **state**: According to the change process of asynchronous schema, some states are introduced in this process, which are bound to column, index, table, and database. These states includes none, delete only, write only, write reorganization, and public. The former order is generally the order in which the operations are created. The state of the delete operation is the reverse of the creating order. Change the write reorganization to delete reorganization. Although they are all reorganization states, the visible level is different. So they are divided into two status flags. * **lease**: At the same time, there are at most two different versions of the schema on the same table for all nodes of the system. That is, there are at most two different states. As a result, every normal node in a lease will automatically load schema information. If this node cannot be loaded normally during the lease, SQL cannot be processed normally. ## Rationale To dive deep into DDL processing, you need to understand the overall architecture of the DDL module. This module is operated on the TiDB Server, but it also involves the use of two other components of the TiDB cluster (PD Server and TiKV Server). diff --git a/docs/design/README.md b/docs/design/README.md index 5d71bbaa991f8..34fe07a336c6c 100644 --- a/docs/design/README.md +++ b/docs/design/README.md @@ -1,50 +1,35 @@ # TiDB Design Documents -## Why We Need Design Documents - -The design document provides a high-level description of the architecture and important details of what we want to do. It is the most powerful tool to ensure that our work is correctly performed. - -Writing a design document can promote us to think deliberately and gather knowledge from others to get our job done better. An excellent design document is closely related to the success of our task. - -## Proposal Process - -1. Before starting to write a design document, please [create a new issue](https://github.com/pingcap/tidb/issues/new/choose) for tracing the process of your design. -2. Create a new [Google Doc](https://docs.google.com/document/u/0/) to write the design document. Please refer to this [proposal template](./TEMPLATE.md). -3. If the design document is ready to be reviewed, please put the shared link with the `can comment` permission in the issue you've created, and one of our engineers will follow up this issue and keep it updated. -4. If needed, we may organize a communications seminar through [Google Hangouts](https://hangouts.google.com/) to discuss your design with you and the interested friends from the community. -5. When your design is finalized, please submit a pull request (PR) to add your new file under this directory, and put the link of your PR in the issue you've created. -6. Once your PR has been merged, please close the old issue. -7. Start the implementation referring to the proposal, and create a new issue to trace the process. - -## Proposal Status - -### Proposed - -- [Proposal: A new command to restore dropped table](./2018-08-10-restore-dropped-table.md) -- [Proposal: Support SQL Plan Management](./2018-12-11-sql-plan-management.md) - -### In Progress - -- [Proposal: A new storage row format for efficient decoding](./2018-07-19-row-format.md) -- [Proposal: Enhance constraint propagation in TiDB logical plan](./2018-07-22-enhance-propagations.md) -- [Proposal: A SQL Planner based on the Volcano/Cascades model](./2018-08-29-new-planner.md) -- [Proposal: Implement Radix Hash Join](./2018-09-21-radix-hashjoin.md) -- [Proposal: Maintaining histograms in plan](./2018-09-04-histograms-in-plan.md) -- [Proposal: Support a Global Column Pool](./2018-10-22-the-column-pool.md) -- [Proposal: Join Reorder Design v1](./2018-10-20-join-reorder-dp-v1.md) -- [Proposal: Support Window Functions](./2018-10-31-window-functions.md) -- [Proposal: Access a table using multiple indexes](./2019-04-11-indexmerge.md) -- [Proposal: Collations in TiDB](./2020-01-24-collations.md) - -### Completed - -- [Proposal: A new aggregate function execution framework](./2018-07-01-refactor-aggregate-framework.md) -- [Proposal: TiDB DDL architecture](./2018-10-08-online-DDL.md) -- [Proposal: Infer the System Timezone of a TiDB cluster via TZ environment variable](./2018-09-10-adding-tz-env.md) -- [Proposal: Table Partition](./2018-10-19-table-partition.md) -- [Proposal: Implement View Feature](./2018-10-24-view-support.md) -- [Proposal: Support restoring SQL text from an AST tree](./2018-11-29-ast-to-sql-text.md) -- [Proposal: Support Plugin](./2018-12-10-plugin-framework.md) -- [Proposal: Support Skyline Pruning](./2019-01-25-skyline-pruning.md) -- [Proposal: Support Index Merge](./2019-04-11-indexmerge.md) -- [Proposal: Support Automatically Index Recommendation](./2019-11-05-index-advisor.md) +Many changes, including bug fixes and documentation improvements can be implemented and reviewed via the normal GitHub pull request workflow. + +Some changes though are "substantial", and we ask that these be put through a bit of a design process and produce a consensus among the TiDB community. + +The process described in this page is intended to provide a consistent and controlled path for new features to enter the TiDB projects, so that all stakeholders can be confident about the direction the projects is evolving in. + +## Who should initiate the design document? + +Everyone is encouraged to initiate a design document, but before doing it, please make sure you have an intention of getting the work done to implement it. + +## Before creating a design document + +A hastily-proposed design document can hurt its chances of acceptance. Low-quality proposals, proposals for previously-rejected features, or those that don't fit into the near-term roadmap, may be quickly rejected, which can be demotivating for the unprepared contributor. Laying some groundwork ahead of the design document can make the process smoother. + +Although there is no single way to prepare for submitting a design document, it is generally a good idea to pursue feedback from other project developers beforehand, to ascertain that the design document may be desirable; having a consistent impact on the project requires concerted effort toward consensus-building. + +The most common preparations for writing and submitting an design document for now is [creating a disucss issue](https://github.com/pingcap/tidb/issues/new/choose), which is going to be converted into a tracking issue of the design implementation. + +## What is the process? + +1. Create a pull request with a design document based on the [template](./TEMPLATE.md) under this directory as `YYYY-MM-DD-my-feature.md`. +2. Discussion takes place, and the text is revised in response. +3. The design document is accepted or rejected when at least two committers reach consensus and no objection from the committer. +4. If accepted, [create a tracking issue](https://github.com/pingcap/tidb/issues/new/choose) for the design document or convert one from a previous discuss issue. The tracking issue basically tracks subtasks and progress. And refer the tracking issue in the design document replacing placeholder in the template. +5. Merge the pull request of design. + +Please update the tracking issue according to the progress of succeeding implementation pull requests. + +An example that almost fits into this model is the proposal "Support global index for partition table", without following the latest template. + +- Its tracking issue: https://github.com/pingcap/tidb/issues/18032 +- Its pull request: https://github.com/pingcap/tidb/pull/18982 +- Its design document: https://github.com/pingcap/tidb/blob/master/docs/design/2020-08-04-global-index.md diff --git a/docs/design/TEMPLATE.md b/docs/design/TEMPLATE.md index 24a6325f6b088..60779340bd41e 100644 --- a/docs/design/TEMPLATE.md +++ b/docs/design/TEMPLATE.md @@ -1,12 +1,8 @@ - +# TiDB Design Documents -# Proposal: - -- Author(s): -- Last updated: -- Discussion at: +- Author(s): [Author Name](http://github.com/your-github-id), [Co-Author Name](http://github.com/your-github-id), ... +- Discussion PR: https://github.com/pingcap/tidb/pull/XXX +- Tracking Issue: https://github.com/pingcap/tidb/issues/XXX ## Table of Contents @@ -24,22 +20,19 @@ This is a template for TiDB's change proposal process, documented [here](./READM ## Introduction - +One para explanation of the proposal that helps others get the brief info of this design doc. ## Motivation or Background - +What's the background and the problem being solved by this design doc? What use cases does it support? + +Please pay attention to this section. When a design is under review, the reviewers usually do not rush into details, but try to figure out the motivation or background of it. In other words, who wants the feature or what problem the feature solves. ## Detailed Design - ## Test Design - ### Functional Tests - +It's used to ensure the basic feature function works as expected. Both the integration test and the unit test should be considered. ### Scenario Tests - +It's used to ensure this feature works as expected in some common scenarios. ### Compatibility Tests - ### Benchmark Tests - ## Impacts & Risks - ## Investigation & Alternatives - ## Unresolved Questions - diff --git a/domain/domain.go b/domain/domain.go index 793c78497fc05..1e0801c34e51c 100644 --- a/domain/domain.go +++ b/domain/domain.go @@ -1030,6 +1030,11 @@ func (do *Domain) handleEvolvePlanTasksLoop(ctx sessionctx.Context) { // in BootstrapSession. func (do *Domain) TelemetryReportLoop(ctx sessionctx.Context) { ctx.GetSessionVars().InRestrictedSQL = true + err := telemetry.InitialRun(ctx, do.GetEtcdClient()) + if err != nil { + logutil.BgLogger().Warn("Initial telemetry run failed", zap.Error(err)) + } + do.wg.Add(1) go func() { defer func() { diff --git a/executor/aggregate_test.go b/executor/aggregate_test.go index 0c327fb9c28b7..56a7a22f4bdd7 100644 --- a/executor/aggregate_test.go +++ b/executor/aggregate_test.go @@ -1236,10 +1236,18 @@ func (s *testSuiteAgg) TestParallelStreamAggGroupConcat(c *C) { tk.MustExec("use test;") tk.MustExec("drop table if exists t;") tk.MustExec("CREATE TABLE t(a bigint, b bigint);") + tk.MustExec("set tidb_init_chunk_size=1;") + tk.MustExec("set tidb_max_chunk_size=32;") - for i := 0; i < 10000; i++ { - tk.MustExec("insert into t values(?, ?);", rand.Intn(100), rand.Intn(100)) + var insertSQL string + for i := 0; i < 1000; i++ { + if i == 0 { + insertSQL += fmt.Sprintf("(%d, %d)", rand.Intn(100), rand.Intn(100)) + } else { + insertSQL += fmt.Sprintf(",(%d, %d)", rand.Intn(100), rand.Intn(100)) + } } + tk.MustExec(fmt.Sprintf("insert into t values %s;", insertSQL)) sql := "select /*+ stream_agg() */ group_concat(a, b) from t group by b;" concurrencies := []int{1, 2, 4, 8} @@ -1283,9 +1291,17 @@ func (s *testSuiteAgg) TestIssue20658(c *C) { tk.MustExec("drop table if exists t;") tk.MustExec("CREATE TABLE t(a bigint, b bigint);") - for i := 0; i < 10000; i++ { - tk.MustExec("insert into t values (?, ?);", rand.Intn(100), rand.Intn(100)) + tk.MustExec("set tidb_init_chunk_size=1;") + tk.MustExec("set tidb_max_chunk_size=32;") + var insertSQL string + for i := 0; i < 1000; i++ { + if i == 0 { + insertSQL += fmt.Sprintf("(%d, %d)", rand.Intn(100), rand.Intn(100)) + } else { + insertSQL += fmt.Sprintf(",(%d, %d)", rand.Intn(100), rand.Intn(100)) + } } + tk.MustExec(fmt.Sprintf("insert into t values %s;", insertSQL)) concurrencies := []int{1, 2, 4, 8} for _, sql := range sqls { diff --git a/executor/analyze.go b/executor/analyze.go index 14e2fec0cb819..a08145aab0702 100644 --- a/executor/analyze.go +++ b/executor/analyze.go @@ -1226,12 +1226,14 @@ func (e *AnalyzeFastExec) buildColumnStats(ID int64, collector *statistics.Sampl sc := e.ctx.GetSessionVars().StmtCtx data := make([][]byte, 0, len(collector.Samples)) fmSketch := statistics.NewFMSketch(maxSketchSize) + notNullSamples := make([]*statistics.SampleItem, 0, len(collector.Samples)) for i, sample := range collector.Samples { sample.Ordinal = i if sample.Value.IsNull() { collector.NullCount++ continue } + notNullSamples = append(notNullSamples, sample) err := fmSketch.InsertValue(sc, sample.Value) if err != nil { return nil, nil, nil, nil, err @@ -1245,6 +1247,7 @@ func (e *AnalyzeFastExec) buildColumnStats(ID int64, collector *statistics.Sampl // Build CMSketch. cmSketch, topN, ndv, scaleRatio := statistics.NewCMSketchAndTopN(int32(e.opts[ast.AnalyzeOptCMSketchDepth]), int32(e.opts[ast.AnalyzeOptCMSketchWidth]), data, uint32(e.opts[ast.AnalyzeOptNumTopN]), uint64(rowCount)) // Build Histogram. + collector.Samples = notNullSamples hist, err := statistics.BuildColumnHist(e.ctx, int64(e.opts[ast.AnalyzeOptNumBuckets]), ID, collector, tp, rowCount, int64(ndv), collector.NullCount*int64(scaleRatio)) return hist, cmSketch, topN, fmSketch, err } diff --git a/executor/analyze_test.go b/executor/analyze_test.go index fbd7ff0a63173..f8eff902fcb3d 100644 --- a/executor/analyze_test.go +++ b/executor/analyze_test.go @@ -42,6 +42,7 @@ import ( "github.com/pingcap/tidb/store/tikv/mockstore/cluster" "github.com/pingcap/tidb/store/tikv/tikvrpc" "github.com/pingcap/tidb/table" + "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/codec" "github.com/pingcap/tidb/util/collate" @@ -725,7 +726,8 @@ func (s *testFastAnalyze) TestFastAnalyzeRetryRowCount(c *C) { for i := 0; i < 30; i++ { tk.MustExec(fmt.Sprintf("insert into retry_row_count values (%d)", i)) } - cls.SplitTable(tid, 6) + tableStart := tablecodec.GenTableRecordPrefix(tid) + cls.SplitKeys(tableStart, tableStart.PrefixNext(), 6) // Flush the region cache first. tk.MustQuery("select * from retry_row_count") tk.MustExec("analyze table retry_row_count") diff --git a/executor/distsql_test.go b/executor/distsql_test.go index 6d36f06bf34b1..74fc76942e7d6 100644 --- a/executor/distsql_test.go +++ b/executor/distsql_test.go @@ -70,7 +70,8 @@ func (s *testSuite3) TestCopClientSend(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) ctx := context.Background() // Send coprocessor request when the table split. diff --git a/executor/executor.go b/executor/executor.go index 3371ca93442e3..8f9025c22d6f3 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -1550,6 +1550,10 @@ func (e *UnionExec) Next(ctx context.Context, req *chunk.Chunk) error { return errors.Trace(result.err) } + if result.chk.NumCols() != req.NumCols() { + return errors.Errorf("Internal error: UnionExec chunk column count mismatch, req: %d, result: %d", + req.NumCols(), result.chk.NumCols()) + } req.SwapColumns(result.chk) result.src <- result.chk return nil @@ -1646,8 +1650,9 @@ func ResetContextOfStmt(ctx sessionctx.Context, s ast.StmtNode) (err error) { sc.IgnoreZeroInDate = !vars.SQLMode.HasNoZeroInDateMode() || !vars.SQLMode.HasNoZeroDateMode() || !vars.StrictSQLMode || stmt.IgnoreErr || sc.AllowInvalidDate sc.Priority = stmt.Priority case *ast.CreateTableStmt, *ast.AlterTableStmt: - // Make sure the sql_mode is strict when checking column default value. sc.InCreateOrAlterStmt = true + sc.AllowInvalidDate = vars.SQLMode.HasAllowInvalidDatesMode() + sc.IgnoreZeroInDate = !vars.SQLMode.HasNoZeroInDateMode() || !vars.SQLMode.HasNoZeroDateMode() || !vars.StrictSQLMode || sc.AllowInvalidDate case *ast.LoadDataStmt: sc.DupKeyAsWarning = true sc.BadNullAsWarning = true diff --git a/executor/executor_pkg_test.go b/executor/executor_pkg_test.go index d3b7294b79845..7cc5a8a69d66e 100644 --- a/executor/executor_pkg_test.go +++ b/executor/executor_pkg_test.go @@ -522,54 +522,15 @@ func (s *pkgTestSuite) TestAggPartialResultMapperB(c *C) { type hmap struct { // Note: the format of the hmap is also encoded in cmd/compile/internal/gc/reflect.go. // Make sure this stays in sync with the compiler's definition. - count int // # live cells == size of map. Must be first (used by len() builtin) - flags uint8 - B uint8 // log_2 of # of buckets (can hold up to loadFactor * 2^B items) - noverflow uint16 // approximate number of overflow buckets; see incrnoverflow for details - hash0 uint32 // hash seed - - buckets unsafe.Pointer // array of 2^B Buckets. may be nil if count==0. - oldbuckets unsafe.Pointer // previous bucket array of half the size, non-nil only when growing - nevacuate uintptr // progress counter for evacuation (buckets less than this have been evacuated) - - extra *mapextra // optional fields -} - -// mapextra holds fields that are not present on all maps. -// nolint:structcheck -type mapextra struct { - // If both key and elem do not contain pointers and are inline, then we mark bucket - // type as containing no pointers. This avoids scanning such maps. - // However, bmap.overflow is a pointer. In order to keep overflow buckets - // alive, we store pointers to all overflow buckets in hmap.extra.overflow and hmap.extra.oldoverflow. - // overflow and oldoverflow are only used if key and elem do not contain pointers. - // overflow contains overflow buckets for hmap.buckets. - // oldoverflow contains overflow buckets for hmap.oldbuckets. - // The indirection allows to store a pointer to the slice in hiter. - overflow *[]*bmap - oldoverflow *[]*bmap - - // nextOverflow holds a pointer to a free overflow bucket. - nextOverflow *bmap -} - -const ( - bucketCntBits = 3 - bucketCnt = 1 << bucketCntBits -) - -// A bucket for a Go map. -// nolint:structcheck -type bmap struct { - // tophash generally contains the top byte of the hash value - // for each key in this bucket. If tophash[0] < minTopHash, - // tophash[0] is a bucket evacuation state instead. - tophash [bucketCnt]uint8 - // Followed by bucketCnt keys and then bucketCnt elems. - // NOTE: packing all the keys together and then all the elems together makes the - // code a bit more complicated than alternating key/elem/key/elem/... but it allows - // us to eliminate padding which would be needed for, e.g., map[int64]int8. - // Followed by an overflow pointer. + count int // nolint:unused // # live cells == size of map. Must be first (used by len() builtin) + flags uint8 // nolint:unused + B uint8 // nolint:unused // log_2 of # of buckets (can hold up to loadFactor * 2^B items) + noverflow uint16 // nolint:unused // approximate number of overflow buckets; see incrnoverflow for details + hash0 uint32 // nolint:unused // hash seed + + buckets unsafe.Pointer // nolint:unused // array of 2^B Buckets. may be nil if count==0. + oldbuckets unsafe.Pointer // nolint:unused // previous bucket array of half the size, non-nil only when growing + nevacuate uintptr // nolint:unused // progress counter for evacuation (buckets less than this have been evacuated) } func getB(m aggPartialResultMapper) int { diff --git a/executor/executor_test.go b/executor/executor_test.go index c6a070efd8684..e8f8793d42676 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -7338,7 +7338,8 @@ func (s *testCoprCache) TestIntegrationCopCache(c *C) { c.Assert(err, IsNil) tid := tblInfo.Meta().ID tk.MustExec(`insert into t values(1),(2),(3),(4),(5),(6),(7),(8),(9),(10),(11),(12)`) - s.cls.SplitTable(tid, 6) + tableStart := tablecodec.GenTableRecordPrefix(tid) + s.cls.SplitKeys(tableStart, tableStart.PrefixNext(), 6) c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/mockstore/unistore/cophandler/mockCopCacheInUnistore", `return(123)`), IsNil) defer func() { @@ -7683,6 +7684,32 @@ func (s *testSuite) TestZeroDateTimeCompatibility(c *C) { } } +// https://github.com/pingcap/tidb/issues/24165. +func (s *testSuite) TestInvalidDateValueInCreateTable(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test;") + tk.MustExec("drop table if exists t;") + tk.MustExec("set @@sql_mode='STRICT_TRANS_TABLES,NO_ZERO_IN_DATE,NO_ZERO_DATE';") + tk.MustGetErrCode("create table t (a datetime default '2999-00-00 00:00:00');", errno.ErrInvalidDefault) + tk.MustGetErrCode("create table t (a datetime default '2999-02-30 00:00:00');", errno.ErrInvalidDefault) + tk.MustExec("create table t (a datetime);") + tk.MustGetErrCode("alter table t modify column a datetime default '2999-00-00 00:00:00';", errno.ErrInvalidDefault) + tk.MustExec("drop table if exists t;") + + tk.MustExec("set @@sql_mode = (select replace(@@sql_mode,'NO_ZERO_IN_DATE',''));") + tk.MustExec("set @@sql_mode = (select replace(@@sql_mode,'NO_ZERO_DATE',''));") + tk.MustExec("set @@sql_mode=(select concat(@@sql_mode, ',ALLOW_INVALID_DATES'));") + // Test create table with zero datetime as a default value. + tk.MustExec("create table t (a datetime default '2999-00-00 00:00:00');") + tk.MustExec("drop table if exists t;") + // Test create table with invalid datetime(02-30) as a default value. + tk.MustExec("create table t (a datetime default '2999-02-30 00:00:00');") + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t (a datetime);") + tk.MustExec("alter table t modify column a datetime default '2999-00-00 00:00:00';") + tk.MustExec("drop table if exists t;") +} + func (s *testSuite) TestOOMActionPriority(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") @@ -7926,7 +7953,7 @@ func (s *testSerialSuite) TestStalenessTransaction(c *C) { tk.MustExec("use test") for _, testcase := range testcases { c.Log(testcase.name) - failpoint.Enable("github.com/pingcap/tidb/config/injectTxnScope", + failpoint.Enable("github.com/pingcap/tidb/store/tikv/config/injectTxnScope", fmt.Sprintf(`return("%v")`, testcase.zone)) tk.MustExec(fmt.Sprintf("set @@txn_scope=%v", testcase.txnScope)) tk.MustExec(testcase.preSQL) @@ -7946,7 +7973,7 @@ func (s *testSerialSuite) TestStalenessTransaction(c *C) { } c.Assert(tk.Se.GetSessionVars().TxnCtx.IsStaleness, Equals, testcase.IsStaleness) tk.MustExec("commit") - failpoint.Disable("github.com/pingcap/tidb/config/injectTxnScope") + failpoint.Disable("github.com/pingcap/tidb/store/tikv/config/injectTxnScope") } } @@ -7986,13 +8013,13 @@ func (s *testSerialSuite) TestStaleReadKVRequest(c *C) { for _, testcase := range testcases { c.Log(testcase.name) tk.MustExec(fmt.Sprintf("set @@txn_scope=%v", testcase.txnScope)) - failpoint.Enable("github.com/pingcap/tidb/config/injectTxnScope", fmt.Sprintf(`return("%v")`, testcase.zone)) + failpoint.Enable("github.com/pingcap/tidb/store/tikv/config/injectTxnScope", fmt.Sprintf(`return("%v")`, testcase.zone)) failpoint.Enable("github.com/pingcap/tidb/store/tikv/assertStoreLabels", fmt.Sprintf(`return("%v_%v")`, placement.DCLabelKey, testcase.txnScope)) failpoint.Enable("github.com/pingcap/tidb/store/tikv/assertStaleReadFlag", `return(true)`) tk.MustExec(`START TRANSACTION READ ONLY WITH TIMESTAMP BOUND EXACT STALENESS '00:00:20';`) tk.MustQuery(testcase.sql) tk.MustExec(`commit`) - failpoint.Disable("github.com/pingcap/tidb/config/injectTxnScope") + failpoint.Disable("github.com/pingcap/tidb/store/tikv/config/injectTxnScope") failpoint.Disable("github.com/pingcap/tidb/store/tikv/assertStoreLabels") failpoint.Disable("github.com/pingcap/tidb/store/tikv/assertStaleReadFlag") } diff --git a/executor/hash_table_test.go b/executor/hash_table_test.go index 91ebc7ed4ad8b..3458002987f1e 100644 --- a/executor/hash_table_test.go +++ b/executor/hash_table_test.go @@ -27,44 +27,6 @@ import ( "github.com/pingcap/tidb/util/mock" ) -func (s *pkgTestSuite) testHashTables(c *C) { - var ht baseHashTable - test := func() { - ht.Put(1, chunk.RowPtr{ChkIdx: 1, RowIdx: 1}) - c.Check(ht.Get(1), DeepEquals, []chunk.RowPtr{{ChkIdx: 1, RowIdx: 1}}) - - rawData := map[uint64][]chunk.RowPtr{} - for i := uint64(0); i < 10; i++ { - for j := uint64(0); j < initialEntrySliceLen*i; j++ { - rawData[i] = append(rawData[i], chunk.RowPtr{ChkIdx: uint32(i), RowIdx: uint32(j)}) - } - } - // put all rawData into ht vertically - for j := uint64(0); j < initialEntrySliceLen*9; j++ { - for i := 9; i >= 0; i-- { - i := uint64(i) - if !(j < initialEntrySliceLen*i) { - break - } - ht.Put(i, rawData[i][j]) - } - } - // check - totalCount := 0 - for i := uint64(0); i < 10; i++ { - totalCount += len(rawData[i]) - c.Check(ht.Get(i), DeepEquals, rawData[i]) - } - c.Check(ht.Len(), Equals, uint64(totalCount)) - } - // test unsafeHashTable - ht = newUnsafeHashTable(0) - test() - // test ConcurrentMapHashTable - ht = newConcurrentMapHashTable() - test() -} - func initBuildChunk(numRows int) (*chunk.Chunk, []*types.FieldType) { numCols := 6 colTypes := make([]*types.FieldType, 0, numCols) diff --git a/executor/index_merge_reader.go b/executor/index_merge_reader.go index c1e780fc26e45..7c43c29972518 100644 --- a/executor/index_merge_reader.go +++ b/executor/index_merge_reader.go @@ -96,7 +96,7 @@ type IndexMergeReaderExecutor struct { memTracker *memory.Tracker // checkIndexValue is used to check the consistency of the index data. - *checkIndexValue + *checkIndexValue // nolint:unused partialPlans [][]plannercore.PhysicalPlan tblPlans []plannercore.PhysicalPlan @@ -252,9 +252,11 @@ func (e *IndexMergeReaderExecutor) startPartialIndexWorker(ctx context.Context, return nil } -func (e *IndexMergeReaderExecutor) buildPartialTableReader(ctx context.Context, workID int) Executor { - tableReaderExec := &TableReaderExecutor{ - baseExecutor: newBaseExecutor(e.ctx, e.schema, e.getPartitalPlanID(workID)), +func (e *IndexMergeReaderExecutor) startPartialTableWorker(ctx context.Context, exitCh <-chan struct{}, fetchCh chan<- *lookupTableTask, workID int, + partialWorkerWg *sync.WaitGroup) error { + ts := e.partialPlans[workID][0].(*plannercore.PhysicalTableScan) + partialTableReader := &TableReaderExecutor{ + baseExecutor: newBaseExecutor(e.ctx, ts.Schema(), e.getPartitalPlanID(workID)), table: e.table, dagPB: e.dagPBs[workID], startTS: e.startTS, @@ -263,18 +265,11 @@ func (e *IndexMergeReaderExecutor) buildPartialTableReader(ctx context.Context, plans: e.partialPlans[workID], ranges: e.ranges[workID], } - return tableReaderExec -} - -func (e *IndexMergeReaderExecutor) startPartialTableWorker(ctx context.Context, exitCh <-chan struct{}, fetchCh chan<- *lookupTableTask, workID int, - partialWorkerWg *sync.WaitGroup) error { - partialTableReader := e.buildPartialTableReader(ctx, workID) err := partialTableReader.Open(ctx) if err != nil { logutil.Logger(ctx).Error("open Select result failed:", zap.Error(err)) return err } - tableInfo := e.partialPlans[workID][0].(*plannercore.PhysicalTableScan).Table worker := &partialTableWorker{ stats: e.stats, sc: e.ctx, @@ -282,7 +277,7 @@ func (e *IndexMergeReaderExecutor) startPartialTableWorker(ctx context.Context, maxBatchSize: e.ctx.GetSessionVars().IndexLookupSize, maxChunkSize: e.maxChunkSize, tableReader: partialTableReader, - tableInfo: tableInfo, + tableInfo: ts.Table, } if worker.batchSize > worker.maxBatchSize { diff --git a/executor/infoschema_reader.go b/executor/infoschema_reader.go index d51886bb109a8..33db7c9fdd18b 100644 --- a/executor/infoschema_reader.go +++ b/executor/infoschema_reader.go @@ -1884,7 +1884,7 @@ func (e *memtableRetriever) setDataForPlacementPolicy(ctx sessionctx.Context) er continue } for _, rule := range bundle.Rules { - constraint, err := placement.RestoreLabelConstraintList(rule.LabelConstraints) + constraint, err := rule.LabelConstraints.Restore() if err != nil { return errors.Wrapf(err, "Restore rule %s in bundle %s failed", rule.ID, bundle.ID) } diff --git a/executor/insert_common.go b/executor/insert_common.go index dab4af4e93533..10fc6cb9edc59 100644 --- a/executor/insert_common.go +++ b/executor/insert_common.go @@ -1176,7 +1176,7 @@ func (e *InsertRuntimeStat) Clone() execdetails.RuntimeStats { } if e.SnapshotRuntimeStats != nil { snapshotStats := e.SnapshotRuntimeStats.Clone() - newRs.SnapshotRuntimeStats = snapshotStats.(*tikv.SnapshotRuntimeStats) + newRs.SnapshotRuntimeStats = snapshotStats } if e.BasicRuntimeStats != nil { basicStats := e.BasicRuntimeStats.Clone() @@ -1194,7 +1194,7 @@ func (e *InsertRuntimeStat) Merge(other execdetails.RuntimeStats) { if tmp.SnapshotRuntimeStats != nil { if e.SnapshotRuntimeStats == nil { snapshotStats := tmp.SnapshotRuntimeStats.Clone() - e.SnapshotRuntimeStats = snapshotStats.(*tikv.SnapshotRuntimeStats) + e.SnapshotRuntimeStats = snapshotStats } else { e.SnapshotRuntimeStats.Merge(tmp.SnapshotRuntimeStats) } diff --git a/executor/insert_test.go b/executor/insert_test.go index 390dcddb2bc27..bee38e51c0fea 100644 --- a/executor/insert_test.go +++ b/executor/insert_test.go @@ -31,6 +31,7 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/execdetails" + "github.com/pingcap/tidb/util/israce" "github.com/pingcap/tidb/util/testkit" "github.com/pingcap/tidb/util/testutil" ) @@ -246,6 +247,9 @@ func (s *testSuite10) TestPaddingCommonHandle(c *C) { } func (s *testSuite2) TestInsertReorgDelete(c *C) { + if israce.RaceEnabled { + c.Skip("exhaustive types test, skip race test") + } tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") diff --git a/executor/inspection_result.go b/executor/inspection_result.go index 4b8c4cb9753e0..1faa2e83150c5 100644 --- a/executor/inspection_result.go +++ b/executor/inspection_result.go @@ -72,10 +72,6 @@ func (f inspectionFilter) enable(name string) bool { return len(f.set) == 0 || f.set.Exist(name) } -func (f inspectionFilter) exist(name string) bool { - return len(f.set) > 0 && f.set.Exist(name) -} - type ( // configInspection is used to check whether a same configuration item has a // different value between different instance in the cluster diff --git a/executor/join.go b/executor/join.go index 472df41ded441..c1a8045aba9a3 100644 --- a/executor/join.go +++ b/executor/join.go @@ -487,7 +487,7 @@ func (e *HashJoinExec) joinMatchedProbeSideRow2ChunkForOuterHashJoin(workerID ui iter := chunk.NewIterator4Slice(buildSideRows) var outerMatchStatus []outerRowStatusFlag - rowIdx := 0 + rowIdx, ok := 0, false for iter.Begin(); iter.Current() != iter.End(); { outerMatchStatus, err = e.joiners[workerID].tryToMatchOuters(iter, probeSideRow, joinResult.chk, outerMatchStatus) if err != nil { @@ -502,7 +502,7 @@ func (e *HashJoinExec) joinMatchedProbeSideRow2ChunkForOuterHashJoin(workerID ui rowIdx += len(outerMatchStatus) if joinResult.chk.IsFull() { e.joinResultCh <- joinResult - ok, joinResult := e.getNewJoinResult(workerID) + ok, joinResult = e.getNewJoinResult(workerID) if !ok { return false, joinResult } @@ -522,7 +522,7 @@ func (e *HashJoinExec) joinMatchedProbeSideRow2Chunk(workerID uint, probeKey uin return true, joinResult } iter := chunk.NewIterator4Slice(buildSideRows) - hasMatch, hasNull := false, false + hasMatch, hasNull, ok := false, false, false for iter.Begin(); iter.Current() != iter.End(); { matched, isNull, err := e.joiners[workerID].tryToMatchInners(probeSideRow, iter, joinResult.chk) if err != nil { @@ -534,7 +534,7 @@ func (e *HashJoinExec) joinMatchedProbeSideRow2Chunk(workerID uint, probeKey uin if joinResult.chk.IsFull() { e.joinResultCh <- joinResult - ok, joinResult := e.getNewJoinResult(workerID) + ok, joinResult = e.getNewJoinResult(workerID) if !ok { return false, joinResult } @@ -1071,13 +1071,6 @@ func (e *joinRuntimeStats) setCacheInfo(useCache bool, hitRatio float64) { e.Unlock() } -func (e *joinRuntimeStats) setHashStat(hashStat hashStatistic) { - e.Lock() - e.hasHashStat = true - e.hashStat = hashStat - e.Unlock() -} - func (e *joinRuntimeStats) String() string { buf := bytes.NewBuffer(make([]byte, 0, 16)) buf.WriteString(e.RuntimeStatsWithConcurrencyInfo.String()) diff --git a/executor/point_get.go b/executor/point_get.go index ea82daaddd91d..b4ec8d13bfe72 100644 --- a/executor/point_get.go +++ b/executor/point_get.go @@ -580,7 +580,7 @@ func (e *runtimeStatsWithSnapshot) Clone() execdetails.RuntimeStats { newRs := &runtimeStatsWithSnapshot{} if e.SnapshotRuntimeStats != nil { snapshotStats := e.SnapshotRuntimeStats.Clone() - newRs.SnapshotRuntimeStats = snapshotStats.(*tikv.SnapshotRuntimeStats) + newRs.SnapshotRuntimeStats = snapshotStats } return newRs } @@ -594,7 +594,7 @@ func (e *runtimeStatsWithSnapshot) Merge(other execdetails.RuntimeStats) { if tmp.SnapshotRuntimeStats != nil { if e.SnapshotRuntimeStats == nil { snapshotStats := tmp.SnapshotRuntimeStats.Clone() - e.SnapshotRuntimeStats = snapshotStats.(*tikv.SnapshotRuntimeStats) + e.SnapshotRuntimeStats = snapshotStats return } e.SnapshotRuntimeStats.Merge(tmp.SnapshotRuntimeStats) diff --git a/executor/prepared_test.go b/executor/prepared_test.go index 34cee0306d948..1f8edf79d942e 100644 --- a/executor/prepared_test.go +++ b/executor/prepared_test.go @@ -28,6 +28,7 @@ import ( "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/util" + "github.com/pingcap/tidb/util/israce" "github.com/pingcap/tidb/util/testkit" ) @@ -78,6 +79,9 @@ func (s *testSuite1) TestIgnorePlanCache(c *C) { } func (s *testSerialSuite) TestPrepareStmtAfterIsolationReadChange(c *C) { + if israce.RaceEnabled { + c.Skip("race test for this case takes too long time") + } tk := testkit.NewTestKitWithInit(c, s.store) tk.Se.Auth(&auth.UserIdentity{Username: "root", Hostname: "localhost", CurrentUser: true, AuthUsername: "root", AuthHostname: "%"}, nil, []byte("012345678901234567890")) diff --git a/executor/seqtest/seq_executor_test.go b/executor/seqtest/seq_executor_test.go index dae4877a1a6e0..7b38a6f6b673e 100644 --- a/executor/seqtest/seq_executor_test.go +++ b/executor/seqtest/seq_executor_test.go @@ -55,6 +55,7 @@ import ( "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/store/tikv/mockstore/cluster" "github.com/pingcap/tidb/store/tikv/tikvrpc" + "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/gcutil" "github.com/pingcap/tidb/util/logutil" @@ -136,7 +137,8 @@ func (s *seqTestSuite) TestEarlyClose(c *C) { tblID := tbl.Meta().ID // Split the table. - s.cluster.SplitTable(tblID, N/2) + tableStart := tablecodec.GenTableRecordPrefix(tblID) + s.cluster.SplitKeys(tableStart, tableStart.PrefixNext(), N/2) ctx := context.Background() for i := 0; i < N/2; i++ { diff --git a/executor/set.go b/executor/set.go index 503871fbe4a55..c4442f38234a2 100644 --- a/executor/set.go +++ b/executor/set.go @@ -235,9 +235,26 @@ func (e *SetExecutor) setSysVariable(name string, v *expression.VarAssignment) e return nil } +// Run the validation function before setting a variable +// This helps normalize values and not depend on user input directly. +// It should technically not be required since the setCharset function +// has already performed its own validation. It also helps maintain consistency +// since eventually SET without validate may become be illegal. +func (e *SetExecutor) validateAndSetVar(name, val string) (err error) { + sysVar := variable.GetSysVar(name) + if sysVar == nil { + return variable.ErrUnknownSystemVar.GenWithStackByArgs(name) + } + val, err = sysVar.Validate(e.ctx.GetSessionVars(), val, variable.ScopeSession) + if err != nil { + return err + } + return e.ctx.GetSessionVars().SetSystemVar(name, val) +} + func (e *SetExecutor) setCharset(cs, co string, isSetName bool) error { var err error - if len(co) == 0 { + if co == "" { if co, err = charset.GetDefaultCollation(cs); err != nil { return err } @@ -250,21 +267,21 @@ func (e *SetExecutor) setCharset(cs, co string, isSetName bool) error { return charset.ErrCollationCharsetMismatch.GenWithStackByArgs(coll.Name, cs) } } - sessionVars := e.ctx.GetSessionVars() if isSetName { for _, v := range variable.SetNamesVariables { - if err = sessionVars.SetSystemVar(v, cs); err != nil { + if err = e.validateAndSetVar(v, cs); err != nil { return errors.Trace(err) } } - return errors.Trace(sessionVars.SetSystemVar(variable.CollationConnection, co)) + return errors.Trace(e.validateAndSetVar(variable.CollationConnection, co)) } // Set charset statement, see also https://dev.mysql.com/doc/refman/8.0/en/set-character-set.html. for _, v := range variable.SetCharsetVariables { - if err = sessionVars.SetSystemVar(v, cs); err != nil { + if err = e.validateAndSetVar(v, cs); err != nil { return errors.Trace(err) } } + sessionVars := e.ctx.GetSessionVars() csDb, err := sessionVars.GlobalVarsAccessor.GetGlobalSysVar(variable.CharsetDatabase) if err != nil { return err @@ -273,11 +290,11 @@ func (e *SetExecutor) setCharset(cs, co string, isSetName bool) error { if err != nil { return err } - err = sessionVars.SetSystemVar(variable.CharacterSetConnection, csDb) + err = e.validateAndSetVar(variable.CharacterSetConnection, csDb) if err != nil { return errors.Trace(err) } - return errors.Trace(sessionVars.SetSystemVar(variable.CollationConnection, coDb)) + return errors.Trace(e.validateAndSetVar(variable.CollationConnection, coDb)) } func (e *SetExecutor) getVarValue(v *expression.VarAssignment, sysVar *variable.SysVar) (value types.Datum, err error) { diff --git a/executor/set_test.go b/executor/set_test.go index a4b92a5a6f35d..ec97914cc12a5 100644 --- a/executor/set_test.go +++ b/executor/set_test.go @@ -158,6 +158,9 @@ func (s *testSerialSuite1) TestSetVar(c *C) { tk.MustGetErrMsg("set @@global.collation_database='non_exist_collation'", expectErrMsg) tk.MustGetErrMsg("set @@global.collation_connection='non_exist_collation'", expectErrMsg) + expectErrMsg = "[parser:1115]Unknown character set: 'boguscharsetname'" + tk.MustGetErrMsg("set names boguscharsetname", expectErrMsg) + tk.MustExec("set character_set_results = NULL") tk.MustQuery("select @@character_set_results").Check(testkit.Rows("")) @@ -397,6 +400,14 @@ func (s *testSerialSuite1) TestSetVar(c *C) { c.Assert(err, ErrorMatches, ".*Variable 'tidb_metric_query_range_duration' can't be set to the value of '9'") tk.MustQuery("select @@session.tidb_metric_query_range_duration;").Check(testkit.Rows("120")) + tk.MustExec("set @@cte_max_recursion_depth=100") + tk.MustQuery("select @@cte_max_recursion_depth").Check(testkit.Rows("100")) + tk.MustExec("set @@global.cte_max_recursion_depth=100") + tk.MustQuery("select @@global.cte_max_recursion_depth").Check(testkit.Rows("100")) + tk.MustExec("set @@cte_max_recursion_depth=-1") + tk.MustQuery(`show warnings`).Check(testkit.Rows("Warning 1292 Truncated incorrect cte_max_recursion_depth value: '-1'")) + tk.MustQuery("select @@cte_max_recursion_depth").Check(testkit.Rows("0")) + // test for tidb_slow_log_masking tk.MustQuery(`select @@global.tidb_slow_log_masking;`).Check(testkit.Rows("0")) tk.MustExec("set global tidb_slow_log_masking = 1") diff --git a/executor/show.go b/executor/show.go index 7f3e1b2ba85d3..22f48ffc3fcd5 100644 --- a/executor/show.go +++ b/executor/show.go @@ -730,7 +730,8 @@ func getDefaultCollate(charsetName string) string { return c.DefaultCollation } } - return "" + // The charset is invalid, return server default. + return charset.CollationUTF8MB4 } // ConstructResultOfShowCreateTable constructs the result for show create table. diff --git a/executor/show_test.go b/executor/show_test.go index 72b7f80bac114..7bebc1843db76 100644 --- a/executor/show_test.go +++ b/executor/show_test.go @@ -1233,27 +1233,37 @@ func (s *testSerialSuite1) TestShowCreateTableWithIntegerDisplayLengthWarnings(c func (s *testSuite5) TestShowVar(c *C) { tk := testkit.NewTestKit(c, s.store) var showSQL string + sessionVars := make([]string, 0, len(variable.GetSysVars())) + globalVars := make([]string, 0, len(variable.GetSysVars())) for _, v := range variable.GetSysVars() { if variable.FilterImplicitFeatureSwitch(v) { continue } - // When ScopeSession only. `show global variables` must return empty. + if v.Scope == variable.ScopeSession { - showSQL = "show variables like '" + v.Name + "'" - res := tk.MustQuery(showSQL) - c.Check(res.Rows(), HasLen, 1) - showSQL = "show global variables like '" + v.Name + "'" - res = tk.MustQuery(showSQL) - c.Check(res.Rows(), HasLen, 0) + sessionVars = append(sessionVars, v.Name) } else { - showSQL = "show global variables like '" + v.Name + "'" - res := tk.MustQuery(showSQL) - c.Check(res.Rows(), HasLen, 1) - showSQL = "show variables like '" + v.Name + "'" - res = tk.MustQuery(showSQL) - c.Check(res.Rows(), HasLen, 1) + globalVars = append(globalVars, v.Name) } } + + // When ScopeSession only. `show global variables` must return empty. + sessionVarsStr := strings.Join(sessionVars, "','") + showSQL = "show variables where variable_name in('" + sessionVarsStr + "')" + res := tk.MustQuery(showSQL) + c.Check(res.Rows(), HasLen, len(sessionVars)) + showSQL = "show global variables where variable_name in('" + sessionVarsStr + "')" + res = tk.MustQuery(showSQL) + c.Check(res.Rows(), HasLen, 0) + + globalVarsStr := strings.Join(globalVars, "','") + showSQL = "show variables where variable_name in('" + globalVarsStr + "')" + res = tk.MustQuery(showSQL) + c.Check(res.Rows(), HasLen, len(globalVars)) + showSQL = "show global variables where variable_name in('" + globalVarsStr + "')" + res = tk.MustQuery(showSQL) + c.Check(res.Rows(), HasLen, len(globalVars)) + // Test for switch variable which shouldn't seen by users. for _, one := range variable.FeatureSwitchVariables { res := tk.MustQuery("show variables like '" + one + "'") diff --git a/executor/simple.go b/executor/simple.go index a490c694dea25..e03a781ca79f0 100644 --- a/executor/simple.go +++ b/executor/simple.go @@ -47,6 +47,7 @@ import ( driver "github.com/pingcap/tidb/types/parser_driver" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/hack" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/sqlexec" @@ -552,19 +553,15 @@ func (e *SimpleExec) executeUse(s *ast.UseStmt) error { } e.ctx.GetSessionVars().CurrentDBChanged = dbname.O != e.ctx.GetSessionVars().CurrentDB e.ctx.GetSessionVars().CurrentDB = dbname.O - // character_set_database is the character set used by the default database. - // The server sets this variable whenever the default database changes. - // See http://dev.mysql.com/doc/refman/5.7/en/server-system-variables.html#sysvar_character_set_database sessionVars := e.ctx.GetSessionVars() - err := sessionVars.SetSystemVar(variable.CharsetDatabase, dbinfo.Charset) - if err != nil { - return err - } dbCollate := dbinfo.Collate if dbCollate == "" { - // Since we have checked the charset, the dbCollate here shouldn't be "". dbCollate = getDefaultCollate(dbinfo.Charset) } + // If new collations are enabled, switch to the default + // collation if this one is not supported. + // The SetSystemVar will also update the CharsetDatabase + dbCollate = collate.SubstituteMissingCollationToDefault(dbCollate) return sessionVars.SetSystemVar(variable.CollationDatabase, dbCollate) } diff --git a/executor/sort_test.go b/executor/sort_test.go index 32ded8ff0da71..57c8a9fc6d71f 100644 --- a/executor/sort_test.go +++ b/executor/sort_test.go @@ -14,6 +14,7 @@ package executor_test import ( + "bytes" "fmt" "os" "strings" @@ -64,11 +65,17 @@ func (s *testSerialSuite1) testSortInDisk(c *C, removeDir bool) { tk.MustExec("set @@tidb_max_chunk_size=32;") tk.MustExec("drop table if exists t") tk.MustExec("create table t(c1 int, c2 int, c3 int)") + var buf bytes.Buffer + buf.WriteString("insert into t values ") for i := 0; i < 5; i++ { for j := i; j < 1024; j += 5 { - tk.MustExec(fmt.Sprintf("insert into t values(%v, %v, %v)", j, j, j)) + if j > 0 { + buf.WriteString(", ") + } + buf.WriteString(fmt.Sprintf("(%v, %v, %v)", j, j, j)) } } + tk.MustExec(buf.String()) result := tk.MustQuery("select * from t order by c1") for i := 0; i < 1024; i++ { c.Assert(result.Rows()[i][0].(string), Equals, fmt.Sprint(i)) diff --git a/executor/tiflash_test.go b/executor/tiflash_test.go index 2f6f9c8e58643..690106a9a38e5 100644 --- a/executor/tiflash_test.go +++ b/executor/tiflash_test.go @@ -32,6 +32,7 @@ import ( "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/store/mockstore/unistore" "github.com/pingcap/tidb/store/tikv/mockstore/cluster" + "github.com/pingcap/tidb/util/israce" "github.com/pingcap/tidb/util/testkit" "github.com/pingcap/tidb/util/testleak" ) @@ -134,6 +135,9 @@ func (s *tiflashTestSuite) TestReadUnsigedPK(c *C) { } func (s *tiflashTestSuite) TestMppExecution(c *C) { + if israce.RaceEnabled { + c.Skip("skip race test because of long running") + } tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t") diff --git a/executor/union_scan_test.go b/executor/union_scan_test.go index 48a5768a10be8..31f9cc6b98670 100644 --- a/executor/union_scan_test.go +++ b/executor/union_scan_test.go @@ -14,6 +14,8 @@ package executor_test import ( + "fmt" + . "github.com/pingcap/check" "github.com/pingcap/tidb/util/testkit" ) @@ -339,9 +341,15 @@ func (s *testSuite7) TestUpdateScanningHandles(c *C) { tk.MustExec("drop table if exists t;") tk.MustExec("create table t(a int primary key, b int);") tk.MustExec("begin") - for i := 2; i < 100000; i++ { - tk.MustExec("insert into t values (?, ?)", i, i) + var insertSQL string + for i := 2; i < 10000; i++ { + if i == 2 { + insertSQL += fmt.Sprintf("(%d, %d)", i, i) + } else { + insertSQL += fmt.Sprintf(",(%d, %d)", i, i) + } } + tk.MustExec(fmt.Sprintf("insert into t values %s;", insertSQL)) tk.MustExec("commit;") tk.MustExec("set tidb_distsql_scan_concurrency = 1;") diff --git a/expression/bench_test.go b/expression/bench_test.go index f3b71abce94cd..3a2b6559c29cb 100644 --- a/expression/bench_test.go +++ b/expression/bench_test.go @@ -21,7 +21,6 @@ import ( "math/rand" "net" "reflect" - "strconv" "strings" "testing" "time" @@ -287,23 +286,9 @@ func (g *defaultGener) gen() interface{} { type charInt64Gener struct{} func (g *charInt64Gener) gen() interface{} { - rand := time.Now().Nanosecond() - rand = rand % 1024 - return int64(rand) -} - -// charsetStringGener is used to generate "ascii" or "gbk" -type charsetStringGener struct{} - -func (g *charsetStringGener) gen() interface{} { - rand := time.Now().Nanosecond() % 3 - if rand == 0 { - return "ascii" - } - if rand == 1 { - return "utf8" - } - return "gbk" + nanosecond := time.Now().Nanosecond() + nanosecond = nanosecond % 1024 + return int64(nanosecond) } // selectStringGener select one string randomly from the candidates array @@ -866,12 +851,6 @@ func (g *randDurDecimal) gen() interface{} { return d.FromFloat64(float64(g.randGen.Intn(types.TimeMaxHour)*10000 + g.randGen.Intn(60)*100 + g.randGen.Intn(60))) } -type randDurString struct{} - -func (g *randDurString) gen() interface{} { - return strconv.Itoa(rand.Intn(types.TimeMaxHour)*10000 + rand.Intn(60)*100 + rand.Intn(60)) -} - // locationGener is used to generate location for the built-in function GetFormat. type locationGener struct { nullRation float64 diff --git a/expression/builtin_time_vec_test.go b/expression/builtin_time_vec_test.go index 5cb99af440060..593cce162d7ff 100644 --- a/expression/builtin_time_vec_test.go +++ b/expression/builtin_time_vec_test.go @@ -64,10 +64,6 @@ func (g *unitStrGener) gen() interface{} { return units[n] } -type dateTimeUnitStrGener struct { - randGen *defaultRandGen -} - // tzStrGener is used to generate strings which are timezones type tzStrGener struct{} @@ -84,24 +80,6 @@ func (g *tzStrGener) gen() interface{} { return tzs[n] } -func (g *dateTimeUnitStrGener) gen() interface{} { - dateTimes := []string{ - "DAY", - "WEEK", - "MONTH", - "QUARTER", - "YEAR", - "DAY_MICROSECOND", - "DAY_SECOND", - "DAY_MINUTE", - "DAY_HOUR", - "YEAR_MONTH", - } - - n := g.randGen.Intn(len(dateTimes)) - return dateTimes[n] -} - var vecBuiltinTimeCases = map[string][]vecExprBenchCase{ ast.DateLiteral: { {retEvalType: types.ETDatetime, childrenTypes: []types.EvalType{types.ETDatetime}, diff --git a/expression/constant_propagation_test.go b/expression/constant_propagation_test.go index 5206264aad9ff..7419203ddb5e4 100644 --- a/expression/constant_propagation_test.go +++ b/expression/constant_propagation_test.go @@ -14,8 +14,6 @@ package expression_test import ( - "fmt" - . "github.com/pingcap/check" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/kv" @@ -34,16 +32,6 @@ type testSuite struct { testData testutil.TestData } -func (s *testSuite) cleanEnv(c *C) { - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test") - r := tk.MustQuery("show tables") - for _, tb := range r.Rows() { - tableName := tb[0] - tk.MustExec(fmt.Sprintf("drop table %v", tableName)) - } -} - func (s *testSuite) SetUpSuite(c *C) { var err error s.store, s.dom, err = newStoreWithBootstrap() diff --git a/expression/flag_simplify_test.go b/expression/flag_simplify_test.go index cb0ef080e00d6..0e9e579aa51dd 100644 --- a/expression/flag_simplify_test.go +++ b/expression/flag_simplify_test.go @@ -14,8 +14,6 @@ package expression_test import ( - "fmt" - . "github.com/pingcap/check" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/kv" @@ -34,16 +32,6 @@ type testFlagSimplifySuite struct { testData testutil.TestData } -func (s *testFlagSimplifySuite) cleanEnv(c *C) { - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test") - r := tk.MustQuery("show tables") - for _, tb := range r.Rows() { - tableName := tb[0] - tk.MustExec(fmt.Sprintf("drop table %v", tableName)) - } -} - func (s *testFlagSimplifySuite) SetUpSuite(c *C) { var err error s.store, s.dom, err = newStoreWithBootstrap() diff --git a/expression/integration_test.go b/expression/integration_test.go index d848acdee0f68..01a3bc0b92c95 100644 --- a/expression/integration_test.go +++ b/expression/integration_test.go @@ -4720,7 +4720,7 @@ func (s *testIntegrationSuite) TestFilterExtractFromDNF(c *C) { } } -func (s *testIntegrationSuite) testTiDBIsOwnerFunc(c *C) { +func (s *testIntegrationSuite) TestTiDBIsOwnerFunc(c *C) { tk := testkit.NewTestKit(c, s.store) defer s.cleanEnv(c) result := tk.MustQuery("select tidb_is_ddl_owner()") @@ -8831,7 +8831,7 @@ PARTITION BY RANGE (c) ( } for _, testcase := range testcases { c.Log(testcase.name) - failpoint.Enable("github.com/pingcap/tidb/config/injectTxnScope", + failpoint.Enable("github.com/pingcap/tidb/store/tikv/config/injectTxnScope", fmt.Sprintf(`return("%v")`, testcase.zone)) _, err = tk.Exec(fmt.Sprintf("set @@txn_scope='%v'", testcase.txnScope)) c.Assert(err, IsNil) @@ -8849,7 +8849,7 @@ PARTITION BY RANGE (c) ( } else { c.Assert(checkErr, IsNil) } - failpoint.Disable("github.com/pingcap/tidb/config/injectTxnScope") + failpoint.Disable("github.com/pingcap/tidb/store/tikv/config/injectTxnScope") } } diff --git a/expression/util_test.go b/expression/util_test.go index 7f56623fd7de4..c2b12c166a7c0 100644 --- a/expression/util_test.go +++ b/expression/util_test.go @@ -33,18 +33,7 @@ import ( var _ = check.Suite(&testUtilSuite{}) -type testUtilSuite struct { -} - -func (s *testUtilSuite) checkPanic(f func()) (ret bool) { - defer func() { - if r := recover(); r != nil { - ret = true - } - }() - f() - return false -} +type testUtilSuite struct{} func (s *testUtilSuite) TestBaseBuiltin(c *check.C) { ctx := mock.NewContext() diff --git a/go.mod b/go.mod index 6e8463bceb9ef..f119e8449f0b0 100644 --- a/go.mod +++ b/go.mod @@ -39,7 +39,7 @@ require ( github.com/opentracing/opentracing-go v1.1.0 github.com/phayes/freeport v0.0.0-20180830031419-95f893ade6f2 github.com/pingcap/badger v1.5.1-0.20200908111422-2e78ee155d19 - github.com/pingcap/br v5.0.0-nightly.0.20210407061032-be5523713acf+incompatible + github.com/pingcap/br v5.0.0-nightly.0.20210419090151-03762465b589+incompatible github.com/pingcap/check v0.0.0-20200212061837-5e12011dc712 github.com/pingcap/errors v0.11.5-0.20201126102027-b0a155152ca3 github.com/pingcap/failpoint v0.0.0-20210316064728-7acb0f0a3dfd @@ -50,7 +50,7 @@ require ( github.com/pingcap/parser v0.0.0-20210330190622-f959a136fc19 github.com/pingcap/sysutil v0.0.0-20210315073920-cc0985d983a3 github.com/pingcap/tidb-tools v4.0.9-0.20201127090955-2707c97b3853+incompatible - github.com/pingcap/tipb v0.0.0-20210326161441-1164ca065d1b + github.com/pingcap/tipb v0.0.0-20210422074242-57dd881b81b1 github.com/prometheus/client_golang v1.5.1 github.com/prometheus/client_model v0.2.0 github.com/prometheus/common v0.9.1 diff --git a/go.sum b/go.sum index d155e69cf417f..194bda3e9e6fc 100644 --- a/go.sum +++ b/go.sum @@ -410,8 +410,8 @@ github.com/phf/go-queue v0.0.0-20170504031614-9abe38d0371d h1:U+PMnTlV2tu7RuMK5e github.com/phf/go-queue v0.0.0-20170504031614-9abe38d0371d/go.mod h1:lXfE4PvvTW5xOjO6Mba8zDPyw8M93B6AQ7frTGnMlA8= github.com/pingcap/badger v1.5.1-0.20200908111422-2e78ee155d19 h1:IXpGy7y9HyoShAFmzW2OPF0xCA5EOoSTyZHwsgYk9Ro= github.com/pingcap/badger v1.5.1-0.20200908111422-2e78ee155d19/go.mod h1:LyrqUOHZrUDf9oGi1yoz1+qw9ckSIhQb5eMa1acOLNQ= -github.com/pingcap/br v5.0.0-nightly.0.20210407061032-be5523713acf+incompatible h1:pkfMiswYXWh4W8ehyOTQxaxQzSvP8NCeVfc2LxmQAAg= -github.com/pingcap/br v5.0.0-nightly.0.20210407061032-be5523713acf+incompatible/go.mod h1:ymVmo50lQydxib0tmK5hHk4oteB7hZ0IMCArunwy3UQ= +github.com/pingcap/br v5.0.0-nightly.0.20210419090151-03762465b589+incompatible h1:VF2oZgvBqSIMmplEWXGGmktuQGdGGIGWwptmjJFhQbU= +github.com/pingcap/br v5.0.0-nightly.0.20210419090151-03762465b589+incompatible/go.mod h1:ymVmo50lQydxib0tmK5hHk4oteB7hZ0IMCArunwy3UQ= github.com/pingcap/check v0.0.0-20190102082844-67f458068fc8/go.mod h1:B1+S9LNcuMyLH/4HMTViQOJevkGiik3wW2AN9zb2fNQ= github.com/pingcap/check v0.0.0-20191107115940-caf2b9e6ccf4/go.mod h1:PYMCGwN0JHjoqGr3HrZoD+b8Tgx8bKnArhSq8YVzUMc= github.com/pingcap/check v0.0.0-20191216031241-8a5a85928f12/go.mod h1:PYMCGwN0JHjoqGr3HrZoD+b8Tgx8bKnArhSq8YVzUMc= @@ -451,8 +451,8 @@ github.com/pingcap/sysutil v0.0.0-20210315073920-cc0985d983a3/go.mod h1:tckvA041 github.com/pingcap/tidb-dashboard v0.0.0-20210312062513-eef5d6404638/go.mod h1:OzFN8H0EDMMqeulPhPMw2i2JaiZWOKFQ7zdRPhENNgo= github.com/pingcap/tidb-tools v4.0.9-0.20201127090955-2707c97b3853+incompatible h1:ceznmu/lLseGHP/jKyOa/3u/5H3wtLLLqkH2V3ssSjg= github.com/pingcap/tidb-tools v4.0.9-0.20201127090955-2707c97b3853+incompatible/go.mod h1:XGdcy9+yqlDSEMTpOXnwf3hiTeqrV6MN/u1se9N8yIM= -github.com/pingcap/tipb v0.0.0-20210326161441-1164ca065d1b h1:sZHSH0mh8PcRbmZlsIqP7CEwnfFuBpmkGt5i9JStLWA= -github.com/pingcap/tipb v0.0.0-20210326161441-1164ca065d1b/go.mod h1:nsEhnMokcn7MRqd2J60yxpn/ac3ZH8A6GOJ9NslabUo= +github.com/pingcap/tipb v0.0.0-20210422074242-57dd881b81b1 h1:Kcp3jIcQrqG+pT1JQ0oWyRncVKQtDgnMFzRt3zJBaBo= +github.com/pingcap/tipb v0.0.0-20210422074242-57dd881b81b1/go.mod h1:nsEhnMokcn7MRqd2J60yxpn/ac3ZH8A6GOJ9NslabUo= github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.9.1 h1:FEBLx1zS214owpjy7qsBeixbURkuhQAwrK5UwLGTwt4= @@ -500,7 +500,6 @@ github.com/russross/blackfriday/v2 v2.0.1/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQD github.com/sasha-s/go-deadlock v0.2.0/go.mod h1:StQn567HiB1fF2yJ44N9au7wOhrPS3iZqiDbRupzT10= github.com/satori/go.uuid v1.2.0/go.mod h1:dA0hQrYB0VpLJoorglMZABFdXlWrHn1NEOzdhQKdks0= github.com/sergi/go-diff v1.0.1-0.20180205163309-da645544ed44/go.mod h1:0CfEIISq7TuYL3j771MWULgwwjU+GofnZX9QAmXWZgo= -github.com/sergi/go-diff v1.1.0 h1:we8PVUC3FE2uYfodKH/nBHMSetSfHDR6scGdBi+erh0= github.com/sergi/go-diff v1.1.0/go.mod h1:STckp+ISIX8hZLjrqAeVduY0gWCT9IjLuqbuNXdaHfM= github.com/shirou/gopsutil v2.19.10+incompatible/go.mod h1:5b4v6he4MtMOwMlS0TUMTu2PcXUg8+E1lC7eC3UO/RA= github.com/shirou/gopsutil v3.21.2+incompatible h1:U+YvJfjCh6MslYlIAXvPtzhW3YZEtc9uncueUNpD/0A= diff --git a/kv/interface_mock_test.go b/kv/interface_mock_test.go index ef49933526ac1..2388c4f48b9f3 100644 --- a/kv/interface_mock_test.go +++ b/kv/interface_mock_test.go @@ -126,11 +126,11 @@ func (t *mockTxn) Reset() { t.valid = false } -func (t *mockTxn) SetVars(vars *Variables) { +func (t *mockTxn) SetVars(vars interface{}) { } -func (t *mockTxn) GetVars() *Variables { +func (t *mockTxn) GetVars() interface{} { return nil } diff --git a/kv/kv.go b/kv/kv.go index 44538c4a56ab9..f9c38c7ada08f 100644 --- a/kv/kv.go +++ b/kv/kv.go @@ -176,9 +176,9 @@ type Transaction interface { // GetUnionStore returns the UnionStore binding to this transaction. GetUnionStore() UnionStore // SetVars sets variables to the transaction. - SetVars(vars *Variables) + SetVars(vars interface{}) // GetVars gets variables from the transaction. - GetVars() *Variables + GetVars() interface{} // BatchGet gets kv from the memory buffer of statement and transaction, and the kv storage. // Do not use len(value) == 0 or value == nil to represent non-exist. // If a key doesn't exist, there shouldn't be any corresponding entry in the result map. @@ -195,7 +195,7 @@ type Transaction interface { // Client is used to send request to KV layer. type Client interface { // Send sends request to KV layer, returns a Response. - Send(ctx context.Context, req *Request, vars *Variables, sessionMemTracker *memory.Tracker, enabledRateLimitAction bool) Response + Send(ctx context.Context, req *Request, vars interface{}, sessionMemTracker *memory.Tracker, enabledRateLimitAction bool) Response // IsRequestTypeSupported checks if reqType and subType is supported. IsRequestTypeSupported(reqType, subType int64) bool diff --git a/kv/mpp.go b/kv/mpp.go index df38894d0c5d1..fd81f2d264804 100644 --- a/kv/mpp.go +++ b/kv/mpp.go @@ -79,7 +79,7 @@ type MPPClient interface { ConstructMPPTasks(context.Context, *MPPBuildTasksRequest) ([]MPPTaskMeta, error) // DispatchMPPTasks dispatches ALL mpp requests at once, and returns an iterator that transfers the data. - DispatchMPPTasks(context.Context, *Variables, []*MPPDispatchRequest) Response + DispatchMPPTasks(ctx context.Context, vars interface{}, reqs []*MPPDispatchRequest) Response } // MPPBuildTasksRequest request the stores allocation for a mpp plan fragment. diff --git a/kv/utils_test.go b/kv/utils_test.go index 03c77a56e8705..d5c184a42074f 100644 --- a/kv/utils_test.go +++ b/kv/utils_test.go @@ -16,10 +16,16 @@ package kv import ( "context" "strconv" + "testing" . "github.com/pingcap/check" ) +func TestT(t *testing.T) { + CustomVerboseFlag = true + TestingT(t) +} + var _ = Suite(testUtilsSuite{}) type testUtilsSuite struct { @@ -50,7 +56,7 @@ func (s *mockMap) Get(ctx context.Context, k Key) ([]byte, error) { return s.value[i], nil } } - return nil, nil + return nil, ErrNotExist } func (s *mockMap) Set(k Key, v []byte) error { diff --git a/kv/variables.go b/kv/variables.go index ad589b280633a..df72d284a5c8d 100644 --- a/kv/variables.go +++ b/kv/variables.go @@ -13,38 +13,14 @@ package kv -// Variables defines the variables used by KV storage. -type Variables struct { - // BackoffLockFast specifies the LockFast backoff base duration in milliseconds. - BackoffLockFast int - - // BackOffWeight specifies the weight of the max back off time duration. - BackOffWeight int - - // Hook is used for test to verify the variable take effect. - Hook func(name string, vars *Variables) +import ( + tikv "github.com/pingcap/tidb/store/tikv/kv" +) - // Pointer to SessionVars.Killed - // Killed is a flag to indicate that this query is killed. - Killed *uint32 -} +// Variables defines the variables used by KV storage. TODO:remove it when br is ready. +type Variables = tikv.Variables -// NewVariables create a new Variables instance with default values. +// NewVariables create a new Variables instance with default values. TODO:remove it when br is ready. func NewVariables(killed *uint32) *Variables { - return &Variables{ - BackoffLockFast: DefBackoffLockFast, - BackOffWeight: DefBackOffWeight, - Killed: killed, - } + return tikv.NewVariables(killed) } - -var ignoreKill uint32 - -// DefaultVars is the default variables instance. -var DefaultVars = NewVariables(&ignoreKill) - -// Default values -const ( - DefBackoffLockFast = 100 - DefBackOffWeight = 2 -) diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index 309091554a3cc..8e8086b8061de 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -698,7 +698,7 @@ func (ds *DataSource) findBestTask(prop *property.PhysicalProperty, planCounter p: dual, }, cntPlan, nil } - canConvertPointGet := len(path.Ranges) > 0 && path.StoreType != kv.TiFlash + canConvertPointGet := len(path.Ranges) > 0 && path.StoreType == kv.TiKV if canConvertPointGet && !path.IsIntHandlePath { // We simply do not build [batch] point get for prefix indexes. This can be optimized. canConvertPointGet = path.Index.Unique && !path.Index.HasPrefixIndex() @@ -826,10 +826,8 @@ func (ds *DataSource) convertToIndexMergeScan(prop *property.PhysicalProperty, c for _, partPath := range path.PartialIndexPaths { var scan PhysicalPlan var partialCost float64 - var needExtraProj bool if partPath.IsTablePath() { - scan, partialCost, needExtraProj = ds.convertToPartialTableScan(prop, partPath) - cop.needExtraProj = cop.needExtraProj || needExtraProj + scan, partialCost = ds.convertToPartialTableScan(prop, partPath) } else { scan, partialCost = ds.convertToPartialIndexScan(prop, partPath) } @@ -840,18 +838,14 @@ func (ds *DataSource) convertToIndexMergeScan(prop *property.PhysicalProperty, c if prop.ExpectedCnt < ds.stats.RowCount { totalRowCount *= prop.ExpectedCnt / ds.stats.RowCount } - ts, partialCost, needExtraProj, err := ds.buildIndexMergeTableScan(prop, path.TableFilters, totalRowCount) + ts, partialCost, err := ds.buildIndexMergeTableScan(prop, path.TableFilters, totalRowCount) if err != nil { return nil, err } - cop.needExtraProj = cop.needExtraProj || needExtraProj totalCost += partialCost cop.tablePlan = ts cop.idxMergePartPlans = scans cop.cst = totalCost - if cop.needExtraProj { - cop.originSchema = ds.schema - } task = cop.convertToRootTask(ds.ctx) return task, nil } @@ -888,19 +882,10 @@ func (ds *DataSource) convertToPartialIndexScan(prop *property.PhysicalProperty, } func (ds *DataSource) convertToPartialTableScan(prop *property.PhysicalProperty, path *util.AccessPath) ( - tablePlan PhysicalPlan, partialCost float64, needExtraProj bool) { + tablePlan PhysicalPlan, partialCost float64) { ts, partialCost, rowCount := ds.getOriginalPhysicalTableScan(prop, path, false) - if ds.tableInfo.IsCommonHandle { - commonHandle := ds.handleCols.(*CommonHandleCols) - for _, col := range commonHandle.columns { - if ts.schema.ColumnIndex(col) == -1 { - ts.Schema().Append(col) - ts.Columns = append(ts.Columns, col.ToInfo()) - needExtraProj = true - } - } - } - rowSize := ds.TblColHists.GetAvgRowSize(ds.ctx, ds.TblCols, false, false) + overwritePartialTableScanSchema(ds, ts) + rowSize := ds.TblColHists.GetAvgRowSize(ds.ctx, ts.schema.Columns, false, false) sessVars := ds.ctx.GetSessionVars() if len(ts.filterCondition) > 0 { selectivity, _, err := ds.tableStats.HistColl.Selectivity(ds.ctx, ts.filterCondition, nil) @@ -912,16 +897,50 @@ func (ds *DataSource) convertToPartialTableScan(prop *property.PhysicalProperty, tablePlan.SetChildren(ts) partialCost += rowCount * sessVars.CopCPUFactor partialCost += selectivity * rowCount * rowSize * sessVars.NetworkFactor - return + return tablePlan, partialCost } partialCost += rowCount * rowSize * sessVars.NetworkFactor tablePlan = ts + return tablePlan, partialCost +} + +// overwritePartialTableScanSchema change the schema of partial table scan to handle columns. +func overwritePartialTableScanSchema(ds *DataSource, ts *PhysicalTableScan) { + handleCols := ds.handleCols + if handleCols == nil { + handleCols = NewIntHandleCols(ds.newExtraHandleSchemaCol()) + } + hdColNum := handleCols.NumCols() + exprCols := make([]*expression.Column, 0, hdColNum) + infoCols := make([]*model.ColumnInfo, 0, hdColNum) + for i := 0; i < hdColNum; i++ { + col := handleCols.GetCol(i) + exprCols = append(exprCols, col) + infoCols = append(infoCols, col.ToInfo()) + } + ts.schema = expression.NewSchema(exprCols...) + ts.Columns = infoCols +} + +// setIndexMergeTableScanHandleCols set the handle columns of the table scan. +func setIndexMergeTableScanHandleCols(ds *DataSource, ts *PhysicalTableScan) (err error) { + handleCols := ds.handleCols + if handleCols == nil { + handleCols = NewIntHandleCols(ds.newExtraHandleSchemaCol()) + } + hdColNum := handleCols.NumCols() + exprCols := make([]*expression.Column, 0, hdColNum) + for i := 0; i < hdColNum; i++ { + col := handleCols.GetCol(i) + exprCols = append(exprCols, col) + } + ts.HandleCols, err = handleCols.ResolveIndices(expression.NewSchema(exprCols...)) return } -func (ds *DataSource) buildIndexMergeTableScan(prop *property.PhysicalProperty, tableFilters []expression.Expression, totalRowCount float64) (PhysicalPlan, float64, bool, error) { +func (ds *DataSource) buildIndexMergeTableScan(prop *property.PhysicalProperty, tableFilters []expression.Expression, + totalRowCount float64) (PhysicalPlan, float64, error) { var partialCost float64 - var needExtraProj bool sessVars := ds.ctx.GetSessionVars() ts := PhysicalTableScan{ Table: ds.tableInfo, @@ -933,27 +952,9 @@ func (ds *DataSource) buildIndexMergeTableScan(prop *property.PhysicalProperty, HandleCols: ds.handleCols, }.Init(ds.ctx, ds.blockOffset) ts.SetSchema(ds.schema.Clone()) - if ts.HandleCols == nil { - handleCol := ds.getPKIsHandleCol() - if handleCol == nil { - handleCol, _ = ts.appendExtraHandleCol(ds) - } - ts.HandleCols = NewIntHandleCols(handleCol) - } - if ds.tableInfo.IsCommonHandle { - commonHandle := ds.handleCols.(*CommonHandleCols) - for _, col := range commonHandle.columns { - if ts.schema.ColumnIndex(col) == -1 { - ts.Schema().Append(col) - ts.Columns = append(ts.Columns, col.ToInfo()) - needExtraProj = true - } - } - } - var err error - ts.HandleCols, err = ts.HandleCols.ResolveIndices(ts.schema) + err := setIndexMergeTableScanHandleCols(ds, ts) if err != nil { - return nil, 0, false, err + return nil, 0, err } if ts.Table.PKIsHandle { if pkColInfo := ts.Table.GetPkColInfo(); pkColInfo != nil { @@ -977,9 +978,9 @@ func (ds *DataSource) buildIndexMergeTableScan(prop *property.PhysicalProperty, } sel := PhysicalSelection{Conditions: tableFilters}.Init(ts.ctx, ts.stats.ScaleByExpectCnt(selectivity*totalRowCount), ts.blockOffset) sel.SetChildren(ts) - return sel, partialCost, needExtraProj, nil + return sel, partialCost, nil } - return ts, partialCost, needExtraProj, nil + return ts, partialCost, nil } func indexCoveringCol(col *expression.Column, indexCols []*expression.Column, idxColLens []int) bool { diff --git a/planner/core/handle_cols.go b/planner/core/handle_cols.go index 57ce33a49b4d5..6e7c712817c50 100644 --- a/planner/core/handle_cols.go +++ b/planner/core/handle_cols.go @@ -34,6 +34,7 @@ type HandleCols interface { // BuildHandleByDatums builds a Handle from a datum slice. BuildHandleByDatums(row []types.Datum) (kv.Handle, error) // BuildHandleFromIndexRow builds a Handle from index row data. + // The last column(s) of `row` must be the handle column(s). BuildHandleFromIndexRow(row chunk.Row) (kv.Handle, error) // ResolveIndices resolves handle column indices. ResolveIndices(schema *expression.Schema) (HandleCols, error) @@ -47,7 +48,7 @@ type HandleCols interface { NumCols() int // Compare compares two datum rows by handle order. Compare(a, b []types.Datum) (int, error) - // GetFieldTypes return field types of columns + // GetFieldTypes return field types of columns. GetFieldsTypes() []*types.FieldType } diff --git a/planner/core/integration_test.go b/planner/core/integration_test.go index 18752066280b1..d861db8548f31 100644 --- a/planner/core/integration_test.go +++ b/planner/core/integration_test.go @@ -1691,12 +1691,11 @@ func (s *testIntegrationSerialSuite) TestIssue16837(c *C) { tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int,b int,c int,d int,e int,unique key idx_ab(a,b),unique key(c),unique key(d))") tk.MustQuery("explain format = 'brief' select /*+ use_index_merge(t,c,idx_ab) */ * from t where a = 1 or (e = 1 and c = 1)").Check(testkit.Rows( - "Projection 10.00 root test.t.a, test.t.b, test.t.c, test.t.d, test.t.e", - "└─IndexMerge 0.01 root ", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:idx_ab(a, b) range:[1,1], keep order:false, stats:pseudo", - " ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t, index:c(c) range:[1,1], keep order:false, stats:pseudo", - " └─Selection(Probe) 0.01 cop[tikv] or(eq(test.t.a, 1), and(eq(test.t.e, 1), eq(test.t.c, 1)))", - " └─TableRowIDScan 11.00 cop[tikv] table:t keep order:false, stats:pseudo")) + "IndexMerge 0.01 root ", + "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:idx_ab(a, b) range:[1,1], keep order:false, stats:pseudo", + "├─IndexRangeScan(Build) 1.00 cop[tikv] table:t, index:c(c) range:[1,1], keep order:false, stats:pseudo", + "└─Selection(Probe) 0.01 cop[tikv] or(eq(test.t.a, 1), and(eq(test.t.e, 1), eq(test.t.c, 1)))", + " └─TableRowIDScan 11.00 cop[tikv] table:t keep order:false, stats:pseudo")) tk.MustQuery("show warnings").Check(testkit.Rows()) tk.MustExec("insert into t values (2, 1, 1, 1, 2)") tk.MustQuery("select /*+ use_index_merge(t,c,idx_ab) */ * from t where a = 1 or (e = 1 and c = 1)").Check(testkit.Rows()) @@ -1729,18 +1728,208 @@ func (s *testIntegrationSerialSuite) TestIndexMerge(c *C) { } } +func (s *testIntegrationSerialSuite) TestIndexMergePartialScansClusteredIndex(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test;") + + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t (a int, b int, c int, primary key (a, b) clustered, key idx_c(c));") + tk.MustExec("insert into t values (1, 1, 1), (10, 10, 10), (100, 100, 100);") + const queryTemplate = "select /*+ use_index_merge(t) */ %s from t where %s order by a, b;" + projections := [][]string{{"a"}, {"b"}, {"c"}, {"a", "b"}, {"b", "c"}, {"c", "a"}, {"b", "a", "c"}} + cases := []struct { + condition string + expected []string + }{ + { + // 3 table scans + "a < 2 or a < 10 or a > 11", []string{"1", "100"}, + }, + { + // 3 index scans + "c < 10 or c < 11 or c > 50", []string{"1", "10", "100"}, + }, + { + // 1 table scan + 1 index scan + "a < 2 or c > 10000", []string{"1"}, + }, + { + // 2 table scans + 1 index scan + "a < 2 or a > 88 or c > 10000", []string{"1", "100"}, + }, + { + // 2 table scans + 2 index scans + "a < 2 or (a >= 10 and b >= 10) or c > 100 or c < 1", []string{"1", "10", "100"}, + }, + { + // 3 table scans + 2 index scans + "a < 2 or (a >= 10 and b >= 10) or (a >= 20 and b < 10) or c > 100 or c < 1", []string{"1", "10", "100"}, + }, + } + for _, p := range projections { + for _, ca := range cases { + query := fmt.Sprintf(queryTemplate, strings.Join(p, ","), ca.condition) + tk.HasPlan(query, "IndexMerge") + expected := make([]string, 0, len(ca.expected)) + for _, datum := range ca.expected { + row := strings.Repeat(datum+" ", len(p)) + expected = append(expected, row[:len(row)-1]) + } + tk.MustQuery(query).Check(testkit.Rows(expected...)) + } + } +} + +func (s *testIntegrationSerialSuite) TestIndexMergePartialScansTiDBRowID(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test;") + + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t (a int, b int, c int, unique key (a, b), key idx_c(c));") + tk.MustExec("insert into t values (1, 1, 1), (10, 10, 10), (100, 100, 100);") + const queryTemplate = "select /*+ use_index_merge(t) */ %s from t where %s order by a;" + projections := [][]string{{"a"}, {"b"}, {"c"}, {"a", "b"}, {"b", "c"}, {"c", "a"}, {"b", "a", "c"}} + cases := []struct { + condition string + expected []string + }{ + { + // 3 index scans + "c < 10 or c < 11 or c > 50", []string{"1", "10", "100"}, + }, + { + // 2 index scans + "c < 10 or a < 2", []string{"1"}, + }, + { + // 1 table scan + 1 index scan + "_tidb_rowid < 2 or c > 10000", []string{"1"}, + }, + { + // 2 table scans + 1 index scan + "_tidb_rowid < 2 or _tidb_rowid < 10 or c > 11", []string{"1", "10", "100"}, + }, + { + // 1 table scans + 3 index scans + "_tidb_rowid < 2 or (a >= 10 and b >= 10) or c > 100 or c < 1", []string{"1", "10", "100"}, + }, + { + // 1 table scans + 4 index scans + "_tidb_rowid < 2 or (a >= 10 and b >= 10) or (a >= 20 and b < 10) or c > 100 or c < 1", []string{"1", "10", "100"}, + }, + } + for _, p := range projections { + for _, ca := range cases { + query := fmt.Sprintf(queryTemplate, strings.Join(p, ","), ca.condition) + tk.HasPlan(query, "IndexMerge") + expected := make([]string, 0, len(ca.expected)) + for _, datum := range ca.expected { + row := strings.Repeat(datum+" ", len(p)) + expected = append(expected, row[:len(row)-1]) + } + tk.MustQuery(query).Check(testkit.Rows(expected...)) + } + } +} + +func (s *testIntegrationSerialSuite) TestIndexMergePartialScansPKIsHandle(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test;") + + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t (a int, b int, c int, primary key (a), unique key (b), key idx_c(c));") + tk.MustExec("insert into t values (1, 1, 1), (10, 10, 10), (100, 100, 100);") + const queryTemplate = "select /*+ use_index_merge(t) */ %s from t where %s order by b;" + projections := [][]string{{"a"}, {"b"}, {"c"}, {"a", "b"}, {"b", "c"}, {"c", "a"}, {"b", "a", "c"}} + cases := []struct { + condition string + expected []string + }{ + { + // 3 index scans + "b < 10 or c < 11 or c > 50", []string{"1", "10", "100"}, + }, + { + // 1 table scan + 1 index scan + "a < 2 or c > 10000", []string{"1"}, + }, + { + // 2 table scans + 1 index scan + "a < 2 or a < 10 or b > 11", []string{"1", "100"}, + }, + { + // 1 table scans + 3 index scans + "a < 2 or b >= 10 or c > 100 or c < 1", []string{"1", "10", "100"}, + }, + { + // 3 table scans + 2 index scans + "a < 2 or a >= 10 or a >= 20 or c > 100 or b < 1", []string{"1", "10", "100"}, + }, + } + for _, p := range projections { + for _, ca := range cases { + query := fmt.Sprintf(queryTemplate, strings.Join(p, ","), ca.condition) + tk.HasPlan(query, "IndexMerge") + expected := make([]string, 0, len(ca.expected)) + for _, datum := range ca.expected { + row := strings.Repeat(datum+" ", len(p)) + expected = append(expected, row[:len(row)-1]) + } + tk.MustQuery(query).Check(testkit.Rows(expected...)) + } + } +} + +func (s *testIntegrationSerialSuite) TestIssue23919(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test;") + + // Test for the minimal reproducible case. + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t (a int, b int, index(a), index(b)) partition by hash (a) partitions 2;") + tk.MustExec("insert into t values (1, 5);") + tk.MustQuery("select /*+ use_index_merge( t ) */ * from t where a in (3) or b in (5) order by a;"). + Check(testkit.Rows("1 5")) + + // Test for the original case. + tk.MustExec("drop table if exists t;") + tk.MustExec(`CREATE TABLE t ( + col_5 text NOT NULL, + col_6 tinyint(3) unsigned DEFAULT NULL, + col_7 float DEFAULT '4779.165058537128', + col_8 smallint(6) NOT NULL DEFAULT '-24790', + col_9 date DEFAULT '2031-01-15', + col_37 int(11) DEFAULT '1350204687', + PRIMARY KEY (col_5(6),col_8) /*T![clustered_index] NONCLUSTERED */, + UNIQUE KEY idx_6 (col_9,col_7,col_8), + KEY idx_8 (col_8,col_6,col_5(6),col_9,col_7), + KEY idx_9 (col_9,col_7,col_8) +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin +PARTITION BY RANGE ( col_8 ) ( + PARTITION p0 VALUES LESS THAN (-17650), + PARTITION p1 VALUES LESS THAN (-13033), + PARTITION p2 VALUES LESS THAN (2521), + PARTITION p3 VALUES LESS THAN (7510) +);`) + tk.MustExec("insert into t values ('', NULL, 6304.0146, -24790, '2031-01-15', 1350204687);") + tk.MustQuery("select var_samp(col_7) aggCol from (select /*+ use_index_merge( t ) */ * from t where " + + "t.col_9 in ( '2002-06-22' ) or t.col_5 in ( 'PkfzI' ) or t.col_8 in ( -24874 ) and t.col_6 > null and " + + "t.col_5 > 'r' and t.col_9 in ( '1979-09-04' ) and t.col_7 < 8143.667552769195 or " + + "t.col_5 in ( 'iZhfEjRWci' , 'T' , '' ) or t.col_9 <> '1976-09-11' and t.col_7 = 8796.436181615773 and " + + "t.col_8 = 7372 order by col_5,col_8 ) ordered_tbl group by col_6;").Check(testkit.Rows("")) +} + func (s *testIntegrationSerialSuite) TestIssue16407(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int,b char(100),key(a),key(b(10)))") tk.MustQuery("explain format = 'brief' select /*+ use_index_merge(t) */ * from t where a=10 or b='x'").Check(testkit.Rows( - "Projection 19.99 root test.t.a, test.t.b", - "└─IndexMerge 0.04 root ", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:a(a) range:[10,10], keep order:false, stats:pseudo", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:b(b) range:[\"x\",\"x\"], keep order:false, stats:pseudo", - " └─Selection(Probe) 0.04 cop[tikv] or(eq(test.t.a, 10), eq(test.t.b, \"x\"))", - " └─TableRowIDScan 19.99 cop[tikv] table:t keep order:false, stats:pseudo")) + "IndexMerge 0.04 root ", + "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:a(a) range:[10,10], keep order:false, stats:pseudo", + "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:b(b) range:[\"x\",\"x\"], keep order:false, stats:pseudo", + "└─Selection(Probe) 0.04 cop[tikv] or(eq(test.t.a, 10), eq(test.t.b, \"x\"))", + " └─TableRowIDScan 19.99 cop[tikv] table:t keep order:false, stats:pseudo")) tk.MustQuery("show warnings").Check(testkit.Rows()) tk.MustExec("insert into t values (1, 'xx')") tk.MustQuery("select /*+ use_index_merge(t) */ * from t where a=10 or b='x'").Check(testkit.Rows()) @@ -3150,23 +3339,21 @@ func (s *testIntegrationSuite) TestIndexMergeTableFilter(c *C) { tk.MustExec("insert into t values(10,1,1,10)") tk.MustQuery("explain format = 'brief' select /*+ use_index_merge(t) */ * from t where a=10 or (b=10 and c=10)").Check(testkit.Rows( - "Projection 10.01 root test.t.a, test.t.b, test.t.c, test.t.d", - "└─IndexMerge 0.02 root ", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:a(a) range:[10,10], keep order:false, stats:pseudo", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:b(b) range:[10,10], keep order:false, stats:pseudo", - " └─Selection(Probe) 0.02 cop[tikv] or(eq(test.t.a, 10), and(eq(test.t.b, 10), eq(test.t.c, 10)))", - " └─TableRowIDScan 19.99 cop[tikv] table:t keep order:false, stats:pseudo", + "IndexMerge 0.02 root ", + "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:a(a) range:[10,10], keep order:false, stats:pseudo", + "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:b(b) range:[10,10], keep order:false, stats:pseudo", + "└─Selection(Probe) 0.02 cop[tikv] or(eq(test.t.a, 10), and(eq(test.t.b, 10), eq(test.t.c, 10)))", + " └─TableRowIDScan 19.99 cop[tikv] table:t keep order:false, stats:pseudo", )) tk.MustQuery("select /*+ use_index_merge(t) */ * from t where a=10 or (b=10 and c=10)").Check(testkit.Rows( "10 1 1 10", )) tk.MustQuery("explain format = 'brief' select /*+ use_index_merge(t) */ * from t where (a=10 and d=10) or (b=10 and c=10)").Check(testkit.Rows( - "Projection 0.02 root test.t.a, test.t.b, test.t.c, test.t.d", - "└─IndexMerge 0.00 root ", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:a(a) range:[10,10], keep order:false, stats:pseudo", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:b(b) range:[10,10], keep order:false, stats:pseudo", - " └─Selection(Probe) 0.00 cop[tikv] or(and(eq(test.t.a, 10), eq(test.t.d, 10)), and(eq(test.t.b, 10), eq(test.t.c, 10)))", - " └─TableRowIDScan 19.99 cop[tikv] table:t keep order:false, stats:pseudo", + "IndexMerge 0.00 root ", + "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:a(a) range:[10,10], keep order:false, stats:pseudo", + "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:b(b) range:[10,10], keep order:false, stats:pseudo", + "└─Selection(Probe) 0.00 cop[tikv] or(and(eq(test.t.a, 10), eq(test.t.d, 10)), and(eq(test.t.b, 10), eq(test.t.c, 10)))", + " └─TableRowIDScan 19.99 cop[tikv] table:t keep order:false, stats:pseudo", )) tk.MustQuery("select /*+ use_index_merge(t) */ * from t where (a=10 and d=10) or (b=10 and c=10)").Check(testkit.Rows( "10 1 1 10", @@ -3339,3 +3526,26 @@ func (s *testIntegrationSuite) TestIssue23839(c *C) { ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin AUTO_INCREMENT=2000001") tk.Exec("explain SELECT OUTR . col2 AS X FROM (SELECT INNR . col1 as col1, SUM( INNR . col2 ) as col2 FROM (SELECT INNR . `col_int_not_null` + 1 as col1, INNR . `pk` as col2 FROM BB AS INNR) AS INNR GROUP BY col1) AS OUTR2 INNER JOIN (SELECT INNR . col1 as col1, MAX( INNR . col2 ) as col2 FROM (SELECT INNR . `col_int_not_null` + 1 as col1, INNR . `pk` as col2 FROM BB AS INNR) AS INNR GROUP BY col1) AS OUTR ON OUTR2.col1 = OUTR.col1 GROUP BY OUTR . col1, OUTR2 . col1 HAVING X <> 'b'") } + +// https://github.com/pingcap/tidb/issues/24095 +func (s *testIntegrationSuite) TestIssue24095(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test;") + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t (id int, value decimal(10,5));") + tk.MustExec("desc format = 'brief' select count(*) from t join (select t.id, t.value v1 from t join t t1 on t.id = t1.id order by t.value limit 1) v on v.id = t.id and v.v1 = t.value;") + + var input []string + var output []struct { + SQL string + Plan []string + } + s.testData.GetTestCases(c, &input, &output) + for i, tt := range input { + s.testData.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + tt).Rows()) + }) + tk.MustQuery("explain format = 'brief' " + tt).Check(testkit.Rows(output[i].Plan...)) + } +} diff --git a/planner/core/partition_pruner_test.go b/planner/core/partition_pruner_test.go index 0d9a66c7dbbda..6f764adfb6341 100644 --- a/planner/core/partition_pruner_test.go +++ b/planner/core/partition_pruner_test.go @@ -39,16 +39,6 @@ type testPartitionPruneSuit struct { testData testutil.TestData } -func (s *testPartitionPruneSuit) cleanEnv(c *C) { - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test_partition") - r := tk.MustQuery("show tables") - for _, tb := range r.Rows() { - tableName := tb[0] - tk.MustExec(fmt.Sprintf("drop table %v", tableName)) - } -} - func (s *testPartitionPruneSuit) SetUpSuite(c *C) { var err error s.store, s.dom, err = newStoreWithBootstrap() diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index f955459ffe495..68679747eaf34 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -481,11 +481,6 @@ type handleColHelper struct { stackTail int } -func (hch *handleColHelper) appendColToLastMap(tblID int64, handleCols HandleCols) { - tailMap := hch.id2HandleMapStack[hch.stackTail-1] - tailMap[tblID] = append(tailMap[tblID], handleCols) -} - func (hch *handleColHelper) popMap() map[int64][]HandleCols { ret := hch.id2HandleMapStack[hch.stackTail-1] hch.stackTail-- diff --git a/planner/core/rule_aggregation_elimination.go b/planner/core/rule_aggregation_elimination.go index f2c0ee36aaf62..079b2a76bace5 100644 --- a/planner/core/rule_aggregation_elimination.go +++ b/planner/core/rule_aggregation_elimination.go @@ -140,7 +140,7 @@ func rewriteExpr(ctx sessionctx.Context, aggFunc *aggregation.AggFuncDesc) (bool func rewriteCount(ctx sessionctx.Context, exprs []expression.Expression, targetTp *types.FieldType) expression.Expression { // If is count(expr), we will change it to if(isnull(expr), 0, 1). - // If is count(distinct x, y, z) we will change it to if(isnull(x) or isnull(y) or isnull(z), 0, 1). + // If is count(distinct x, y, z), we will change it to if(isnull(x) or isnull(y) or isnull(z), 0, 1). // If is count(expr not null), we will change it to constant 1. isNullExprs := make([]expression.Expression, 0, len(exprs)) for _, expr := range exprs { diff --git a/planner/core/rule_column_pruning.go b/planner/core/rule_column_pruning.go index e55c7b2dea17b..7b66ce1c2eb87 100644 --- a/planner/core/rule_column_pruning.go +++ b/planner/core/rule_column_pruning.go @@ -389,7 +389,7 @@ func (p *LogicalLock) PruneColumns(parentUsedCols []*expression.Column) error { // PruneColumns implements LogicalPlan interface. func (p *LogicalWindow) PruneColumns(parentUsedCols []*expression.Column) error { windowColumns := p.GetWindowResultColumns() - len := 0 + cnt := 0 for _, col := range parentUsedCols { used := false for _, windowColumn := range windowColumns { @@ -399,11 +399,11 @@ func (p *LogicalWindow) PruneColumns(parentUsedCols []*expression.Column) error } } if !used { - parentUsedCols[len] = col - len++ + parentUsedCols[cnt] = col + cnt++ } } - parentUsedCols = parentUsedCols[:len] + parentUsedCols = parentUsedCols[:cnt] parentUsedCols = p.extractUsedCols(parentUsedCols) err := p.children[0].PruneColumns(parentUsedCols) if err != nil { diff --git a/planner/core/rule_join_reorder.go b/planner/core/rule_join_reorder.go index 08aacdbc8683a..2aa0ced4c1f7d 100644 --- a/planner/core/rule_join_reorder.go +++ b/planner/core/rule_join_reorder.go @@ -91,10 +91,14 @@ func (s *joinReOrderSolver) optimizeRecursive(ctx sessionctx.Context, p LogicalP return nil, err } schemaChanged := false - for i, col := range p.Schema().Columns { - if !col.Equal(nil, originalSchema.Columns[i]) { - schemaChanged = true - break + if len(p.Schema().Columns) != len(originalSchema.Columns) { + schemaChanged = true + } else { + for i, col := range p.Schema().Columns { + if !col.Equal(nil, originalSchema.Columns[i]) { + schemaChanged = true + break + } } } if schemaChanged { diff --git a/planner/core/testdata/integration_serial_suite_out.json b/planner/core/testdata/integration_serial_suite_out.json index cda112427f14c..d262608fb7238 100644 --- a/planner/core/testdata/integration_serial_suite_out.json +++ b/planner/core/testdata/integration_serial_suite_out.json @@ -2494,49 +2494,45 @@ { "SQL": "desc format='brief' select /*+ use_index_merge(t) */ * from t where a =1 or (b=1 and b+2>1)", "Plan": [ - "Projection 8.00 root test.t.a, test.t.b", - "└─IndexMerge 8.00 root ", - " ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t, index:a(a) range:[1,1], keep order:false", - " ├─Selection(Build) 0.80 cop[tikv] 1", - " │ └─IndexRangeScan 1.00 cop[tikv] table:t, index:b(b) range:[1,1], keep order:false", - " └─TableRowIDScan(Probe) 8.00 cop[tikv] table:t keep order:false" + "IndexMerge 8.00 root ", + "├─IndexRangeScan(Build) 1.00 cop[tikv] table:t, index:a(a) range:[1,1], keep order:false", + "├─Selection(Build) 0.80 cop[tikv] 1", + "│ └─IndexRangeScan 1.00 cop[tikv] table:t, index:b(b) range:[1,1], keep order:false", + "└─TableRowIDScan(Probe) 8.00 cop[tikv] table:t keep order:false" ], "Warnings": null }, { "SQL": "desc format='brief' select /*+ use_index_merge(t) */ * from t where a =1 or (b=1 and length(b)=1)", "Plan": [ - "Projection 1.72 root test.t.a, test.t.b", - "└─IndexMerge 1.72 root ", - " ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t, index:a(a) range:[1,1], keep order:false", - " ├─Selection(Build) 0.80 cop[tikv] eq(length(cast(1, var_string(20))), 1)", - " │ └─IndexRangeScan 1.00 cop[tikv] table:t, index:b(b) range:[1,1], keep order:false", - " └─TableRowIDScan(Probe) 1.72 cop[tikv] table:t keep order:false" + "IndexMerge 1.72 root ", + "├─IndexRangeScan(Build) 1.00 cop[tikv] table:t, index:a(a) range:[1,1], keep order:false", + "├─Selection(Build) 0.80 cop[tikv] eq(length(cast(1, var_string(20))), 1)", + "│ └─IndexRangeScan 1.00 cop[tikv] table:t, index:b(b) range:[1,1], keep order:false", + "└─TableRowIDScan(Probe) 1.72 cop[tikv] table:t keep order:false" ], "Warnings": null }, { "SQL": "desc format='brief' select /*+ use_index_merge(t) */ * from t where (a=1 and length(a)=1) or (b=1 and length(b)=1)", "Plan": [ - "Projection 1.54 root test.t.a, test.t.b", - "└─IndexMerge 1.54 root ", - " ├─Selection(Build) 0.80 cop[tikv] eq(length(cast(1, var_string(20))), 1)", - " │ └─IndexRangeScan 1.00 cop[tikv] table:t, index:a(a) range:[1,1], keep order:false", - " ├─Selection(Build) 0.80 cop[tikv] eq(length(cast(1, var_string(20))), 1)", - " │ └─IndexRangeScan 1.00 cop[tikv] table:t, index:b(b) range:[1,1], keep order:false", - " └─TableRowIDScan(Probe) 1.54 cop[tikv] table:t keep order:false" + "IndexMerge 1.54 root ", + "├─Selection(Build) 0.80 cop[tikv] eq(length(cast(1, var_string(20))), 1)", + "│ └─IndexRangeScan 1.00 cop[tikv] table:t, index:a(a) range:[1,1], keep order:false", + "├─Selection(Build) 0.80 cop[tikv] eq(length(cast(1, var_string(20))), 1)", + "│ └─IndexRangeScan 1.00 cop[tikv] table:t, index:b(b) range:[1,1], keep order:false", + "└─TableRowIDScan(Probe) 1.54 cop[tikv] table:t keep order:false" ], "Warnings": null }, { "SQL": "desc format='brief' select /*+ use_index_merge(t) */ * from t where (a=1 and length(b)=1) or (b=1 and length(a)=1)", "Plan": [ - "Projection 1.54 root test.t.a, test.t.b", - "└─IndexMerge 0.29 root ", - " ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t, index:a(a) range:[1,1], keep order:false", - " ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t, index:b(b) range:[1,1], keep order:false", - " └─Selection(Probe) 0.29 cop[tikv] or(and(eq(test.t.a, 1), eq(length(cast(test.t.b, var_string(20))), 1)), and(eq(test.t.b, 1), eq(length(cast(test.t.a, var_string(20))), 1)))", - " └─TableRowIDScan 1.90 cop[tikv] table:t keep order:false" + "IndexMerge 0.29 root ", + "├─IndexRangeScan(Build) 1.00 cop[tikv] table:t, index:a(a) range:[1,1], keep order:false", + "├─IndexRangeScan(Build) 1.00 cop[tikv] table:t, index:b(b) range:[1,1], keep order:false", + "└─Selection(Probe) 0.29 cop[tikv] or(and(eq(test.t.a, 1), eq(length(cast(test.t.b, var_string(20))), 1)), and(eq(test.t.b, 1), eq(length(cast(test.t.a, var_string(20))), 1)))", + " └─TableRowIDScan 1.90 cop[tikv] table:t keep order:false" ], "Warnings": null } diff --git a/planner/core/testdata/integration_suite_in.json b/planner/core/testdata/integration_suite_in.json index 09b34323beeef..7a734a9b9a4cc 100644 --- a/planner/core/testdata/integration_suite_in.json +++ b/planner/core/testdata/integration_suite_in.json @@ -279,5 +279,11 @@ "select (select c from t2 where t2.a = t1.a and t2.b = 1) from t1", "select (select c from t2 where t2.a = t1.a and (t2.b = 1 or t2.b = 2)) from t1" ] + }, + { + "name": "TestIssue24095", + "cases": [ + "select count(*) from t join (select t.id, t.value v1 from t join t t1 on t.id = t1.id order by t.value limit 1) v on v.id = t.id and v.v1 = t.value;" + ] } ] diff --git a/planner/core/testdata/integration_suite_out.json b/planner/core/testdata/integration_suite_out.json index d3cb7bd7ecec8..06382a91e38b1 100644 --- a/planner/core/testdata/integration_suite_out.json +++ b/planner/core/testdata/integration_suite_out.json @@ -1246,21 +1246,19 @@ { "SQL": "select * from pt where id = 4 or c < 7", "Plan": [ - "Projection_4 3330.01 root test.pt.id, test.pt.c", - "└─IndexMerge_11 3330.01 root partition:all ", - " ├─IndexRangeScan_8(Build) 10.00 cop[tikv] table:pt, index:i_id(id) range:[4,4], keep order:false, stats:pseudo", - " ├─IndexRangeScan_9(Build) 3323.33 cop[tikv] table:pt, index:i_c(c) range:[-inf,7), keep order:false, stats:pseudo", - " └─TableRowIDScan_10(Probe) 3330.01 cop[tikv] table:pt keep order:false, stats:pseudo" + "IndexMerge_11 3330.01 root partition:all ", + "├─IndexRangeScan_8(Build) 10.00 cop[tikv] table:pt, index:i_id(id) range:[4,4], keep order:false, stats:pseudo", + "├─IndexRangeScan_9(Build) 3323.33 cop[tikv] table:pt, index:i_c(c) range:[-inf,7), keep order:false, stats:pseudo", + "└─TableRowIDScan_10(Probe) 3330.01 cop[tikv] table:pt keep order:false, stats:pseudo" ] }, { "SQL": "select * from pt where id > 4 or c = 7", "Plan": [ - "Projection_4 3340.00 root test.pt.id, test.pt.c", - "└─IndexMerge_11 3340.00 root partition:all ", - " ├─IndexRangeScan_8(Build) 3333.33 cop[tikv] table:pt, index:i_id(id) range:(4,+inf], keep order:false, stats:pseudo", - " ├─IndexRangeScan_9(Build) 10.00 cop[tikv] table:pt, index:i_c(c) range:[7,7], keep order:false, stats:pseudo", - " └─TableRowIDScan_10(Probe) 3340.00 cop[tikv] table:pt keep order:false, stats:pseudo" + "IndexMerge_11 3340.00 root partition:all ", + "├─IndexRangeScan_8(Build) 3333.33 cop[tikv] table:pt, index:i_id(id) range:(4,+inf], keep order:false, stats:pseudo", + "├─IndexRangeScan_9(Build) 10.00 cop[tikv] table:pt, index:i_c(c) range:[7,7], keep order:false, stats:pseudo", + "└─TableRowIDScan_10(Probe) 3340.00 cop[tikv] table:pt keep order:false, stats:pseudo" ] } ] @@ -1511,5 +1509,30 @@ ] } ] + }, + { + "Name": "TestIssue24095", + "Cases": [ + { + "SQL": "select count(*) from t join (select t.id, t.value v1 from t join t t1 on t.id = t1.id order by t.value limit 1) v on v.id = t.id and v.v1 = t.value;", + "Plan": [ + "StreamAgg 1.00 root funcs:count(1)->Column#10", + "└─Projection 1.00 root 1->Column#11", + " └─HashJoin 1.00 root inner join, equal:[eq(test.t.id, test.t.id) eq(test.t.value, test.t.value)]", + " ├─Selection(Build) 0.80 root not(isnull(test.t.id)), not(isnull(test.t.value))", + " │ └─TopN 1.00 root test.t.value, offset:0, count:1", + " │ └─HashJoin 12487.50 root inner join, equal:[eq(test.t.id, test.t.id)]", + " │ ├─TableReader(Build) 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t.id))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + " │ └─TableReader(Probe) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t.id))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + " └─TableReader(Probe) 9980.01 root data:Selection", + " └─Selection 9980.01 cop[tikv] not(isnull(test.t.id)), not(isnull(test.t.value))", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + } + ] } ] diff --git a/server/http_handler.go b/server/http_handler.go index a8e82ddcb618a..3c20518db0aa4 100644 --- a/server/http_handler.go +++ b/server/http_handler.go @@ -729,9 +729,9 @@ func (h settingsHandler) ServeHTTP(w http.ResponseWriter, req *http.Request) { } switch asyncCommit { case "0": - err = s.GetSessionVars().GlobalVarsAccessor.SetGlobalSysVar(variable.TiDBEnableAsyncCommit, variable.BoolOff) + err = s.GetSessionVars().GlobalVarsAccessor.SetGlobalSysVar(variable.TiDBEnableAsyncCommit, variable.Off) case "1": - err = s.GetSessionVars().GlobalVarsAccessor.SetGlobalSysVar(variable.TiDBEnableAsyncCommit, variable.BoolOn) + err = s.GetSessionVars().GlobalVarsAccessor.SetGlobalSysVar(variable.TiDBEnableAsyncCommit, variable.On) default: writeError(w, errors.New("illegal argument")) return @@ -752,9 +752,9 @@ func (h settingsHandler) ServeHTTP(w http.ResponseWriter, req *http.Request) { } switch onePC { case "0": - err = s.GetSessionVars().GlobalVarsAccessor.SetGlobalSysVar(variable.TiDBEnable1PC, variable.BoolOff) + err = s.GetSessionVars().GlobalVarsAccessor.SetGlobalSysVar(variable.TiDBEnable1PC, variable.Off) case "1": - err = s.GetSessionVars().GlobalVarsAccessor.SetGlobalSysVar(variable.TiDBEnable1PC, variable.BoolOn) + err = s.GetSessionVars().GlobalVarsAccessor.SetGlobalSysVar(variable.TiDBEnable1PC, variable.On) default: writeError(w, errors.New("illegal argument")) return diff --git a/server/http_handler_test.go b/server/http_handler_test.go index 7f6c86aa75978..583f5a8a2dca2 100644 --- a/server/http_handler_test.go +++ b/server/http_handler_test.go @@ -1156,10 +1156,10 @@ func (ts *HTTPHandlerTestSuite) TestPostSettings(c *C) { c.Assert(variable.ProcessGeneralLog.Load(), IsTrue) val, err := variable.GetGlobalSystemVar(se.GetSessionVars(), variable.TiDBEnableAsyncCommit) c.Assert(err, IsNil) - c.Assert(val, Equals, variable.BoolOn) + c.Assert(val, Equals, variable.On) val, err = variable.GetGlobalSystemVar(se.GetSessionVars(), variable.TiDBEnable1PC) c.Assert(err, IsNil) - c.Assert(val, Equals, variable.BoolOn) + c.Assert(val, Equals, variable.On) form = make(url.Values) form.Set("log_level", "fatal") @@ -1175,10 +1175,10 @@ func (ts *HTTPHandlerTestSuite) TestPostSettings(c *C) { c.Assert(config.GetGlobalConfig().Log.Level, Equals, "fatal") val, err = variable.GetGlobalSystemVar(se.GetSessionVars(), variable.TiDBEnableAsyncCommit) c.Assert(err, IsNil) - c.Assert(val, Equals, variable.BoolOff) + c.Assert(val, Equals, variable.Off) val, err = variable.GetGlobalSystemVar(se.GetSessionVars(), variable.TiDBEnable1PC) c.Assert(err, IsNil) - c.Assert(val, Equals, variable.BoolOff) + c.Assert(val, Equals, variable.Off) form.Set("log_level", os.Getenv("log_level")) // test ddl_slow_threshold diff --git a/session/bootstrap.go b/session/bootstrap.go index e1c1ecb067c9d..34d6748ae38c1 100644 --- a/session/bootstrap.go +++ b/session/bootstrap.go @@ -1599,17 +1599,17 @@ func doDMLWorks(s Session) { } } if v.Name == variable.TiDBEnableChangeMultiSchema { - vVal = variable.BoolOff + vVal = variable.Off if flag.Lookup("test.v") != nil || flag.Lookup("check.v") != nil { // enable change multi schema in test case for compatibility with old cases. - vVal = variable.BoolOn + vVal = variable.On } } if v.Name == variable.TiDBEnableAsyncCommit && config.GetGlobalConfig().Store == "tikv" { - vVal = variable.BoolOn + vVal = variable.On } if v.Name == variable.TiDBEnable1PC && config.GetGlobalConfig().Store == "tikv" { - vVal = variable.BoolOn + vVal = variable.On } value := fmt.Sprintf(`("%s", "%s")`, strings.ToLower(k), vVal) values = append(values, value) diff --git a/session/pessimistic_test.go b/session/pessimistic_test.go index 1e1b597b7058a..2a591c782c7aa 100644 --- a/session/pessimistic_test.go +++ b/session/pessimistic_test.go @@ -214,7 +214,8 @@ func (s *testPessimisticSuite) TestSingleStatementRollback(c *C) { tk.MustExec("create table single_statement (id int primary key, v int)") tk.MustExec("insert into single_statement values (1, 1), (2, 1), (3, 1), (4, 1)") tblID := tk.GetTableID("single_statement") - s.cluster.SplitTable(tblID, 2) + tableStart := tablecodec.GenTableRecordPrefix(tblID) + s.cluster.SplitKeys(tableStart, tableStart.PrefixNext(), 2) region1Key := codec.EncodeBytes(nil, tablecodec.EncodeRowKeyWithHandle(tblID, kv.IntHandle(1))) region1, _ := s.cluster.GetRegionByKey(region1Key) region1ID := region1.Id @@ -2058,9 +2059,9 @@ func (s *testPessimisticSuite) TestAsyncCommitWithSchemaChange(c *C) { conf.TiKVClient.AsyncCommit.SafeWindow = time.Second conf.TiKVClient.AsyncCommit.AllowedClockDrift = 0 }) - c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/beforeSchemaCheck", "return"), IsNil) + c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/asyncCommitDoNothing", "return"), IsNil) defer func() { - c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/beforeSchemaCheck"), IsNil) + c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/asyncCommitDoNothing"), IsNil) }() tk := s.newAsyncCommitTestKitWithInit(c) @@ -2108,18 +2109,20 @@ func (s *testPessimisticSuite) TestAsyncCommitWithSchemaChange(c *C) { tk.MustExec("create table tk (c1 int primary key, c2 int)") tk.MustExec("begin pessimistic") tk.MustExec("insert into tk values(1, 1)") + c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/beforePrewrite", "1*pause"), IsNil) go func() { time.Sleep(200 * time.Millisecond) tk2.MustExec("alter table tk add index k2(c2)") + c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/beforePrewrite"), IsNil) + ch <- struct{}{} }() - c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/beforePrewrite", "1*sleep(1200)"), IsNil) tk.MustExec("commit") - c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/beforePrewrite"), IsNil) + <-ch + tk.MustQuery("select * from tk where c2 = 1").Check(testkit.Rows("1 1")) tk3.MustExec("admin check table tk") } func (s *testPessimisticSuite) Test1PCWithSchemaChange(c *C) { - c.Skip("unstable") // TODO: implement commit_ts calculation in unistore if !*withTiKV { return @@ -2165,13 +2168,16 @@ func (s *testPessimisticSuite) Test1PCWithSchemaChange(c *C) { tk.MustExec("create table tk (c1 int primary key, c2 int)") tk.MustExec("begin pessimistic") tk.MustExec("insert into tk values(1, 1)") + c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/beforePrewrite", "1*pause"), IsNil) go func() { time.Sleep(200 * time.Millisecond) tk2.MustExec("alter table tk add index k2(c2)") + c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/beforePrewrite"), IsNil) + ch <- struct{}{} }() - c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/beforePrewrite", "1*sleep(1200)"), IsNil) tk.MustExec("commit") - c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/beforePrewrite"), IsNil) + <-ch + tk.MustQuery("select * from tk where c2 = 1").Check(testkit.Rows("1 1")) tk3.MustExec("admin check table tk") } diff --git a/session/session.go b/session/session.go index 924bffd6e8c6e..6018406f2f472 100644 --- a/session/session.go +++ b/session/session.go @@ -353,6 +353,9 @@ func (s *session) SetCollation(coID int) error { if err != nil { return err } + // If new collations are enabled, switch to the default + // collation if this one is not supported. + co = collate.SubstituteMissingCollationToDefault(co) for _, v := range variable.SetNamesVariables { terror.Log(s.sessionVars.SetSystemVar(v, cs)) } @@ -509,6 +512,16 @@ func (s *session) doCommit(ctx context.Context) error { return s.txn.Commit(tikvutil.SetSessionID(ctx, s.GetSessionVars().ConnectionID)) } +// errIsNoisy is used to filter DUPLCATE KEY errors. +// These can observed by users in INFORMATION_SCHEMA.CLIENT_ERRORS_SUMMARY_GLOBAL instead. +// +// The rationale for filtering these errors is because they are "client generated errors". i.e. +// of the errors defined in kv/error.go, these look to be clearly related to a client-inflicted issue, +// and the server is only responsible for handling the error correctly. It does not need to log. +func errIsNoisy(err error) bool { + return kv.ErrKeyExists.Equal(err) +} + func (s *session) doCommitWithRetry(ctx context.Context) error { defer func() { s.GetSessionVars().SetTxnIsolationLevelOneShotStateForNextTxn() @@ -546,7 +559,7 @@ func (s *session) doCommitWithRetry(ctx context.Context) error { txnSizeRate := float64(txnSize) / float64(kv.TxnTotalSizeLimit) maxRetryCount := commitRetryLimit - int64(float64(commitRetryLimit-1)*txnSizeRate) err = s.retry(ctx, uint(maxRetryCount)) - } else { + } else if !errIsNoisy(err) { logutil.Logger(ctx).Warn("can not retry txn", zap.String("label", s.getSQLLabel()), zap.Error(err), @@ -562,9 +575,11 @@ func (s *session) doCommitWithRetry(ctx context.Context) error { s.recordOnTransactionExecution(err, counter, duration) if err != nil { - logutil.Logger(ctx).Warn("commit failed", - zap.String("finished txn", s.txn.GoString()), - zap.Error(err)) + if !errIsNoisy(err) { + logutil.Logger(ctx).Warn("commit failed", + zap.String("finished txn", s.txn.GoString()), + zap.Error(err)) + } return err } mapper := s.GetSessionVars().TxnCtx.TableDeltaMap @@ -1014,9 +1029,9 @@ func (s *session) setTiDBTableValue(name, val string) error { // but sysvars use the convention ON/OFF. func trueFalseToOnOff(str string) string { if strings.EqualFold("true", str) { - return variable.BoolOn + return variable.On } else if strings.EqualFold("false", str) { - return variable.BoolOff + return variable.Off } return str } @@ -2550,6 +2565,7 @@ var builtinGlobalVariable = []string{ variable.TiDBEnableExchangePartition, variable.TiDBAllowFallbackToTiKV, variable.TiDBEnableDynamicPrivileges, + variable.CTEMaxRecursionDepth, } // loadCommonGlobalVariablesIfNeeded loads and applies commonly used global variables for the session. diff --git a/session/session_fail_test.go b/session/session_fail_test.go index 8bf7e277cad57..12f49e0ed1abf 100644 --- a/session/session_fail_test.go +++ b/session/session_fail_test.go @@ -19,7 +19,7 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/failpoint" - "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/util/testkit" ) @@ -84,7 +84,7 @@ func (s *testSessionSerialSuite) TestKillFlagInBackoff(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) tk.MustExec("create table kill_backoff (id int)") var killValue uint32 - tk.Se.GetSessionVars().KVVars.Hook = func(name string, vars *kv.Variables) { + tk.Se.GetSessionVars().KVVars.Hook = func(name string, vars *tikv.Variables) { killValue = atomic.LoadUint32(vars.Killed) } c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/tikvStoreSendReqResult", `return("callBackofferHook")`), IsNil) diff --git a/session/session_test.go b/session/session_test.go index 7109a73e6ec88..9ce875fa07868 100644 --- a/session/session_test.go +++ b/session/session_test.go @@ -55,6 +55,7 @@ import ( "github.com/pingcap/tidb/store/tikv/oracle" tikvutil "github.com/pingcap/tidb/store/tikv/util" "github.com/pingcap/tidb/table/tables" + "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/sqlexec" "github.com/pingcap/tidb/util/testkit" @@ -2215,7 +2216,8 @@ func (s *testSchemaSuite) TestTableReaderChunk(c *C) { } tbl, err := domain.GetDomain(tk.Se).InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("chk")) c.Assert(err, IsNil) - s.cluster.SplitTable(tbl.Meta().ID, 10) + tableStart := tablecodec.GenTableRecordPrefix(tbl.Meta().ID) + s.cluster.SplitKeys(tableStart, tableStart.PrefixNext(), 10) tk.Se.GetSessionVars().SetDistSQLScanConcurrency(1) tk.MustExec("set tidb_init_chunk_size = 2") @@ -2400,7 +2402,8 @@ func (s *testSchemaSuite) TestIndexLookUpReaderChunk(c *C) { } tbl, err := domain.GetDomain(tk.Se).InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("chk")) c.Assert(err, IsNil) - s.cluster.SplitIndex(tbl.Meta().ID, tbl.Indices()[0].Meta().ID, 10) + indexStart := tablecodec.EncodeTableIndexPrefix(tbl.Meta().ID, tbl.Indices()[0].Meta().ID) + s.cluster.SplitKeys(indexStart, indexStart.PrefixNext(), 10) tk.Se.GetSessionVars().IndexLookupSize = 10 rs, err := tk.Exec("select * from chk order by k") @@ -2655,7 +2658,7 @@ func (s *testSessionSerialSuite) TestKVVars(c *C) { tk.MustExec("begin") txn, err := tk.Se.Txn(false) c.Assert(err, IsNil) - vars := txn.GetVars() + vars := txn.GetVars().(*tikv.Variables) c.Assert(vars.BackoffLockFast, Equals, 1) c.Assert(vars.BackOffWeight, Equals, 100) tk.MustExec("rollback") @@ -2665,7 +2668,7 @@ func (s *testSessionSerialSuite) TestKVVars(c *C) { c.Assert(tk.Se.GetSessionVars().InTxn(), IsTrue) txn, err = tk.Se.Txn(false) c.Assert(err, IsNil) - vars = txn.GetVars() + vars = txn.GetVars().(*tikv.Variables) c.Assert(vars.BackOffWeight, Equals, 50) tk.MustExec("set @@autocommit = 1") @@ -3254,7 +3257,7 @@ func (s *testSessionSuite2) TestPerStmtTaskID(c *C) { } func (s *testSessionSerialSuite) TestSetTxnScope(c *C) { - failpoint.Enable("github.com/pingcap/tidb/config/injectTxnScope", `return("")`) + failpoint.Enable("github.com/pingcap/tidb/store/tikv/config/injectTxnScope", `return("")`) tk := testkit.NewTestKitWithInit(c, s.store) // assert default value result := tk.MustQuery("select @@txn_scope;") @@ -3265,9 +3268,9 @@ func (s *testSessionSerialSuite) TestSetTxnScope(c *C) { result = tk.MustQuery("select @@txn_scope;") result.Check(testkit.Rows(oracle.GlobalTxnScope)) c.Assert(tk.Se.GetSessionVars().CheckAndGetTxnScope(), Equals, oracle.GlobalTxnScope) - failpoint.Disable("github.com/pingcap/tidb/config/injectTxnScope") - failpoint.Enable("github.com/pingcap/tidb/config/injectTxnScope", `return("bj")`) - defer failpoint.Disable("github.com/pingcap/tidb/config/injectTxnScope") + failpoint.Disable("github.com/pingcap/tidb/store/tikv/config/injectTxnScope") + failpoint.Enable("github.com/pingcap/tidb/store/tikv/config/injectTxnScope", `return("bj")`) + defer failpoint.Disable("github.com/pingcap/tidb/store/tikv/config/injectTxnScope") tk = testkit.NewTestKitWithInit(c, s.store) // assert default value result = tk.MustQuery("select @@txn_scope;") @@ -3375,8 +3378,8 @@ PARTITION BY RANGE (c) ( result = tk.MustQuery("select * from t1") // read dc-1 and dc-2 with global scope c.Assert(len(result.Rows()), Equals, 3) - failpoint.Enable("github.com/pingcap/tidb/config/injectTxnScope", `return("dc-1")`) - defer failpoint.Disable("github.com/pingcap/tidb/config/injectTxnScope") + failpoint.Enable("github.com/pingcap/tidb/store/tikv/config/injectTxnScope", `return("dc-1")`) + defer failpoint.Disable("github.com/pingcap/tidb/store/tikv/config/injectTxnScope") // set txn_scope to local tk.MustExec("set @@session.txn_scope = 'local';") result = tk.MustQuery("select @@txn_scope;") diff --git a/sessionctx/binloginfo/binloginfo_test.go b/sessionctx/binloginfo/binloginfo_test.go index 78cb30ada9a9c..ca01fe0fc90e3 100644 --- a/sessionctx/binloginfo/binloginfo_test.go +++ b/sessionctx/binloginfo/binloginfo_test.go @@ -540,6 +540,19 @@ func (s *testBinlogSuite) TestPartitionedTable(c *C) { } } +func (s *testBinlogSuite) TestPessimisticLockThenCommit(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.Se.GetSessionVars().BinlogClient = s.client + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int, b int)") + tk.MustExec("begin pessimistic") + tk.MustExec("insert into t select 1, 1") + tk.MustExec("commit") + prewriteVal := getLatestBinlogPrewriteValue(c, s.pump) + c.Assert(len(prewriteVal.Mutations), Equals, 1) +} + func (s *testBinlogSuite) TestDeleteSchema(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") diff --git a/sessionctx/stmtctx/stmtctx.go b/sessionctx/stmtctx/stmtctx.go index b634330da0906..8df0001427173 100644 --- a/sessionctx/stmtctx/stmtctx.go +++ b/sessionctx/stmtctx/stmtctx.go @@ -542,19 +542,19 @@ func (sc *StatementContext) MergeExecDetails(details *execdetails.ExecDetails, c } // MergeScanDetail merges scan details into self. -func (sc *StatementContext) MergeScanDetail(scanDetail *execdetails.ScanDetail) { +func (sc *StatementContext) MergeScanDetail(scanDetail *util.ScanDetail) { // Currently TiFlash cop task does not fill scanDetail, so need to skip it if scanDetail is nil if scanDetail == nil { return } if sc.mu.execDetails.ScanDetail == nil { - sc.mu.execDetails.ScanDetail = &execdetails.ScanDetail{} + sc.mu.execDetails.ScanDetail = &util.ScanDetail{} } sc.mu.execDetails.ScanDetail.Merge(scanDetail) } // MergeTimeDetail merges time details into self. -func (sc *StatementContext) MergeTimeDetail(timeDetail execdetails.TimeDetail) { +func (sc *StatementContext) MergeTimeDetail(timeDetail util.TimeDetail) { sc.mu.execDetails.TimeDetail.ProcessTime += timeDetail.ProcessTime sc.mu.execDetails.TimeDetail.WaitTime += timeDetail.WaitTime } diff --git a/sessionctx/stmtctx/stmtctx_test.go b/sessionctx/stmtctx/stmtctx_test.go index cd0a51800a471..30e351f612955 100644 --- a/sessionctx/stmtctx/stmtctx_test.go +++ b/sessionctx/stmtctx/stmtctx_test.go @@ -20,6 +20,7 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/tidb/sessionctx/stmtctx" + "github.com/pingcap/tidb/store/tikv/util" "github.com/pingcap/tidb/util/execdetails" ) @@ -39,7 +40,7 @@ func (s *stmtctxSuit) TestCopTasksDetails(c *C) { CalleeAddress: fmt.Sprintf("%v", i+1), BackoffSleep: make(map[string]time.Duration), BackoffTimes: make(map[string]int), - TimeDetail: execdetails.TimeDetail{ + TimeDetail: util.TimeDetail{ ProcessTime: time.Second * time.Duration(i+1), WaitTime: time.Millisecond * time.Duration(i+1), }, diff --git a/sessionctx/variable/noop.go b/sessionctx/variable/noop.go index f79291665bf1e..4cc78799918f2 100644 --- a/sessionctx/variable/noop.go +++ b/sessionctx/variable/noop.go @@ -26,39 +26,39 @@ var noopSysVars = []*SysVar{ // It is unsafe to pretend that any variation of "read only" is enabled when the server // does not support it. It is possible that these features will be supported in future, // but until then... - {Scope: ScopeGlobal | ScopeSession, Name: TxReadOnly, Value: BoolOff, Type: TypeBool, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + {Scope: ScopeGlobal | ScopeSession, Name: TxReadOnly, Value: Off, Type: TypeBool, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { return checkReadOnly(vars, normalizedValue, originalValue, scope, false) }}, - {Scope: ScopeGlobal | ScopeSession, Name: TransactionReadOnly, Value: BoolOff, Type: TypeBool, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + {Scope: ScopeGlobal | ScopeSession, Name: TransactionReadOnly, Value: Off, Type: TypeBool, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { return checkReadOnly(vars, normalizedValue, originalValue, scope, false) }}, - {Scope: ScopeGlobal, Name: OfflineMode, Value: BoolOff, Type: TypeBool, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + {Scope: ScopeGlobal, Name: OfflineMode, Value: Off, Type: TypeBool, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { return checkReadOnly(vars, normalizedValue, originalValue, scope, true) }}, - {Scope: ScopeGlobal, Name: SuperReadOnly, Value: BoolOff, Type: TypeBool, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + {Scope: ScopeGlobal, Name: SuperReadOnly, Value: Off, Type: TypeBool, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { return checkReadOnly(vars, normalizedValue, originalValue, scope, false) }}, - {Scope: ScopeGlobal, Name: serverReadOnly, Value: BoolOff, Type: TypeBool, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + {Scope: ScopeGlobal, Name: serverReadOnly, Value: Off, Type: TypeBool, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { return checkReadOnly(vars, normalizedValue, originalValue, scope, false) }}, {Scope: ScopeGlobal, Name: ConnectTimeout, Value: "10", Type: TypeUnsigned, MinValue: 2, MaxValue: secondsPerYear, AutoConvertOutOfRange: true}, - {Scope: ScopeGlobal | ScopeSession, Name: QueryCacheWlockInvalidate, Value: BoolOff, Type: TypeBool}, - {Scope: ScopeGlobal | ScopeSession, Name: "sql_buffer_result", Value: BoolOff, IsHintUpdatable: true}, - {Scope: ScopeGlobal, Name: MyISAMUseMmap, Value: BoolOff, Type: TypeBool, AutoConvertNegativeBool: true}, - {Scope: ScopeGlobal, Name: "gtid_mode", Value: BoolOff, Type: TypeBool}, + {Scope: ScopeGlobal | ScopeSession, Name: QueryCacheWlockInvalidate, Value: Off, Type: TypeBool}, + {Scope: ScopeGlobal | ScopeSession, Name: "sql_buffer_result", Value: Off, IsHintUpdatable: true}, + {Scope: ScopeGlobal, Name: MyISAMUseMmap, Value: Off, Type: TypeBool, AutoConvertNegativeBool: true}, + {Scope: ScopeGlobal, Name: "gtid_mode", Value: Off, Type: TypeBool}, {Scope: ScopeGlobal, Name: FlushTime, Value: "0", Type: TypeUnsigned, MinValue: 0, MaxValue: secondsPerYear, AutoConvertOutOfRange: true}, {Scope: ScopeNone, Name: "performance_schema_max_mutex_classes", Value: "200"}, - {Scope: ScopeGlobal | ScopeSession, Name: LowPriorityUpdates, Value: BoolOff, Type: TypeBool}, - {Scope: ScopeGlobal | ScopeSession, Name: SessionTrackGtids, Value: BoolOff, Type: TypeEnum, PossibleValues: []string{BoolOff, "OWN_GTID", "ALL_GTIDS"}}, + {Scope: ScopeGlobal | ScopeSession, Name: LowPriorityUpdates, Value: Off, Type: TypeBool}, + {Scope: ScopeGlobal | ScopeSession, Name: SessionTrackGtids, Value: Off, Type: TypeEnum, PossibleValues: []string{Off, "OWN_GTID", "ALL_GTIDS"}}, {Scope: ScopeGlobal | ScopeSession, Name: "ndbinfo_max_rows", Value: ""}, {Scope: ScopeGlobal | ScopeSession, Name: "ndb_index_stat_option", Value: ""}, {Scope: ScopeGlobal | ScopeSession, Name: OldPasswords, Value: "0", Type: TypeUnsigned, MinValue: 0, MaxValue: 2, AutoConvertOutOfRange: true}, {Scope: ScopeNone, Name: "innodb_version", Value: "5.6.25"}, - {Scope: ScopeGlobal | ScopeSession, Name: BigTables, Value: BoolOff, Type: TypeBool}, + {Scope: ScopeGlobal | ScopeSession, Name: BigTables, Value: Off, Type: TypeBool}, {Scope: ScopeNone, Name: "skip_external_locking", Value: "1"}, {Scope: ScopeNone, Name: "innodb_sync_array_size", Value: "1"}, {Scope: ScopeSession, Name: "rand_seed2", Value: ""}, - {Scope: ScopeGlobal, Name: ValidatePasswordCheckUserName, Value: BoolOff, Type: TypeBool}, + {Scope: ScopeGlobal, Name: ValidatePasswordCheckUserName, Value: Off, Type: TypeBool}, {Scope: ScopeGlobal, Name: ValidatePasswordNumberCount, Value: "1", Type: TypeUnsigned, MinValue: 0, MaxValue: math.MaxUint64, AutoConvertOutOfRange: true}, {Scope: ScopeSession, Name: "gtid_next", Value: ""}, {Scope: ScopeGlobal, Name: "ndb_show_foreign_key_mock_tables", Value: ""}, @@ -77,7 +77,7 @@ var noopSysVars = []*SysVar{ {Scope: ScopeNone, Name: "lc_messages_dir", Value: "/usr/local/mysql-5.6.25-osx10.8-x86_64/share/"}, {Scope: ScopeGlobal, Name: "ft_boolean_syntax", Value: "+ -><()~*:\"\"&|"}, {Scope: ScopeGlobal, Name: TableDefinitionCache, Value: "-1", Type: TypeUnsigned, MinValue: 400, MaxValue: 524288, AutoConvertOutOfRange: true}, - {Scope: ScopeNone, Name: SkipNameResolve, Value: BoolOff, Type: TypeBool}, + {Scope: ScopeNone, Name: SkipNameResolve, Value: Off, Type: TypeBool}, {Scope: ScopeNone, Name: "performance_schema_max_file_handles", Value: "32768"}, {Scope: ScopeSession, Name: "transaction_allow_batching", Value: ""}, {Scope: ScopeNone, Name: "performance_schema_max_statement_classes", Value: "168"}, @@ -86,7 +86,7 @@ var noopSysVars = []*SysVar{ {Scope: ScopeGlobal | ScopeSession, Name: TmpTableSize, Value: "16777216", Type: TypeUnsigned, MinValue: 1024, MaxValue: math.MaxUint64, AutoConvertOutOfRange: true, IsHintUpdatable: true}, {Scope: ScopeGlobal, Name: "innodb_max_purge_lag", Value: "0"}, {Scope: ScopeGlobal | ScopeSession, Name: "preload_buffer_size", Value: "32768"}, - {Scope: ScopeGlobal, Name: CheckProxyUsers, Value: BoolOff, Type: TypeBool}, + {Scope: ScopeGlobal, Name: CheckProxyUsers, Value: Off, Type: TypeBool}, {Scope: ScopeNone, Name: "have_query_cache", Value: "YES"}, {Scope: ScopeGlobal, Name: "innodb_flush_log_at_timeout", Value: "1"}, {Scope: ScopeGlobal, Name: "innodb_max_undo_log_size", Value: ""}, @@ -103,12 +103,12 @@ var noopSysVars = []*SysVar{ {Scope: ScopeGlobal, Name: "relay_log_info_repository", Value: "FILE"}, {Scope: ScopeGlobal | ScopeSession, Name: "max_delayed_threads", Value: "20"}, {Scope: ScopeNone, Name: "protocol_version", Value: "10"}, - {Scope: ScopeGlobal | ScopeSession, Name: "new", Value: BoolOff}, + {Scope: ScopeGlobal | ScopeSession, Name: "new", Value: Off}, {Scope: ScopeGlobal | ScopeSession, Name: "myisam_sort_buffer_size", Value: "8388608"}, {Scope: ScopeGlobal | ScopeSession, Name: "optimizer_trace_offset", Value: "-1"}, {Scope: ScopeGlobal, Name: InnodbBufferPoolDumpAtShutdown, Value: "0"}, {Scope: ScopeGlobal | ScopeSession, Name: SQLNotes, Value: "1"}, - {Scope: ScopeGlobal, Name: InnodbCmpPerIndexEnabled, Value: BoolOff, Type: TypeBool, AutoConvertNegativeBool: true}, + {Scope: ScopeGlobal, Name: InnodbCmpPerIndexEnabled, Value: Off, Type: TypeBool, AutoConvertNegativeBool: true}, {Scope: ScopeGlobal, Name: "innodb_ft_server_stopword_table", Value: ""}, {Scope: ScopeNone, Name: "performance_schema_max_file_instances", Value: "7693"}, {Scope: ScopeNone, Name: "log_output", Value: "FILE"}, @@ -118,24 +118,24 @@ var noopSysVars = []*SysVar{ {Scope: ScopeGlobal, Name: "innodb_log_write_ahead_size", Value: ""}, {Scope: ScopeNone, Name: "innodb_log_group_home_dir", Value: "./"}, {Scope: ScopeNone, Name: "performance_schema_events_statements_history_size", Value: "10"}, - {Scope: ScopeGlobal, Name: GeneralLog, Value: BoolOff, Type: TypeBool}, + {Scope: ScopeGlobal, Name: GeneralLog, Value: Off, Type: TypeBool}, {Scope: ScopeGlobal, Name: "validate_password_dictionary_file", Value: ""}, - {Scope: ScopeGlobal, Name: BinlogOrderCommits, Value: BoolOn, Type: TypeBool}, + {Scope: ScopeGlobal, Name: BinlogOrderCommits, Value: On, Type: TypeBool}, {Scope: ScopeGlobal, Name: "key_cache_division_limit", Value: "100"}, {Scope: ScopeGlobal | ScopeSession, Name: "max_insert_delayed_threads", Value: "20"}, {Scope: ScopeNone, Name: "performance_schema_session_connect_attrs_size", Value: "512"}, {Scope: ScopeGlobal, Name: "innodb_max_dirty_pages_pct", Value: "75"}, - {Scope: ScopeGlobal, Name: InnodbFilePerTable, Value: BoolOn, Type: TypeBool, AutoConvertNegativeBool: true}, + {Scope: ScopeGlobal, Name: InnodbFilePerTable, Value: On, Type: TypeBool, AutoConvertNegativeBool: true}, {Scope: ScopeGlobal, Name: InnodbLogCompressedPages, Value: "1"}, {Scope: ScopeNone, Name: "skip_networking", Value: "0"}, {Scope: ScopeGlobal, Name: "innodb_monitor_reset", Value: ""}, {Scope: ScopeNone, Name: "ssl_cipher", Value: ""}, {Scope: ScopeNone, Name: "tls_version", Value: "TLSv1,TLSv1.1,TLSv1.2"}, - {Scope: ScopeGlobal, Name: InnodbPrintAllDeadlocks, Value: BoolOff, Type: TypeBool, AutoConvertNegativeBool: true}, + {Scope: ScopeGlobal, Name: InnodbPrintAllDeadlocks, Value: Off, Type: TypeBool, AutoConvertNegativeBool: true}, {Scope: ScopeNone, Name: "innodb_autoinc_lock_mode", Value: "1"}, {Scope: ScopeGlobal, Name: "key_buffer_size", Value: "8388608"}, {Scope: ScopeGlobal, Name: "host_cache_size", Value: "279"}, - {Scope: ScopeGlobal, Name: DelayKeyWrite, Value: BoolOn, Type: TypeEnum, PossibleValues: []string{BoolOff, BoolOn, "ALL"}}, + {Scope: ScopeGlobal, Name: DelayKeyWrite, Value: On, Type: TypeEnum, PossibleValues: []string{Off, On, "ALL"}}, {Scope: ScopeNone, Name: "metadata_locks_cache_size", Value: "1024"}, {Scope: ScopeNone, Name: "innodb_force_recovery", Value: "0"}, {Scope: ScopeGlobal, Name: "innodb_file_format_max", Value: "Antelope"}, @@ -147,8 +147,8 @@ var noopSysVars = []*SysVar{ {Scope: ScopeNone, Name: "innodb_mirrored_log_groups", Value: "1"}, {Scope: ScopeGlobal, Name: "max_binlog_size", Value: "1073741824"}, {Scope: ScopeGlobal, Name: "concurrent_insert", Value: "AUTO"}, - {Scope: ScopeGlobal, Name: InnodbAdaptiveHashIndex, Value: BoolOn, Type: TypeBool, AutoConvertNegativeBool: true}, - {Scope: ScopeGlobal, Name: InnodbFtEnableStopword, Value: BoolOn, Type: TypeBool, AutoConvertNegativeBool: true}, + {Scope: ScopeGlobal, Name: InnodbAdaptiveHashIndex, Value: On, Type: TypeBool, AutoConvertNegativeBool: true}, + {Scope: ScopeGlobal, Name: InnodbFtEnableStopword, Value: On, Type: TypeBool, AutoConvertNegativeBool: true}, {Scope: ScopeGlobal, Name: "general_log_file", Value: "/usr/local/mysql/data/localhost.log"}, {Scope: ScopeGlobal | ScopeSession, Name: InnodbSupportXA, Value: "1"}, {Scope: ScopeGlobal, Name: "innodb_compression_level", Value: "6"}, @@ -159,7 +159,7 @@ var noopSysVars = []*SysVar{ {Scope: ScopeNone, Name: "character_set_system", Value: "utf8"}, {Scope: ScopeGlobal, Name: InnodbOptimizeFullTextOnly, Value: "0"}, {Scope: ScopeNone, Name: "character_sets_dir", Value: "/usr/local/mysql-5.6.25-osx10.8-x86_64/share/charsets/"}, - {Scope: ScopeGlobal | ScopeSession, Name: QueryCacheType, Value: BoolOff, Type: TypeEnum, PossibleValues: []string{BoolOff, BoolOn, "DEMAND"}}, + {Scope: ScopeGlobal | ScopeSession, Name: QueryCacheType, Value: Off, Type: TypeEnum, PossibleValues: []string{Off, On, "DEMAND"}}, {Scope: ScopeNone, Name: "innodb_rollback_on_timeout", Value: "0"}, {Scope: ScopeGlobal | ScopeSession, Name: "query_alloc_block_size", Value: "8192"}, {Scope: ScopeNone, Name: "have_compress", Value: "YES"}, @@ -172,11 +172,11 @@ var noopSysVars = []*SysVar{ {Scope: ScopeNone, Name: "disconnect_on_expired_password", Value: "1"}, {Scope: ScopeNone, Name: "performance_schema_max_file_classes", Value: "50"}, {Scope: ScopeGlobal, Name: "expire_logs_days", Value: "0"}, - {Scope: ScopeGlobal | ScopeSession, Name: BinlogRowQueryLogEvents, Value: BoolOff, Type: TypeBool}, + {Scope: ScopeGlobal | ScopeSession, Name: BinlogRowQueryLogEvents, Value: Off, Type: TypeBool}, {Scope: ScopeGlobal, Name: "default_password_lifetime", Value: ""}, {Scope: ScopeNone, Name: "pid_file", Value: "/usr/local/mysql/data/localhost.pid"}, {Scope: ScopeNone, Name: "innodb_undo_tablespaces", Value: "0"}, - {Scope: ScopeGlobal, Name: InnodbStatusOutputLocks, Value: BoolOff, Type: TypeBool, AutoConvertNegativeBool: true}, + {Scope: ScopeGlobal, Name: InnodbStatusOutputLocks, Value: Off, Type: TypeBool, AutoConvertNegativeBool: true}, {Scope: ScopeNone, Name: "performance_schema_accounts_size", Value: "100"}, {Scope: ScopeGlobal | ScopeSession, Name: "max_error_count", Value: "64", IsHintUpdatable: true}, {Scope: ScopeGlobal, Name: "max_write_lock_count", Value: "18446744073709551615"}, @@ -187,16 +187,16 @@ var noopSysVars = []*SysVar{ {Scope: ScopeNone, Name: "innodb_open_files", Value: "2000"}, {Scope: ScopeGlobal, Name: "innodb_spin_wait_delay", Value: "6"}, {Scope: ScopeGlobal, Name: "thread_cache_size", Value: "9"}, - {Scope: ScopeGlobal, Name: LogSlowAdminStatements, Value: BoolOff, Type: TypeBool}, - {Scope: ScopeNone, Name: "innodb_checksums", Type: TypeBool, Value: BoolOn}, + {Scope: ScopeGlobal, Name: LogSlowAdminStatements, Value: Off, Type: TypeBool}, + {Scope: ScopeNone, Name: "innodb_checksums", Type: TypeBool, Value: On}, {Scope: ScopeNone, Name: "ft_stopword_file", Value: "(built-in)"}, {Scope: ScopeGlobal, Name: "innodb_max_dirty_pages_pct_lwm", Value: "0"}, - {Scope: ScopeGlobal, Name: LogQueriesNotUsingIndexes, Value: BoolOff, Type: TypeBool}, + {Scope: ScopeGlobal, Name: LogQueriesNotUsingIndexes, Value: Off, Type: TypeBool}, {Scope: ScopeGlobal | ScopeSession, Name: "max_heap_table_size", Value: "16777216", IsHintUpdatable: true}, {Scope: ScopeGlobal | ScopeSession, Name: "div_precision_increment", Value: "4", IsHintUpdatable: true}, {Scope: ScopeGlobal, Name: "innodb_lru_scan_depth", Value: "1024"}, {Scope: ScopeGlobal, Name: "innodb_purge_rseg_truncate_frequency", Value: ""}, - {Scope: ScopeGlobal | ScopeSession, Name: SQLAutoIsNull, Value: BoolOff, Type: TypeBool, IsHintUpdatable: true}, + {Scope: ScopeGlobal | ScopeSession, Name: SQLAutoIsNull, Value: Off, Type: TypeBool, IsHintUpdatable: true}, {Scope: ScopeNone, Name: "innodb_api_enable_binlog", Value: "0"}, {Scope: ScopeGlobal | ScopeSession, Name: "innodb_ft_user_stopword_table", Value: ""}, {Scope: ScopeNone, Name: "server_id_bits", Value: "32"}, @@ -211,14 +211,14 @@ var noopSysVars = []*SysVar{ {Scope: ScopeNone, Name: "performance_schema_max_stage_classes", Value: "150"}, {Scope: ScopeGlobal, Name: "innodb_purge_batch_size", Value: "300"}, {Scope: ScopeNone, Name: "have_profiling", Value: "NO"}, - {Scope: ScopeGlobal, Name: InnodbBufferPoolDumpNow, Value: BoolOff, Type: TypeBool, AutoConvertNegativeBool: true}, - {Scope: ScopeGlobal, Name: RelayLogPurge, Value: BoolOn, Type: TypeBool}, + {Scope: ScopeGlobal, Name: InnodbBufferPoolDumpNow, Value: Off, Type: TypeBool, AutoConvertNegativeBool: true}, + {Scope: ScopeGlobal, Name: RelayLogPurge, Value: On, Type: TypeBool}, {Scope: ScopeGlobal, Name: "ndb_distribution", Value: ""}, {Scope: ScopeGlobal, Name: "myisam_data_pointer_size", Value: "6"}, {Scope: ScopeGlobal, Name: "ndb_optimization_delay", Value: ""}, {Scope: ScopeGlobal, Name: "innodb_ft_num_word_optimize", Value: "2000"}, {Scope: ScopeGlobal | ScopeSession, Name: "max_join_size", Value: "18446744073709551615", IsHintUpdatable: true}, - {Scope: ScopeNone, Name: CoreFile, Value: BoolOff, Type: TypeBool}, + {Scope: ScopeNone, Name: CoreFile, Value: Off, Type: TypeBool}, {Scope: ScopeGlobal | ScopeSession, Name: "max_seeks_for_key", Value: "18446744073709551615", IsHintUpdatable: true}, {Scope: ScopeNone, Name: "innodb_log_buffer_size", Value: "8388608"}, {Scope: ScopeGlobal, Name: "delayed_insert_timeout", Value: "300"}, @@ -243,14 +243,14 @@ var noopSysVars = []*SysVar{ {Scope: ScopeNone, Name: "innodb_api_disable_rowlock", Value: "0"}, {Scope: ScopeGlobal, Name: "innodb_adaptive_flushing_lwm", Value: "10"}, {Scope: ScopeNone, Name: "innodb_log_files_in_group", Value: "2"}, - {Scope: ScopeGlobal, Name: InnodbBufferPoolLoadNow, Value: BoolOff, Type: TypeBool, AutoConvertNegativeBool: true}, + {Scope: ScopeGlobal, Name: InnodbBufferPoolLoadNow, Value: Off, Type: TypeBool, AutoConvertNegativeBool: true}, {Scope: ScopeNone, Name: "performance_schema_max_rwlock_classes", Value: "40"}, {Scope: ScopeNone, Name: "binlog_gtid_simple_recovery", Value: "1"}, {Scope: ScopeNone, Name: "performance_schema_digests_size", Value: "10000"}, - {Scope: ScopeGlobal | ScopeSession, Name: Profiling, Value: BoolOff, Type: TypeBool}, + {Scope: ScopeGlobal | ScopeSession, Name: Profiling, Value: Off, Type: TypeBool}, {Scope: ScopeSession, Name: "rand_seed1", Value: ""}, {Scope: ScopeGlobal, Name: "sha256_password_proxy_users", Value: ""}, - {Scope: ScopeGlobal | ScopeSession, Name: SQLQuoteShowCreate, Value: BoolOn, Type: TypeBool}, + {Scope: ScopeGlobal | ScopeSession, Name: SQLQuoteShowCreate, Value: On, Type: TypeBool}, {Scope: ScopeGlobal | ScopeSession, Name: "binlogging_impossible_mode", Value: "IGNORE_ERROR"}, {Scope: ScopeGlobal | ScopeSession, Name: QueryCacheSize, Value: "1048576"}, {Scope: ScopeGlobal, Name: "innodb_stats_transient_sample_pages", Value: "8"}, @@ -261,7 +261,7 @@ var noopSysVars = []*SysVar{ {Scope: ScopeNone, Name: "skip_show_database", Value: "0"}, {Scope: ScopeGlobal, Name: "log_timestamps", Value: ""}, {Scope: ScopeNone, Name: "version_compile_machine", Value: "x86_64"}, - {Scope: ScopeGlobal, Name: "event_scheduler", Value: BoolOff}, + {Scope: ScopeGlobal, Name: "event_scheduler", Value: Off}, {Scope: ScopeGlobal | ScopeSession, Name: "ndb_deferred_constraints", Value: ""}, {Scope: ScopeGlobal, Name: "log_syslog_include_pid", Value: ""}, {Scope: ScopeNone, Name: "innodb_ft_cache_size", Value: "8000000"}, @@ -274,9 +274,9 @@ var noopSysVars = []*SysVar{ {Scope: ScopeGlobal, Name: InnodbStatsAutoRecalc, Value: "1"}, {Scope: ScopeGlobal | ScopeSession, Name: "lc_messages", Value: "en_US"}, {Scope: ScopeGlobal | ScopeSession, Name: "bulk_insert_buffer_size", Value: "8388608", IsHintUpdatable: true}, - {Scope: ScopeGlobal | ScopeSession, Name: BinlogDirectNonTransactionalUpdates, Value: BoolOff, Type: TypeBool}, + {Scope: ScopeGlobal | ScopeSession, Name: BinlogDirectNonTransactionalUpdates, Value: Off, Type: TypeBool}, {Scope: ScopeGlobal, Name: "innodb_change_buffering", Value: "all"}, - {Scope: ScopeGlobal | ScopeSession, Name: SQLBigSelects, Value: BoolOn, Type: TypeBool, IsHintUpdatable: true}, + {Scope: ScopeGlobal | ScopeSession, Name: SQLBigSelects, Value: On, Type: TypeBool, IsHintUpdatable: true}, {Scope: ScopeGlobal, Name: "innodb_max_purge_lag_delay", Value: "0"}, {Scope: ScopeGlobal | ScopeSession, Name: "session_track_schema", Value: ""}, {Scope: ScopeGlobal, Name: "innodb_io_capacity_max", Value: "2000"}, @@ -285,26 +285,26 @@ var noopSysVars = []*SysVar{ {Scope: ScopeGlobal | ScopeSession, Name: "optimizer_trace", Value: "enabled=off,one_line=off"}, {Scope: ScopeGlobal | ScopeSession, Name: "read_rnd_buffer_size", Value: "262144", IsHintUpdatable: true}, {Scope: ScopeGlobal | ScopeSession, Name: NetWriteTimeout, Value: "60"}, - {Scope: ScopeGlobal, Name: InnodbBufferPoolLoadAbort, Value: BoolOff, Type: TypeBool, AutoConvertNegativeBool: true}, + {Scope: ScopeGlobal, Name: InnodbBufferPoolLoadAbort, Value: Off, Type: TypeBool, AutoConvertNegativeBool: true}, {Scope: ScopeGlobal | ScopeSession, Name: "transaction_prealloc_size", Value: "4096"}, {Scope: ScopeNone, Name: "performance_schema_setup_objects_size", Value: "100"}, {Scope: ScopeGlobal, Name: "sync_relay_log", Value: "10000"}, {Scope: ScopeGlobal, Name: "innodb_ft_result_cache_limit", Value: "2000000000"}, {Scope: ScopeNone, Name: "innodb_sort_buffer_size", Value: "1048576"}, - {Scope: ScopeGlobal, Name: "innodb_ft_enable_diag_print", Type: TypeBool, Value: BoolOff}, + {Scope: ScopeGlobal, Name: "innodb_ft_enable_diag_print", Type: TypeBool, Value: Off}, {Scope: ScopeNone, Name: "thread_handling", Value: "one-thread-per-connection"}, {Scope: ScopeGlobal, Name: "stored_program_cache", Value: "256"}, {Scope: ScopeNone, Name: "performance_schema_max_mutex_instances", Value: "15906"}, {Scope: ScopeGlobal, Name: "innodb_adaptive_max_sleep_delay", Value: "150000"}, - {Scope: ScopeNone, Name: "large_pages", Value: BoolOff}, + {Scope: ScopeNone, Name: "large_pages", Value: Off}, {Scope: ScopeGlobal | ScopeSession, Name: "session_track_system_variables", Value: ""}, {Scope: ScopeGlobal, Name: "innodb_change_buffer_max_size", Value: "25"}, - {Scope: ScopeGlobal, Name: LogBinTrustFunctionCreators, Value: BoolOff, Type: TypeBool}, + {Scope: ScopeGlobal, Name: LogBinTrustFunctionCreators, Value: Off, Type: TypeBool}, {Scope: ScopeNone, Name: "innodb_write_io_threads", Value: "4"}, {Scope: ScopeGlobal, Name: "mysql_native_password_proxy_users", Value: ""}, {Scope: ScopeNone, Name: "large_page_size", Value: "0"}, {Scope: ScopeNone, Name: "table_open_cache_instances", Value: "1"}, - {Scope: ScopeGlobal, Name: InnodbStatsPersistent, Value: BoolOn, Type: TypeBool, AutoConvertNegativeBool: true}, + {Scope: ScopeGlobal, Name: InnodbStatsPersistent, Value: On, Type: TypeBool, AutoConvertNegativeBool: true}, {Scope: ScopeGlobal | ScopeSession, Name: "session_track_state_change", Value: ""}, {Scope: ScopeNone, Name: OptimizerSwitch, Value: "index_merge=on,index_merge_union=on,index_merge_sort_union=on,index_merge_intersection=on,engine_condition_pushdown=on,index_condition_pushdown=on,mrr=on,mrr_cost_based=on,block_nested_loop=on,batched_key_access=off,materialization=on,semijoin=on,loosescan=on,firstmatch=on,subquery_materialization_cost_based=on,use_index_extensions=on", IsHintUpdatable: true}, {Scope: ScopeGlobal, Name: "delayed_queue_size", Value: "1000"}, @@ -312,14 +312,14 @@ 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: BoolOff}, + {Scope: ScopeGlobal, Name: "innodb_large_prefix", Type: TypeBool, Value: Off}, {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"}, {Scope: ScopeGlobal | ScopeSession, Name: "ndb_index_stat_enable", Value: ""}, {Scope: ScopeGlobal, Name: "executed_gtids_compression_period", Value: ""}, {Scope: ScopeNone, Name: "time_format", Value: "%H:%i:%s"}, - {Scope: ScopeGlobal | ScopeSession, Name: OldAlterTable, Value: BoolOff, Type: TypeBool}, + {Scope: ScopeGlobal | ScopeSession, Name: OldAlterTable, Value: Off, Type: TypeBool}, {Scope: ScopeGlobal | ScopeSession, Name: "long_query_time", Value: "10.000000"}, {Scope: ScopeNone, Name: "innodb_use_native_aio", Value: "0"}, {Scope: ScopeGlobal, Name: "log_throttle_queries_not_using_indexes", Value: "0"}, @@ -329,16 +329,16 @@ var noopSysVars = []*SysVar{ {Scope: ScopeGlobal, Name: "innodb_compression_pad_pct_max", Value: "50"}, {Scope: ScopeGlobal, Name: InnodbCommitConcurrency, Value: "0", Type: TypeUnsigned, MinValue: 0, MaxValue: 1000, AutoConvertOutOfRange: true}, {Scope: ScopeNone, Name: "ft_min_word_len", Value: "4"}, - {Scope: ScopeGlobal, Name: EnforceGtidConsistency, Value: BoolOff, Type: TypeEnum, PossibleValues: []string{BoolOff, BoolOn, "WARN"}}, - {Scope: ScopeGlobal, Name: SecureAuth, Value: BoolOn, Type: TypeBool, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + {Scope: ScopeGlobal, Name: EnforceGtidConsistency, Value: Off, Type: TypeEnum, PossibleValues: []string{Off, On, "WARN"}}, + {Scope: ScopeGlobal, Name: SecureAuth, Value: On, Type: TypeBool, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { if TiDBOptOn(normalizedValue) { - return BoolOn, nil + return On, nil } return normalizedValue, ErrWrongValueForVar.GenWithStackByArgs(SecureAuth, originalValue) }}, {Scope: ScopeNone, Name: "max_tmp_tables", Value: "32"}, - {Scope: ScopeGlobal, Name: InnodbRandomReadAhead, Value: BoolOff, Type: TypeBool, AutoConvertNegativeBool: true}, - {Scope: ScopeGlobal | ScopeSession, Name: UniqueChecks, Value: BoolOn, Type: TypeBool, IsHintUpdatable: true}, + {Scope: ScopeGlobal, Name: InnodbRandomReadAhead, Value: Off, Type: TypeBool, AutoConvertNegativeBool: true}, + {Scope: ScopeGlobal | ScopeSession, Name: UniqueChecks, Value: On, Type: TypeBool, IsHintUpdatable: true}, {Scope: ScopeGlobal, Name: "internal_tmp_disk_storage_engine", Value: ""}, {Scope: ScopeGlobal | ScopeSession, Name: "myisam_repair_threads", Value: "1"}, {Scope: ScopeGlobal, Name: "ndb_eventbuffer_max_alloc", Value: ""}, @@ -355,8 +355,8 @@ var noopSysVars = []*SysVar{ {Scope: ScopeGlobal, Name: "slow_query_log_file", Value: "/usr/local/mysql/data/localhost-slow.log"}, {Scope: ScopeGlobal, Name: "innodb_thread_sleep_delay", Value: "10000"}, {Scope: ScopeGlobal, Name: "innodb_ft_aux_table", Value: ""}, - {Scope: ScopeGlobal | ScopeSession, Name: SQLWarnings, Value: BoolOff, Type: TypeBool}, - {Scope: ScopeGlobal | ScopeSession, Name: KeepFilesOnCreate, Value: BoolOff, Type: TypeBool}, + {Scope: ScopeGlobal | ScopeSession, Name: SQLWarnings, Value: Off, Type: TypeBool}, + {Scope: ScopeGlobal | ScopeSession, Name: KeepFilesOnCreate, Value: Off, Type: TypeBool}, {Scope: ScopeNone, Name: "innodb_data_file_path", Value: "ibdata1:12M:autoextend"}, {Scope: ScopeNone, Name: "performance_schema_setup_actors_size", Value: "100"}, {Scope: ScopeNone, Name: "innodb_additional_mem_pool_size", Value: "8388608"}, @@ -379,7 +379,7 @@ var noopSysVars = []*SysVar{ // In MySQL, the default value of `explicit_defaults_for_timestamp` is `0`. // But In TiDB, it's set to `1` to be consistent with TiDB timestamp behavior. // See: https://github.com/pingcap/tidb/pull/6068 for details - {Scope: ScopeNone, Name: "explicit_defaults_for_timestamp", Value: BoolOn, Type: TypeBool}, + {Scope: ScopeNone, Name: "explicit_defaults_for_timestamp", Value: On, Type: TypeBool}, {Scope: ScopeNone, Name: "performance_schema_events_waits_history_size", Value: "10"}, {Scope: ScopeGlobal, Name: "log_syslog_tag", Value: ""}, {Scope: ScopeGlobal, Name: "innodb_undo_log_truncate", Value: ""}, @@ -399,10 +399,10 @@ var noopSysVars = []*SysVar{ {Scope: ScopeGlobal | ScopeSession, Name: "completion_type", Value: "NO_CHAIN"}, {Scope: ScopeGlobal, Name: "binlog_checksum", Value: "CRC32"}, {Scope: ScopeNone, Name: "report_port", Value: "3306"}, - {Scope: ScopeGlobal | ScopeSession, Name: ShowOldTemporals, Value: BoolOff, Type: TypeBool}, + {Scope: ScopeGlobal | ScopeSession, Name: ShowOldTemporals, Value: Off, Type: TypeBool}, {Scope: ScopeGlobal, Name: "query_cache_limit", Value: "1048576"}, {Scope: ScopeGlobal, Name: "innodb_buffer_pool_size", Value: "134217728"}, - {Scope: ScopeGlobal, Name: InnodbAdaptiveFlushing, Value: BoolOn, Type: TypeBool, AutoConvertNegativeBool: true}, + {Scope: ScopeGlobal, Name: InnodbAdaptiveFlushing, Value: On, Type: TypeBool, AutoConvertNegativeBool: true}, {Scope: ScopeGlobal, Name: "innodb_monitor_enable", Value: ""}, {Scope: ScopeNone, Name: "date_format", Value: "%Y-%m-%d"}, {Scope: ScopeGlobal, Name: "innodb_buffer_pool_filename", Value: "ib_buffer_pool"}, @@ -416,14 +416,14 @@ var noopSysVars = []*SysVar{ {Scope: ScopeNone, Name: "basedir", Value: "/usr/local/mysql"}, {Scope: ScopeGlobal, Name: "innodb_old_blocks_time", Value: "1000"}, {Scope: ScopeGlobal, Name: "innodb_stats_method", Value: "nulls_equal"}, - {Scope: ScopeGlobal, Name: LocalInFile, Value: BoolOn, Type: TypeBool}, + {Scope: ScopeGlobal, Name: LocalInFile, Value: On, Type: TypeBool}, {Scope: ScopeGlobal | ScopeSession, Name: "myisam_stats_method", Value: "nulls_unequal"}, {Scope: ScopeNone, Name: "version_compile_os", Value: "osx10.8"}, {Scope: ScopeNone, Name: "relay_log_recovery", Value: "0"}, {Scope: ScopeNone, Name: "old", Value: "0"}, - {Scope: ScopeGlobal | ScopeSession, Name: InnodbTableLocks, Value: BoolOn, Type: TypeBool, AutoConvertNegativeBool: true}, - {Scope: ScopeNone, Name: PerformanceSchema, Value: BoolOff, Type: TypeBool}, - {Scope: ScopeNone, Name: "myisam_recover_options", Value: BoolOff}, + {Scope: ScopeGlobal | ScopeSession, Name: InnodbTableLocks, Value: On, Type: TypeBool, AutoConvertNegativeBool: true}, + {Scope: ScopeNone, Name: PerformanceSchema, Value: Off, Type: TypeBool}, + {Scope: ScopeNone, Name: "myisam_recover_options", Value: Off}, {Scope: ScopeGlobal | ScopeSession, Name: NetBufferLength, Value: "16384"}, {Scope: ScopeGlobal | ScopeSession, Name: "binlog_row_image", Value: "FULL"}, {Scope: ScopeNone, Name: "innodb_locks_unsafe_for_binlog", Value: "0"}, @@ -436,7 +436,7 @@ var noopSysVars = []*SysVar{ {Scope: ScopeGlobal, Name: "innodb_undo_logs", Value: "128"}, {Scope: ScopeNone, Name: "performance_schema_max_cond_instances", Value: "3504"}, {Scope: ScopeGlobal, Name: "delayed_insert_limit", Value: "100"}, - {Scope: ScopeGlobal, Name: Flush, Value: BoolOff, Type: TypeBool}, + {Scope: ScopeGlobal, Name: Flush, Value: Off, Type: TypeBool}, {Scope: ScopeGlobal | ScopeSession, Name: "eq_range_index_dive_limit", Value: "200", IsHintUpdatable: true}, {Scope: ScopeNone, Name: "performance_schema_events_stages_history_size", Value: "10"}, {Scope: ScopeGlobal | ScopeSession, Name: "ndb_join_pushdown", Value: ""}, @@ -455,19 +455,19 @@ var noopSysVars = []*SysVar{ {Scope: ScopeNone, Name: "innodb_undo_directory", Value: "."}, {Scope: ScopeNone, Name: "bind_address", Value: "*"}, {Scope: ScopeGlobal, Name: "innodb_sync_spin_loops", Value: "30"}, - {Scope: ScopeGlobal | ScopeSession, Name: SQLSafeUpdates, Value: BoolOff, Type: TypeBool, IsHintUpdatable: true}, + {Scope: ScopeGlobal | ScopeSession, Name: SQLSafeUpdates, Value: Off, Type: TypeBool, IsHintUpdatable: true}, {Scope: ScopeNone, Name: "tmpdir", Value: "/var/tmp/"}, {Scope: ScopeGlobal, Name: "innodb_thread_concurrency", Value: "0"}, {Scope: ScopeGlobal, Name: "innodb_buffer_pool_dump_pct", Value: ""}, {Scope: ScopeGlobal | ScopeSession, Name: "lc_time_names", Value: "en_US"}, {Scope: ScopeGlobal | ScopeSession, Name: "max_statement_time", Value: ""}, - {Scope: ScopeGlobal | ScopeSession, Name: EndMarkersInJSON, Value: BoolOff, Type: TypeBool, IsHintUpdatable: true}, - {Scope: ScopeGlobal, Name: AvoidTemporalUpgrade, Value: BoolOff, Type: TypeBool}, + {Scope: ScopeGlobal | ScopeSession, Name: EndMarkersInJSON, Value: Off, Type: TypeBool, IsHintUpdatable: true}, + {Scope: ScopeGlobal, Name: AvoidTemporalUpgrade, Value: Off, Type: TypeBool}, {Scope: ScopeGlobal, Name: "key_cache_age_threshold", Value: "300"}, - {Scope: ScopeGlobal, Name: InnodbStatusOutput, Value: BoolOff, Type: TypeBool, AutoConvertNegativeBool: true}, + {Scope: ScopeGlobal, Name: InnodbStatusOutput, Value: Off, Type: TypeBool, AutoConvertNegativeBool: true}, {Scope: ScopeSession, Name: "identity", Value: ""}, {Scope: ScopeGlobal | ScopeSession, Name: "min_examined_row_limit", Value: "0"}, - {Scope: ScopeGlobal, Name: "sync_frm", Type: TypeBool, Value: BoolOn}, + {Scope: ScopeGlobal, Name: "sync_frm", Type: TypeBool, Value: On}, {Scope: ScopeGlobal, Name: "innodb_online_alter_log_max_size", Value: "134217728"}, {Scope: ScopeGlobal | ScopeSession, Name: "information_schema_stats_expiry", Value: "86400"}, {Scope: ScopeGlobal, Name: ThreadPoolSize, Value: "16", Type: TypeUnsigned, MinValue: 1, MaxValue: 64, AutoConvertOutOfRange: true}, @@ -479,33 +479,33 @@ var noopSysVars = []*SysVar{ {Scope: ScopeGlobal, Name: "slave_pending_jobs_size_max", Value: "16777216"}, {Scope: ScopeGlobal, Name: "slave_transaction_retries", Value: "10"}, {Scope: ScopeGlobal, Name: "slave_checkpoint_period", Value: "300"}, - {Scope: ScopeGlobal, Name: MasterVerifyChecksum, Value: BoolOff, Type: TypeBool}, + {Scope: ScopeGlobal, Name: MasterVerifyChecksum, Value: Off, Type: TypeBool}, {Scope: ScopeGlobal, Name: "rpl_semi_sync_master_trace_level", Value: ""}, {Scope: ScopeGlobal, Name: "master_info_repository", Value: "FILE"}, {Scope: ScopeGlobal, Name: "rpl_stop_slave_timeout", Value: "31536000"}, {Scope: ScopeGlobal, Name: "slave_net_timeout", Value: "3600"}, {Scope: ScopeGlobal, Name: "sync_master_info", Value: "10000"}, {Scope: ScopeGlobal, Name: "init_slave", Value: ""}, - {Scope: ScopeGlobal, Name: SlaveCompressedProtocol, Value: BoolOff, Type: TypeBool}, + {Scope: ScopeGlobal, Name: SlaveCompressedProtocol, Value: Off, Type: TypeBool}, {Scope: ScopeGlobal, Name: "rpl_semi_sync_slave_trace_level", Value: ""}, - {Scope: ScopeGlobal, Name: LogSlowSlaveStatements, Value: BoolOff, Type: TypeBool}, + {Scope: ScopeGlobal, Name: LogSlowSlaveStatements, Value: Off, Type: TypeBool}, {Scope: ScopeGlobal, Name: "slave_checkpoint_group", Value: "512"}, {Scope: ScopeNone, Name: "slave_load_tmpdir", Value: "/var/tmp/"}, {Scope: ScopeGlobal, Name: "slave_parallel_type", Value: ""}, {Scope: ScopeGlobal, Name: "slave_parallel_workers", Value: "0"}, {Scope: ScopeGlobal, Name: "rpl_semi_sync_master_timeout", Value: "10000", Type: TypeInt}, - {Scope: ScopeNone, Name: "slave_skip_errors", Value: BoolOff}, + {Scope: ScopeNone, Name: "slave_skip_errors", Value: Off}, {Scope: ScopeGlobal, Name: "sql_slave_skip_counter", Value: "0"}, - {Scope: ScopeGlobal, Name: "rpl_semi_sync_slave_enabled", Value: BoolOff, Type: TypeBool}, - {Scope: ScopeGlobal, Name: "rpl_semi_sync_master_enabled", Value: BoolOff, Type: TypeBool}, - {Scope: ScopeGlobal, Name: "slave_preserve_commit_order", Value: BoolOff, Type: TypeBool}, + {Scope: ScopeGlobal, Name: "rpl_semi_sync_slave_enabled", Value: Off, Type: TypeBool}, + {Scope: ScopeGlobal, Name: "rpl_semi_sync_master_enabled", Value: Off, Type: TypeBool}, + {Scope: ScopeGlobal, Name: "slave_preserve_commit_order", Value: Off, Type: TypeBool}, {Scope: ScopeGlobal, Name: "slave_exec_mode", Value: "STRICT"}, - {Scope: ScopeNone, Name: "log_slave_updates", Value: BoolOff, Type: TypeBool}, + {Scope: ScopeNone, Name: "log_slave_updates", Value: Off, Type: TypeBool}, {Scope: ScopeGlobal, Name: "rpl_semi_sync_master_wait_point", Value: "AFTER_SYNC", Type: TypeEnum, PossibleValues: []string{"AFTER_SYNC", "AFTER_COMMIT"}}, - {Scope: ScopeGlobal, Name: "slave_sql_verify_checksum", Value: BoolOn, Type: TypeBool}, + {Scope: ScopeGlobal, Name: "slave_sql_verify_checksum", Value: On, Type: TypeBool}, {Scope: ScopeGlobal, Name: "slave_max_allowed_packet", Value: "1073741824"}, {Scope: ScopeGlobal, Name: "rpl_semi_sync_master_wait_for_slave_count", Value: "1", Type: TypeInt, MinValue: 1, MaxValue: 65535}, - {Scope: ScopeGlobal, Name: "rpl_semi_sync_master_wait_no_slave", Value: BoolOn, Type: TypeBool}, + {Scope: ScopeGlobal, Name: "rpl_semi_sync_master_wait_no_slave", Value: On, Type: TypeBool}, {Scope: ScopeGlobal, Name: "slave_rows_search_algorithms", Value: "TABLE_SCAN,INDEX_SCAN"}, - {Scope: ScopeGlobal, Name: SlaveAllowBatching, Value: BoolOff, Type: TypeBool}, + {Scope: ScopeGlobal, Name: SlaveAllowBatching, Value: Off, Type: TypeBool}, } diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index d4d83d456f0b3..b47199f1c0916 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -32,7 +32,6 @@ import ( "github.com/pingcap/parser" "github.com/pingcap/parser/ast" "github.com/pingcap/parser/auth" - "github.com/pingcap/parser/charset" "github.com/pingcap/parser/mysql" "github.com/pingcap/parser/terror" pumpcli "github.com/pingcap/tidb-tools/tidb-binlog/pump_client" @@ -47,7 +46,6 @@ import ( "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/execdetails" - "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/rowcodec" "github.com/pingcap/tidb/util/stringutil" "github.com/pingcap/tidb/util/timeutil" @@ -403,7 +401,7 @@ type SessionVars struct { TxnCtx *TransactionContext // KVVars is the variables for KV storage. - KVVars *kv.Variables + KVVars *tikvstore.Variables // txnIsolationLevelOneShot is used to implements "set transaction isolation level ..." txnIsolationLevelOneShot struct { @@ -826,6 +824,10 @@ type SessionVars struct { // EnableDynamicPrivileges indicates whether to permit experimental support for MySQL 8.0 compatible dynamic privileges. EnableDynamicPrivileges bool + + // CTEMaxRecursionDepth indicates The common table expression (CTE) maximum recursion depth. + // see https://dev.mysql.com/doc/refman/8.0/en/server-system-variables.html#sysvar_cte_max_recursion_depth + CTEMaxRecursionDepth int } // AllocMPPTaskID allocates task id for mpp tasks. It will reset the task id if the query's @@ -1020,7 +1022,7 @@ func NewSessionVars() *SessionVars { EnableIndexMergeJoin: DefTiDBEnableIndexMergeJoin, AllowFallbackToTiKV: make(map[kv.StoreType]struct{}), } - vars.KVVars = kv.NewVariables(&vars.Killed) + vars.KVVars = tikvstore.NewVariables(&vars.Killed) vars.Concurrency = Concurrency{ indexLookupConcurrency: DefIndexLookupConcurrency, indexSerialScanConcurrency: DefIndexSerialScanConcurrency, @@ -1368,62 +1370,13 @@ func (s *SessionVars) ClearStmtVars() { } // SetSystemVar sets the value of a system variable for session scope. -// Validation has already been performed, and the values have been normalized. +// Validation is expected to be performed before calling this function, +// and the values should been normalized. // i.e. oN / on / 1 => ON func (s *SessionVars) SetSystemVar(name string, val string) error { - switch name { - case CollationConnection, CollationDatabase, CollationServer: - coll, err := collate.GetCollationByName(val) - if err != nil { - logutil.BgLogger().Warn(err.Error()) - coll, err = collate.GetCollationByName(charset.CollationUTF8MB4) - if err != nil { - return err - } - } - switch name { - case CollationConnection: - s.systems[CollationConnection] = coll.Name - s.systems[CharacterSetConnection] = coll.CharsetName - case CollationDatabase: - s.systems[CollationDatabase] = coll.Name - s.systems[CharsetDatabase] = coll.CharsetName - case CollationServer: - s.systems[CollationServer] = coll.Name - s.systems[CharacterSetServer] = coll.CharsetName - } - val = coll.Name - case CharacterSetConnection, CharacterSetClient, CharacterSetResults, - CharacterSetServer, CharsetDatabase, CharacterSetFilesystem: - if val == "" { - if name == CharacterSetResults { - s.systems[CharacterSetResults] = "" - return nil - } - return ErrWrongValueForVar.GenWithStackByArgs(name, "NULL") - } - cht, coll, err := charset.GetCharsetInfo(val) - if err != nil { - logutil.BgLogger().Warn(err.Error()) - cht, coll = charset.GetDefaultCharsetAndCollate() - } - switch name { - case CharacterSetConnection: - s.systems[CollationConnection] = coll - s.systems[CharacterSetConnection] = cht - case CharsetDatabase: - s.systems[CollationDatabase] = coll - s.systems[CharsetDatabase] = cht - case CharacterSetServer: - s.systems[CollationServer] = coll - s.systems[CharacterSetServer] = cht - } - val = cht - default: - sv := GetSysVar(name) - if err := sv.SetSessionFromHook(s, val); err != nil { - return err - } + sv := GetSysVar(name) + if err := sv.SetSessionFromHook(s, val); err != nil { + return err } s.systems[name] = val return nil diff --git a/sessionctx/variable/session_test.go b/sessionctx/variable/session_test.go index 6ed452f7c0b5f..c51efdd49c9ba 100644 --- a/sessionctx/variable/session_test.go +++ b/sessionctx/variable/session_test.go @@ -23,6 +23,7 @@ import ( "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/store/tikv/util" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/execdetails" "github.com/pingcap/tidb/util/mock" @@ -152,11 +153,11 @@ func (*testSessionSuite) TestSlowLogFormat(c *C) { execDetail := execdetails.ExecDetails{ BackoffTime: time.Millisecond, RequestCount: 2, - ScanDetail: &execdetails.ScanDetail{ + ScanDetail: &util.ScanDetail{ ProcessedKeys: 20001, TotalKeys: 10000, }, - TimeDetail: execdetails.TimeDetail{ + TimeDetail: util.TimeDetail{ ProcessTime: time.Second * time.Duration(2), WaitTime: time.Minute, }, diff --git a/sessionctx/variable/statusvar.go b/sessionctx/variable/statusvar.go index 0d3a4eee1aabf..ce7d17de09705 100644 --- a/sessionctx/variable/statusvar.go +++ b/sessionctx/variable/statusvar.go @@ -103,7 +103,6 @@ var tlsSupportedCiphers string // Taken from https://github.com/openssl/openssl/blob/c784a838e0947fcca761ee62def7d077dc06d37f/include/openssl/ssl.h#L141 . var tlsVersionString = map[uint16]string{ - tls.VersionSSL30: "SSLv3", tls.VersionTLS10: "TLSv1", tls.VersionTLS11: "TLSv1.1", tls.VersionTLS12: "TLSv1.2", @@ -137,7 +136,11 @@ func (s defaultStatusStat) Stats(vars *SessionVars) (map[string]interface{}, err statusVars["Ssl_cipher_list"] = tlsSupportedCiphers // tls.VerifyClientCertIfGiven == SSL_VERIFY_PEER | SSL_VERIFY_CLIENT_ONCE statusVars["Ssl_verify_mode"] = 0x01 | 0x04 - statusVars["Ssl_version"] = tlsVersionString[vars.TLSConnectionState.Version] + if tlsVersion, tlsVersionKnown := tlsVersionString[vars.TLSConnectionState.Version]; tlsVersionKnown { + statusVars["Ssl_version"] = tlsVersion + } else { + statusVars["Ssl_version"] = "unknown_tls_version" + } } return statusVars, nil diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index ec693444018eb..5a7ca2902c63a 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -24,6 +24,7 @@ import ( "github.com/cznic/mathutil" "github.com/pingcap/errors" + "github.com/pingcap/parser/charset" "github.com/pingcap/parser/mysql" pmysql "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/config" @@ -69,19 +70,12 @@ const ( // TypeDuration for a golang duration (a TiDB extension) TypeDuration TypeFlag = 7 - // BoolOff is the canonical string representation of a boolean false. - BoolOff = "OFF" - // BoolOn is the canonical string representation of a boolean true. - BoolOn = "ON" // On is the canonical string for ON On = "ON" - // Off is the canonical string for OFF Off = "OFF" - // Warn means return warnings Warn = "WARN" - // IntOnly means enable for int type IntOnly = "INT_ONLY" ) @@ -308,9 +302,9 @@ func (sv *SysVar) checkFloatSystemVar(value string, vars *SessionVars) (string, func (sv *SysVar) checkBoolSystemVar(value string, vars *SessionVars) (string, error) { if strings.EqualFold(value, "ON") { - return BoolOn, nil + return On, nil } else if strings.EqualFold(value, "OFF") { - return BoolOff, nil + return Off, nil } val, err := strconv.ParseInt(value, 10, 64) if err == nil { @@ -319,15 +313,15 @@ func (sv *SysVar) checkBoolSystemVar(value string, vars *SessionVars) (string, e // negative integer to 1. if !sv.AutoConvertNegativeBool { if val == 0 { - return BoolOff, nil + return Off, nil } else if val == 1 { - return BoolOn, nil + return On, nil } } else { if val == 1 || val < 0 { - return BoolOn, nil + return On, nil } else if val == 0 { - return BoolOff, nil + return Off, nil } } } @@ -487,12 +481,14 @@ var defaultSysVars = []*SysVar{ return nil }}, {Scope: ScopeGlobal | ScopeSession, Name: CollationServer, Value: mysql.DefaultCollationName, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { - if _, err := collate.GetCollationByName(normalizedValue); err != nil { - return normalizedValue, errors.Trace(err) + return checkCollation(vars, normalizedValue, originalValue, scope) + }, SetSession: func(s *SessionVars, val string) error { + if coll, err := collate.GetCollationByName(val); err == nil { + s.systems[CharacterSetServer] = coll.CharsetName } - return normalizedValue, nil + return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: SQLLogBin, Value: BoolOn, Type: TypeBool}, + {Scope: ScopeGlobal | ScopeSession, Name: SQLLogBin, Value: On, Type: TypeBool}, {Scope: ScopeGlobal | ScopeSession, Name: TimeZone, Value: "SYSTEM", IsHintUpdatable: true, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { if strings.EqualFold(normalizedValue, "SYSTEM") { return "SYSTEM", nil @@ -508,27 +504,29 @@ var defaultSysVars = []*SysVar{ return nil }}, {Scope: ScopeNone, Name: SystemTimeZone, Value: "CST"}, - {Scope: ScopeGlobal | ScopeSession, Name: ForeignKeyChecks, Value: BoolOff, Type: TypeBool, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + {Scope: ScopeGlobal | ScopeSession, Name: ForeignKeyChecks, Value: Off, Type: TypeBool, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { if TiDBOptOn(normalizedValue) { // TiDB does not yet support foreign keys. // Return the original value in the warning, so that users are not confused. vars.StmtCtx.AppendWarning(ErrUnsupportedValueForVar.GenWithStackByArgs(ForeignKeyChecks, originalValue)) - return BoolOff, nil + return Off, nil } else if !TiDBOptOn(normalizedValue) { - return BoolOff, nil + return Off, nil } return normalizedValue, ErrWrongValueForVar.GenWithStackByArgs(ForeignKeyChecks, originalValue) }}, {Scope: ScopeNone, Name: Hostname, Value: ServerHostname}, {Scope: ScopeSession, Name: Timestamp, Value: ""}, {Scope: ScopeGlobal | ScopeSession, Name: CharacterSetFilesystem, Value: "binary", Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { - return checkCharacterValid(normalizedValue, CharacterSetFilesystem) + return checkCharacterSet(normalizedValue, CharacterSetFilesystem) }}, {Scope: ScopeGlobal | ScopeSession, Name: CollationDatabase, Value: mysql.DefaultCollationName, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { - if _, err := collate.GetCollationByName(normalizedValue); err != nil { - return normalizedValue, errors.Trace(err) + return checkCollation(vars, normalizedValue, originalValue, scope) + }, SetSession: func(s *SessionVars, val string) error { + if coll, err := collate.GetCollationByName(val); err == nil { + s.systems[CharsetDatabase] = coll.CharsetName } - return normalizedValue, nil + return nil }}, {Scope: ScopeGlobal | ScopeSession, Name: AutoIncrementIncrement, Value: strconv.FormatInt(DefAutoIncrementIncrement, 10), Type: TypeUnsigned, MinValue: 1, MaxValue: math.MaxUint16, AutoConvertOutOfRange: true, SetSession: func(s *SessionVars, val string) error { // AutoIncrementIncrement is valid in [1, 65535]. @@ -541,16 +539,16 @@ var defaultSysVars = []*SysVar{ return nil }}, {Scope: ScopeGlobal | ScopeSession, Name: CharacterSetClient, Value: mysql.DefaultCharset, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { - return checkCharacterValid(normalizedValue, CharacterSetClient) + return checkCharacterSet(normalizedValue, CharacterSetClient) }}, {Scope: ScopeNone, Name: Port, Value: "4000", Type: TypeUnsigned, MinValue: 0, MaxValue: math.MaxUint16}, {Scope: ScopeNone, Name: LowerCaseTableNames, Value: "2"}, - {Scope: ScopeNone, Name: LogBin, Value: BoolOff, Type: TypeBool}, + {Scope: ScopeNone, Name: LogBin, Value: Off, Type: TypeBool}, {Scope: ScopeGlobal | ScopeSession, Name: CharacterSetResults, Value: mysql.DefaultCharset, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { if normalizedValue == "" { return normalizedValue, nil } - return checkCharacterValid(normalizedValue, "") + return checkCharacterSet(normalizedValue, "") }}, {Scope: ScopeNone, Name: VersionComment, Value: "TiDB Server (Apache License 2.0) " + versioninfo.TiDBEdition + " Edition, MySQL 5.7 compatible"}, {Scope: ScopeGlobal | ScopeSession, Name: TxnIsolation, Value: "REPEATABLE-READ", Type: TypeEnum, PossibleValues: []string{"READ-UNCOMMITTED", "READ-COMMITTED", "REPEATABLE-READ", "SERIALIZABLE"}, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { @@ -576,13 +574,15 @@ var defaultSysVars = []*SysVar{ return normalizedValue, nil }}, {Scope: ScopeGlobal | ScopeSession, Name: CollationConnection, Value: mysql.DefaultCollationName, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { - if _, err := collate.GetCollationByName(normalizedValue); err != nil { - return normalizedValue, errors.Trace(err) + return checkCollation(vars, normalizedValue, originalValue, scope) + }, SetSession: func(s *SessionVars, val string) error { + if coll, err := collate.GetCollationByName(val); err == nil { + s.systems[CharacterSetConnection] = coll.CharsetName } - return normalizedValue, nil + return nil }}, {Scope: ScopeNone, Name: Version, Value: mysql.ServerVersion}, - {Scope: ScopeGlobal | ScopeSession, Name: AutoCommit, Value: BoolOn, Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeGlobal | ScopeSession, Name: AutoCommit, Value: On, Type: TypeBool, SetSession: func(s *SessionVars, val string) error { isAutocommit := TiDBOptOn(val) s.SetStatusFlag(mysql.ServerStatusAutocommit, isAutocommit) if isAutocommit { @@ -591,7 +591,12 @@ var defaultSysVars = []*SysVar{ return nil }}, {Scope: ScopeGlobal | ScopeSession, Name: CharsetDatabase, Value: mysql.DefaultCharset, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { - return checkCharacterValid(normalizedValue, CharsetDatabase) + return checkCharacterSet(normalizedValue, CharsetDatabase) + }, SetSession: func(s *SessionVars, val string) error { + if _, coll, err := charset.GetCharsetInfo(val); err == nil { + s.systems[CollationDatabase] = coll + } + return nil }}, {Scope: ScopeGlobal, Name: MaxPreparedStmtCount, Value: strconv.FormatInt(DefMaxPreparedStmtCount, 10), Type: TypeInt, MinValue: -1, MaxValue: 1048576, AutoConvertOutOfRange: true}, {Scope: ScopeNone, Name: DataDir, Value: "/usr/local/mysql/data/"}, @@ -619,15 +624,25 @@ var defaultSysVars = []*SysVar{ }}, {Scope: ScopeNone, Name: Socket, Value: "/tmp/myssock"}, {Scope: ScopeGlobal | ScopeSession, Name: CharacterSetConnection, Value: mysql.DefaultCharset, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { - return checkCharacterValid(normalizedValue, CharacterSetConnection) + return checkCharacterSet(normalizedValue, CharacterSetConnection) + }, SetSession: func(s *SessionVars, val string) error { + if _, coll, err := charset.GetCharsetInfo(val); err == nil { + s.systems[CollationConnection] = coll + } + return nil }}, {Scope: ScopeGlobal | ScopeSession, Name: CharacterSetServer, Value: mysql.DefaultCharset, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { - return checkCharacterValid(normalizedValue, CharacterSetServer) + return checkCharacterSet(normalizedValue, CharacterSetServer) + }, SetSession: func(s *SessionVars, val string) error { + if _, coll, err := charset.GetCharsetInfo(val); err == nil { + s.systems[CollationServer] = coll + } + return nil }}, {Scope: ScopeGlobal | ScopeSession, Name: MaxAllowedPacket, Value: "67108864", Type: TypeUnsigned, MinValue: 1024, MaxValue: MaxOfMaxAllowedPacket, AutoConvertOutOfRange: true}, {Scope: ScopeSession, Name: WarningCount, Value: "0", ReadOnly: true}, {Scope: ScopeSession, Name: ErrorCount, Value: "0", ReadOnly: true}, - {Scope: ScopeGlobal | ScopeSession, Name: WindowingUseHighPrecision, Value: BoolOn, Type: TypeBool, IsHintUpdatable: true, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeGlobal | ScopeSession, Name: WindowingUseHighPrecision, Value: On, Type: TypeBool, IsHintUpdatable: true, SetSession: func(s *SessionVars, val string) error { s.WindowingUseHighPrecision = TiDBOptOn(val) return nil }}, @@ -831,11 +846,11 @@ var defaultSysVars = []*SysVar{ s.InitChunkSize = tidbOptPositiveInt32(val, DefInitChunkSize) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableCascadesPlanner, Value: BoolOff, Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableCascadesPlanner, Value: Off, Type: TypeBool, SetSession: func(s *SessionVars, val string) error { s.SetEnableCascadesPlanner(TiDBOptOn(val)) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableIndexMerge, Value: BoolOff, Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableIndexMerge, Value: Off, Type: TypeBool, SetSession: func(s *SessionVars, val string) error { s.SetEnableIndexMerge(TiDBOptOn(val)) return nil }}, @@ -867,11 +882,11 @@ var defaultSysVars = []*SysVar{ s.MemQuotaIndexLookupJoin = tidbOptInt64(val, DefTiDBMemQuotaIndexLookupJoin) return nil }}, - {Scope: ScopeSession, Name: TiDBEnableStreaming, Value: BoolOff, Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeSession, Name: TiDBEnableStreaming, Value: Off, Type: TypeBool, SetSession: func(s *SessionVars, val string) error { s.EnableStreaming = TiDBOptOn(val) return nil }}, - {Scope: ScopeSession, Name: TiDBEnableChunkRPC, Value: BoolOn, Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeSession, Name: TiDBEnableChunkRPC, Value: On, Type: TypeBool, SetSession: func(s *SessionVars, val string) error { s.EnableChunkRPC = TiDBOptOn(val) return nil }}, @@ -889,11 +904,11 @@ var defaultSysVars = []*SysVar{ s.txnIsolationLevelOneShot.value = val return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableTablePartition, Value: BoolOn, Type: TypeEnum, PossibleValues: []string{BoolOff, BoolOn, "AUTO"}, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableTablePartition, Value: On, Type: TypeEnum, PossibleValues: []string{Off, On, "AUTO"}, SetSession: func(s *SessionVars, val string) error { s.EnableTablePartition = val return nil }}, - {Scope: ScopeSession, Name: TiDBEnableListTablePartition, Value: BoolOff, Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeSession, Name: TiDBEnableListTablePartition, Value: Off, Type: TypeBool, SetSession: func(s *SessionVars, val string) error { s.EnableListTablePartition = TiDBOptOn(val) return nil }}, @@ -933,12 +948,12 @@ var defaultSysVars = []*SysVar{ s.MemQuotaApplyCache = tidbOptInt64(val, DefTiDBMemQuotaApplyCache) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBBackoffLockFast, Value: strconv.Itoa(kv.DefBackoffLockFast), Type: TypeUnsigned, MinValue: 1, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error { - s.KVVars.BackoffLockFast = tidbOptPositiveInt32(val, kv.DefBackoffLockFast) + {Scope: ScopeGlobal | ScopeSession, Name: TiDBBackoffLockFast, Value: strconv.Itoa(tikvstore.DefBackoffLockFast), Type: TypeUnsigned, MinValue: 1, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error { + s.KVVars.BackoffLockFast = tidbOptPositiveInt32(val, tikvstore.DefBackoffLockFast) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBBackOffWeight, Value: strconv.Itoa(kv.DefBackOffWeight), Type: TypeUnsigned, MinValue: 1, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error { - s.KVVars.BackOffWeight = tidbOptPositiveInt32(val, kv.DefBackOffWeight) + {Scope: ScopeGlobal | ScopeSession, Name: TiDBBackOffWeight, Value: strconv.Itoa(tikvstore.DefBackOffWeight), Type: TypeUnsigned, MinValue: 1, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error { + s.KVVars.BackOffWeight = tidbOptPositiveInt32(val, tikvstore.DefBackOffWeight) return nil }}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBRetryLimit, Value: strconv.Itoa(DefTiDBRetryLimit), Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { @@ -1025,7 +1040,7 @@ var defaultSysVars = []*SysVar{ return nil }}, /* The following variable is defined as session scope but is actually server scope. */ - {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableDynamicPrivileges, Value: BoolOff, Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableDynamicPrivileges, Value: Off, Type: TypeBool, SetSession: func(s *SessionVars, val string) error { s.EnableDynamicPrivileges = TiDBOptOn(val) return nil }}, @@ -1095,7 +1110,7 @@ var defaultSysVars = []*SysVar{ s.WaitSplitRegionTimeout = uint64(tidbOptPositiveInt32(val, DefWaitSplitRegionTimeout)) return nil }}, - {Scope: ScopeSession, Name: TiDBLowResolutionTSO, Value: BoolOff, Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeSession, Name: TiDBLowResolutionTSO, Value: Off, Type: TypeBool, SetSession: func(s *SessionVars, val string) error { s.LowResolutionTSO = TiDBOptOn(val) return nil }}, @@ -1113,7 +1128,7 @@ var defaultSysVars = []*SysVar{ // TxReadOnly, TransactionReadOnly, OfflineMode, SuperReadOnly, serverReadOnly // To prevent this strange position, prevent setting to OFF when any of these sysVars are ON of the same scope. - if normalizedValue == BoolOff { + if normalizedValue == Off { for _, potentialIncompatibleSysVar := range []string{TxReadOnly, TransactionReadOnly, OfflineMode, SuperReadOnly, serverReadOnly} { val, _ := vars.GetSystemVar(potentialIncompatibleSysVar) // session scope if scope == ScopeGlobal { // global scope @@ -1153,7 +1168,7 @@ var defaultSysVars = []*SysVar{ {Scope: ScopeGlobal | ScopeSession, Name: TiDBStmtSummaryHistorySize, Value: strconv.Itoa(config.GetGlobalConfig().StmtSummary.HistorySize), Type: TypeInt, MinValue: 0, MaxValue: uint64(math.MaxUint8), AllowEmpty: true}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBStmtSummaryMaxStmtCount, Value: strconv.FormatUint(uint64(config.GetGlobalConfig().StmtSummary.MaxStmtCount), 10), Type: TypeInt, MinValue: 1, MaxValue: uint64(math.MaxInt16), AllowEmpty: true}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBStmtSummaryMaxSQLLength, Value: strconv.FormatUint(uint64(config.GetGlobalConfig().StmtSummary.MaxSQLLength), 10), Type: TypeInt, MinValue: 0, MaxValue: uint64(math.MaxInt32), AllowEmpty: true}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBCapturePlanBaseline, Value: BoolOff, Type: TypeBool, AllowEmptyAll: true}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBCapturePlanBaseline, Value: Off, Type: TypeBool, AllowEmptyAll: true}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBUsePlanBaselines, Value: BoolToOnOff(DefTiDBUsePlanBaselines), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { s.UsePlanBaselines = TiDBOptOn(val) return nil @@ -1231,6 +1246,10 @@ var defaultSysVars = []*SysVar{ atomic.StoreUint64(&config.GetGlobalConfig().Log.QueryLogMaxLen, uint64(tidbOptInt64(val, logutil.DefaultQueryLogMaxLen))) return nil }}, + {Scope: ScopeGlobal | ScopeSession, Name: CTEMaxRecursionDepth, Value: strconv.Itoa(DefCTEMaxRecursionDepth), Type: TypeInt, MinValue: 0, MaxValue: 4294967295, AutoConvertOutOfRange: true, SetSession: func(s *SessionVars, val string) error { + s.CTEMaxRecursionDepth = tidbOptInt(val, DefCTEMaxRecursionDepth) + return nil + }}, {Scope: ScopeSession, Name: TiDBCheckMb4ValueInUTF8, Value: BoolToOnOff(config.GetGlobalConfig().CheckMb4ValueInUTF8), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { config.GetGlobalConfig().CheckMb4ValueInUTF8 = TiDBOptOn(val) return nil @@ -1342,10 +1361,10 @@ var defaultSysVars = []*SysVar{ s.TiDBEnableExchangePartition = TiDBOptOn(val) return nil }}, - {Scope: ScopeNone, Name: TiDBEnableEnhancedSecurity, Value: BoolOff, Type: TypeBool}, + {Scope: ScopeNone, Name: TiDBEnableEnhancedSecurity, Value: Off, Type: TypeBool}, /* tikv gc metrics */ - {Scope: ScopeGlobal, Name: TiDBGCEnable, Value: BoolOn, Type: TypeBool}, + {Scope: ScopeGlobal, Name: TiDBGCEnable, Value: On, Type: TypeBool}, {Scope: ScopeGlobal, Name: TiDBGCRunInterval, Value: "10m0s", Type: TypeDuration, MinValue: int64(time.Minute * 10), MaxValue: math.MaxInt64}, {Scope: ScopeGlobal, Name: TiDBGCLifetime, Value: "10m0s", Type: TypeDuration, MinValue: int64(time.Minute * 10), MaxValue: math.MaxInt64}, {Scope: ScopeGlobal, Name: TiDBGCConcurrency, Value: "-1", Type: TypeInt, MinValue: 1, MaxValue: 128, AllowAutoValue: true}, @@ -1640,6 +1659,8 @@ const ( OptimizerSwitch = "optimizer_switch" // SystemTimeZone is the name of 'system_time_zone' system variable. SystemTimeZone = "system_time_zone" + // CTEMaxRecursionDepth is the name of 'cte_max_recursion_depth' system variable. + CTEMaxRecursionDepth = "cte_max_recursion_depth" ) // GlobalVarAccessor is the interface for accessing global scope system and status variables. diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index f67753e71e52c..4d3e9edb0b17c 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -674,6 +674,7 @@ const ( DefTiDBEnableIndexMergeJoin = false DefTiDBTrackAggregateMemoryUsage = true DefTiDBEnableExchangePartition = false + DefCTEMaxRecursionDepth = 1000 ) // Process global variables. @@ -696,7 +697,7 @@ var ( MaxOfMaxAllowedPacket uint64 = 1073741824 ExpensiveQueryTimeThreshold uint64 = DefTiDBExpensiveQueryTimeThreshold MinExpensiveQueryTimeThreshold uint64 = 10 // 10s - CapturePlanBaseline = serverGlobalVariable{globalVal: BoolOff} + CapturePlanBaseline = serverGlobalVariable{globalVal: Off} DefExecutorConcurrency = 5 MemoryUsageAlarmRatio = atomic.NewFloat64(config.GetGlobalConfig().Performance.MemoryUsageAlarmRatio) ) diff --git a/sessionctx/variable/varsutil.go b/sessionctx/variable/varsutil.go index 9af28d6b90d04..b93e9735731a5 100644 --- a/sessionctx/variable/varsutil.go +++ b/sessionctx/variable/varsutil.go @@ -27,6 +27,7 @@ import ( "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/timeutil" ) @@ -97,19 +98,26 @@ func GetMaxDeltaSchemaCount() int64 { // BoolToOnOff returns the string representation of a bool, i.e. "ON/OFF" func BoolToOnOff(b bool) string { if b { - return BoolOn + return On } - return BoolOff + return Off } func int32ToBoolStr(i int32) string { if i == 1 { - return BoolOn + return On } - return BoolOff + return Off } -func checkCharacterValid(normalizedValue string, argName string) (string, error) { +func checkCollation(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + if _, err := collate.GetCollationByName(normalizedValue); err != nil { + return normalizedValue, errors.Trace(err) + } + return normalizedValue, nil +} + +func checkCharacterSet(normalizedValue string, argName string) (string, error) { if normalizedValue == "" { return normalizedValue, errors.Trace(ErrWrongValueForVar.GenWithStackByArgs(argName, "NULL")) } @@ -128,14 +136,14 @@ func checkReadOnly(vars *SessionVars, normalizedValue string, originalValue stri } if TiDBOptOn(normalizedValue) { if !vars.EnableNoopFuncs && scope == ScopeSession { - return BoolOff, ErrFunctionsNoopImpl.GenWithStackByArgs(feature) + return Off, ErrFunctionsNoopImpl.GenWithStackByArgs(feature) } val, err := vars.GlobalVarsAccessor.GetGlobalSysVar(TiDBEnableNoopFuncs) if err != nil { return originalValue, errUnknownSystemVariable.GenWithStackByArgs(TiDBEnableNoopFuncs) } if scope == ScopeGlobal && !TiDBOptOn(val) { - return BoolOff, ErrFunctionsNoopImpl.GenWithStackByArgs(feature) + return Off, ErrFunctionsNoopImpl.GenWithStackByArgs(feature) } } return normalizedValue, nil @@ -364,9 +372,9 @@ const ( // TiDBOptMultiStmt converts multi-stmt options to int. func TiDBOptMultiStmt(opt string) int { switch opt { - case BoolOff: + case Off: return OffInt - case BoolOn: + case On: return OnInt } return WarnInt @@ -404,6 +412,14 @@ func tidbOptPositiveInt32(opt string, defaultVal int) int { return val } +func tidbOptInt(opt string, defaultVal int) int { + val, err := strconv.Atoi(opt) + if err != nil { + return defaultVal + } + return val +} + func tidbOptInt64(opt string, defaultVal int64) int64 { val, err := strconv.ParseInt(opt, 10, 64) if err != nil { diff --git a/statistics/cmsketch_test.go b/statistics/cmsketch_test.go index 38fe81c79c60f..48be1e716e178 100644 --- a/statistics/cmsketch_test.go +++ b/statistics/cmsketch_test.go @@ -301,5 +301,6 @@ func (s *testStatisticsSuite) TestCMSketchCodingTopN(c *C) { c.Assert(err, IsNil) c.Assert(lSketch.Equal(rSketch), IsTrue) // do not panic - DecodeCMSketchAndTopN([]byte{}, rows) + _, _, err = DecodeCMSketchAndTopN([]byte{}, rows) + c.Assert(err, IsNil) } diff --git a/statistics/fmsketch.go b/statistics/fmsketch.go index 3fe66077b1272..9ee68bc1e98e9 100644 --- a/statistics/fmsketch.go +++ b/statistics/fmsketch.go @@ -96,6 +96,25 @@ func (s *FMSketch) InsertValue(sc *stmtctx.StatementContext, value types.Datum) return nil } +// InsertRowValue inserts multi-column values to the sketch. +func (s *FMSketch) InsertRowValue(sc *stmtctx.StatementContext, values []types.Datum) error { + b := make([]byte, 0, 8) + s.hashFunc.Reset() + for _, v := range values { + b = b[:0] + b, err := codec.EncodeValue(sc, b, v) + if err != nil { + return err + } + _, err = s.hashFunc.Write(b) + if err != nil { + return err + } + } + s.insertHashValue(s.hashFunc.Sum64()) + return nil +} + func buildFMSketch(sc *stmtctx.StatementContext, values []types.Datum, maxSize int) (*FMSketch, int64, error) { s := NewFMSketch(maxSize) for _, value := range values { diff --git a/statistics/handle/handle.go b/statistics/handle/handle.go index 1b27a36ca2a86..32c256afe8834 100644 --- a/statistics/handle/handle.go +++ b/statistics/handle/handle.go @@ -266,6 +266,9 @@ func (h *Handle) Update(is infoschema.InfoSchema) error { continue } tableInfo := table.Meta() + if oldTbl, ok := oldCache.tables[physicalID]; ok && oldTbl.Version >= version && tableInfo.UpdateTS == oldTbl.TblInfoUpdateTS { + continue + } tbl, err := h.TableStatsFromStorage(tableInfo, physicalID, false, 0) // Error is not nil may mean that there are some ddl changes on this table, we will not update it. if err != nil { @@ -280,6 +283,7 @@ func (h *Handle) Update(is infoschema.InfoSchema) error { tbl.Count = count tbl.ModifyCount = modifyCount tbl.Name = getFullTableName(is, tableInfo) + tbl.TblInfoUpdateTS = tableInfo.UpdateTS tables = append(tables, tbl) } h.updateStatsCache(oldCache.update(tables, deletedTableIDs, lastVersion)) diff --git a/statistics/handle/handle_test.go b/statistics/handle/handle_test.go index 6a7a397697d4b..0a55f03722b75 100644 --- a/statistics/handle/handle_test.go +++ b/statistics/handle/handle_test.go @@ -92,7 +92,8 @@ func (s *testStatsSuite) TestStatsCache(c *C) { testKit.MustExec("alter table t drop column c2") is = do.InfoSchema() do.StatsHandle().Clear() - do.StatsHandle().Update(is) + err = do.StatsHandle().Update(is) + c.Assert(err, IsNil) statsTbl = do.StatsHandle().GetTableStats(tableInfo) c.Assert(statsTbl.Pseudo, IsFalse) @@ -101,7 +102,8 @@ func (s *testStatsSuite) TestStatsCache(c *C) { is = do.InfoSchema() do.StatsHandle().Clear() - do.StatsHandle().Update(is) + err = do.StatsHandle().Update(is) + c.Assert(err, IsNil) statsTbl = do.StatsHandle().GetTableStats(tableInfo) c.Assert(statsTbl.Pseudo, IsFalse) } @@ -143,7 +145,8 @@ func (s *testStatsSuite) TestStatsCacheMemTracker(c *C) { testKit.MustExec("alter table t drop column c2") is = do.InfoSchema() do.StatsHandle().Clear() - do.StatsHandle().Update(is) + err = do.StatsHandle().Update(is) + c.Assert(err, IsNil) statsTbl = do.StatsHandle().GetTableStats(tableInfo) c.Assert(statsTbl.MemoryUsage() > 0, IsTrue) @@ -154,7 +157,8 @@ func (s *testStatsSuite) TestStatsCacheMemTracker(c *C) { is = do.InfoSchema() do.StatsHandle().Clear() - do.StatsHandle().Update(is) + err = do.StatsHandle().Update(is) + c.Assert(err, IsNil) statsTbl = do.StatsHandle().GetTableStats(tableInfo) c.Assert(statsTbl.Pseudo, IsFalse) } @@ -236,7 +240,8 @@ func (s *testStatsSuite) TestStatsStoreAndLoad(c *C) { statsTbl1 := do.StatsHandle().GetTableStats(tableInfo) do.StatsHandle().Clear() - do.StatsHandle().Update(is) + err = do.StatsHandle().Update(is) + c.Assert(err, IsNil) statsTbl2 := do.StatsHandle().GetTableStats(tableInfo) c.Assert(statsTbl2.Pseudo, IsFalse) c.Assert(statsTbl2.Count, Equals, int64(recordCount)) @@ -281,7 +286,8 @@ func (s *testStatsSuite) TestColumnIDs(c *C) { testKit.MustExec("alter table t drop column c1") is = do.InfoSchema() do.StatsHandle().Clear() - do.StatsHandle().Update(is) + err = do.StatsHandle().Update(is) + c.Assert(err, IsNil) tbl, err = is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) c.Assert(err, IsNil) tableInfo = tbl.Meta() @@ -448,7 +454,8 @@ func (s *testStatsSuite) TestLoadHist(c *C) { testKit.MustExec("insert into t values('bb','sdfga')") } c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) - h.Update(do.InfoSchema()) + err = h.Update(do.InfoSchema()) + c.Assert(err, IsNil) newStatsTbl := h.GetTableStats(tableInfo) // The stats table is updated. c.Assert(oldStatsTbl == newStatsTbl, IsFalse) @@ -1711,7 +1718,8 @@ func (s *testStatsSuite) TestExtendedStatsOps(c *C) { tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) c.Assert(err, IsNil) tableInfo := tbl.Meta() - do.StatsHandle().Update(is) + err = do.StatsHandle().Update(is) + c.Assert(err, IsNil) statsTbl := do.StatsHandle().GetTableStats(tableInfo) c.Assert(statsTbl, NotNil) c.Assert(statsTbl.ExtendedStats, NotNil) @@ -1719,7 +1727,8 @@ func (s *testStatsSuite) TestExtendedStatsOps(c *C) { tk.MustExec("update mysql.stats_extended set status = 1 where name = 's1'") do.StatsHandle().Clear() - do.StatsHandle().Update(is) + err = do.StatsHandle().Update(is) + c.Assert(err, IsNil) statsTbl = do.StatsHandle().GetTableStats(tableInfo) c.Assert(statsTbl, NotNil) c.Assert(statsTbl.ExtendedStats, NotNil) @@ -1729,7 +1738,8 @@ func (s *testStatsSuite) TestExtendedStatsOps(c *C) { tk.MustQuery("select type, column_ids, stats, status from mysql.stats_extended where name = 's1'").Check(testkit.Rows( "2 [2,3] 2", )) - do.StatsHandle().Update(is) + err = do.StatsHandle().Update(is) + c.Assert(err, IsNil) statsTbl = do.StatsHandle().GetTableStats(tableInfo) c.Assert(statsTbl.ExtendedStats, NotNil) c.Assert(len(statsTbl.ExtendedStats.Stats), Equals, 0) @@ -1752,7 +1762,8 @@ func (s *testStatsSuite) TestAdminReloadStatistics1(c *C) { tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) c.Assert(err, IsNil) tableInfo := tbl.Meta() - do.StatsHandle().Update(is) + err = do.StatsHandle().Update(is) + c.Assert(err, IsNil) statsTbl := do.StatsHandle().GetTableStats(tableInfo) c.Assert(statsTbl, NotNil) c.Assert(statsTbl.ExtendedStats, NotNil) @@ -1760,14 +1771,16 @@ func (s *testStatsSuite) TestAdminReloadStatistics1(c *C) { tk.MustExec("update mysql.stats_extended set status = 1 where name = 's1'") do.StatsHandle().Clear() - do.StatsHandle().Update(is) + err = do.StatsHandle().Update(is) + c.Assert(err, IsNil) statsTbl = do.StatsHandle().GetTableStats(tableInfo) c.Assert(statsTbl, NotNil) c.Assert(statsTbl.ExtendedStats, NotNil) c.Assert(len(statsTbl.ExtendedStats.Stats), Equals, 1) tk.MustExec("delete from mysql.stats_extended where name = 's1'") - do.StatsHandle().Update(is) + err = do.StatsHandle().Update(is) + c.Assert(err, IsNil) statsTbl = do.StatsHandle().GetTableStats(tableInfo) c.Assert(statsTbl.ExtendedStats, NotNil) c.Assert(len(statsTbl.ExtendedStats.Stats), Equals, 1) @@ -1826,7 +1839,8 @@ func (s *testStatsSuite) TestCorrelationStatsCompute(c *C) { tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) c.Assert(err, IsNil) tableInfo := tbl.Meta() - do.StatsHandle().Update(is) + err = do.StatsHandle().Update(is) + c.Assert(err, IsNil) statsTbl := do.StatsHandle().GetTableStats(tableInfo) c.Assert(statsTbl, NotNil) c.Assert(statsTbl.ExtendedStats, NotNil) @@ -1837,7 +1851,8 @@ func (s *testStatsSuite) TestCorrelationStatsCompute(c *C) { "2 [1,2] 1.000000 1", "2 [1,3] -1.000000 1", )) - do.StatsHandle().Update(is) + err = do.StatsHandle().Update(is) + c.Assert(err, IsNil) statsTbl = do.StatsHandle().GetTableStats(tableInfo) c.Assert(statsTbl, NotNil) c.Assert(statsTbl.ExtendedStats, NotNil) @@ -2734,3 +2749,38 @@ func (s *testSerialStatsSuite) TestCorrelationWithDefinedCollate(c *C) { c.Assert(rows[0][3], Equals, "[b,c]") c.Assert(rows[0][5], Equals, "-1.000000") } + +func (s *testSerialStatsSuite) TestFastAnalyzeColumnHistWithNullValue(c *C) { + defer cleanEnv(c, s.store, s.do) + testKit := testkit.NewTestKit(c, s.store) + testKit.MustExec("use test") + testKit.MustExec("drop table if exists t") + testKit.MustExec("create table t (a int)") + testKit.MustExec("insert into t values (1), (2), (3), (4), (NULL)") + testKit.MustExec("set @@tidb_enable_fast_analyze=1") + defer testKit.MustExec("set @@tidb_enable_fast_analyze=0") + testKit.MustExec("analyze table t with 0 topn, 2 buckets") + // If NULL is in hist, the min(lower_bound) will be "". + testKit.MustQuery("select min(lower_bound) from mysql.stats_buckets").Check(testkit.Rows("1")) +} + +func (s *testStatsSuite) TestStatsCacheUpdateSkip(c *C) { + defer cleanEnv(c, s.store, s.do) + testKit := testkit.NewTestKit(c, s.store) + do := s.do + h := do.StatsHandle() + testKit.MustExec("use test") + testKit.MustExec("create table t (c1 int, c2 int)") + testKit.MustExec("insert into t values(1, 2)") + c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) + testKit.MustExec("analyze table t") + is := do.InfoSchema() + tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + c.Assert(err, IsNil) + tableInfo := tbl.Meta() + statsTbl1 := h.GetTableStats(tableInfo) + c.Assert(statsTbl1.Pseudo, IsFalse) + h.Update(is) + statsTbl2 := h.GetTableStats(tableInfo) + c.Assert(statsTbl1, Equals, statsTbl2) +} diff --git a/statistics/handle/update_test.go b/statistics/handle/update_test.go index 02d6a0a2c589a..b105738098f4b 100644 --- a/statistics/handle/update_test.go +++ b/statistics/handle/update_test.go @@ -121,8 +121,10 @@ func (s *testStatsSuite) TestSingleSessionInsert(c *C) { tableInfo1 := tbl1.Meta() h := s.do.StatsHandle() - h.HandleDDLEvent(<-h.DDLEventCh()) - h.HandleDDLEvent(<-h.DDLEventCh()) + err = h.HandleDDLEvent(<-h.DDLEventCh()) + c.Assert(err, IsNil) + err = h.HandleDDLEvent(<-h.DDLEventCh()) + c.Assert(err, IsNil) c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) c.Assert(h.Update(is), IsNil) @@ -202,14 +204,16 @@ func (s *testStatsSuite) TestSingleSessionInsert(c *C) { for i := 0; i < rowCount1; i++ { testKit.MustExec("insert into t1 values (1,2)") } - h.DumpStatsDeltaToKV(handle.DumpDelta) + err = h.DumpStatsDeltaToKV(handle.DumpDelta) + c.Assert(err, IsNil) c.Assert(h.Update(is), IsNil) stats1 = h.GetTableStats(tableInfo1) c.Assert(stats1.Count, Equals, int64(rowCount1)) // not dumped testKit.MustExec("insert into t1 values (1,2)") - h.DumpStatsDeltaToKV(handle.DumpDelta) + err = h.DumpStatsDeltaToKV(handle.DumpDelta) + c.Assert(err, IsNil) c.Assert(h.Update(is), IsNil) stats1 = h.GetTableStats(tableInfo1) c.Assert(stats1.Count, Equals, int64(rowCount1)) @@ -234,7 +238,8 @@ func (s *testStatsSuite) TestRollback(c *C) { c.Assert(err, IsNil) tableInfo := tbl.Meta() h := s.do.StatsHandle() - h.HandleDDLEvent(<-h.DDLEventCh()) + err = h.HandleDDLEvent(<-h.DDLEventCh()) + c.Assert(err, IsNil) c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) c.Assert(h.Update(is), IsNil) @@ -268,7 +273,8 @@ func (s *testStatsSuite) TestMultiSession(c *C) { tableInfo1 := tbl1.Meta() h := s.do.StatsHandle() - h.HandleDDLEvent(<-h.DDLEventCh()) + err = h.HandleDDLEvent(<-h.DDLEventCh()) + c.Assert(err, IsNil) c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) c.Assert(h.Update(is), IsNil) @@ -312,7 +318,8 @@ func (s *testStatsSuite) TestTxnWithFailure(c *C) { tableInfo1 := tbl1.Meta() h := s.do.StatsHandle() - h.HandleDDLEvent(<-h.DDLEventCh()) + err = h.HandleDDLEvent(<-h.DDLEventCh()) + c.Assert(err, IsNil) rowCount1 := 10 testKit.MustExec("begin") @@ -352,7 +359,8 @@ func (s *testStatsSuite) TestUpdatePartition(c *C) { testKit.MustQuery("select @@tidb_partition_prune_mode").Check(testkit.Rows(string(s.do.StatsHandle().CurrentPruneMode()))) testKit.MustExec("use test") testkit.WithPruneMode(testKit, variable.Static, func() { - s.do.StatsHandle().RefreshVars() + err := s.do.StatsHandle().RefreshVars() + c.Assert(err, IsNil) testKit.MustExec("drop table if exists t") createTable := `CREATE TABLE t (a int, b char(5)) PARTITION BY RANGE (a) (PARTITION p0 VALUES LESS THAN (6),PARTITION p1 VALUES LESS THAN (11))` testKit.MustExec(createTable) @@ -421,7 +429,8 @@ func (s *testStatsSuite) TestAutoUpdate(c *C) { tableInfo := tbl.Meta() h := do.StatsHandle() - h.HandleDDLEvent(<-h.DDLEventCh()) + err = h.HandleDDLEvent(<-h.DDLEventCh()) + c.Assert(err, IsNil) c.Assert(h.Update(is), IsNil) stats := h.GetTableStats(tableInfo) c.Assert(stats.Count, Equals, int64(0)) @@ -586,10 +595,11 @@ func (s *testSerialStatsSuite) TestAutoAnalyzeOnChangeAnalyzeVer(c *C) { handle.AutoAnalyzeMinCnt = 1000 }() h := do.StatsHandle() - h.HandleDDLEvent(<-h.DDLEventCh()) + err := h.HandleDDLEvent(<-h.DDLEventCh()) + c.Assert(err, IsNil) c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) is := do.InfoSchema() - err := h.UpdateSessionVar() + err = h.UpdateSessionVar() c.Assert(err, IsNil) c.Assert(h.Update(is), IsNil) // Auto analyze when global ver is 1. @@ -626,7 +636,8 @@ func (s *testSerialStatsSuite) TestAutoAnalyzeOnChangeAnalyzeVer(c *C) { // Add a new table after the analyze version set to 2. tk.MustExec("create table tt(a int, index idx(a))") tk.MustExec("insert into tt values(1), (2), (3), (4), (5)") - h.HandleDDLEvent(<-h.DDLEventCh()) + err = h.HandleDDLEvent(<-h.DDLEventCh()) + c.Assert(err, IsNil) c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) is = do.InfoSchema() tbl2, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("tt")) @@ -700,7 +711,8 @@ func (s *testStatsSuite) TestUpdateErrorRate(c *C) { testKit := testkit.NewTestKit(c, s.store) testKit.MustExec("use test") testKit.MustExec("create table t (a bigint(64), b bigint(64), primary key(a), index idx(b))") - h.HandleDDLEvent(<-h.DDLEventCh()) + err := h.HandleDDLEvent(<-h.DDLEventCh()) + c.Assert(err, IsNil) testKit.MustExec("insert into t values (1, 3)") @@ -776,7 +788,8 @@ func (s *testStatsSuite) TestUpdatePartitionErrorRate(c *C) { testKit.MustExec("use test") testKit.MustExec(`set @@tidb_partition_prune_mode='` + string(variable.Static) + `'`) testKit.MustExec("create table t (a bigint(64), primary key(a)) partition by range (a) (partition p0 values less than (30))") - h.HandleDDLEvent(<-h.DDLEventCh()) + err := h.HandleDDLEvent(<-h.DDLEventCh()) + c.Assert(err, IsNil) testKit.MustExec("insert into t values (1)") @@ -1105,7 +1118,8 @@ func (s *testStatsSuite) TestOutOfOrderUpdate(c *C) { c.Assert(err, IsNil) tableInfo := tbl.Meta() h := do.StatsHandle() - h.HandleDDLEvent(<-h.DDLEventCh()) + err = h.HandleDDLEvent(<-h.DDLEventCh()) + c.Assert(err, IsNil) // Simulate the case that another tidb has inserted some value, but delta info has not been dumped to kv yet. testKit.MustExec("insert into t values (2,2),(4,5)") @@ -1558,7 +1572,8 @@ func (s *testStatsSuite) TestIndexQueryFeedback(c *C) { testKit.MustExec(fmt.Sprintf(`insert into t values (1, %d, %d, %d, %d, %d, %d, "%s")`, i, i, i, i, i, i, fmt.Sprintf("1000-01-%02d", i+1))) } h := s.do.StatsHandle() - h.HandleDDLEvent(<-h.DDLEventCh()) + err := h.HandleDDLEvent(<-h.DDLEventCh()) + c.Assert(err, IsNil) c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) testKit.MustExec("analyze table t with 3 buckets") for i := 0; i < 20; i++ { @@ -1696,7 +1711,8 @@ func (s *testStatsSuite) TestIndexQueryFeedback4TopN(c *C) { testKit.MustExec(`insert into t values (1)`) } h := s.do.StatsHandle() - h.HandleDDLEvent(<-h.DDLEventCh()) + err := h.HandleDDLEvent(<-h.DDLEventCh()) + c.Assert(err, IsNil) c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) testKit.MustExec("set @@tidb_enable_fast_analyze = 1") testKit.MustExec("analyze table t with 3 buckets") @@ -1817,7 +1833,8 @@ func (s *testStatsSuite) TestFeedbackRanges(c *C) { for i := 0; i < 20; i++ { testKit.MustExec(fmt.Sprintf("insert into t values (%d, %d)", i, i)) } - h.HandleDDLEvent(<-h.DDLEventCh()) + err := h.HandleDDLEvent(<-h.DDLEventCh()) + c.Assert(err, IsNil) c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) testKit.MustExec("analyze table t with 3 buckets") for i := 30; i < 40; i++ { @@ -1895,8 +1912,10 @@ func (s *testStatsSuite) TestUnsignedFeedbackRanges(c *C) { testKit.MustExec(fmt.Sprintf("insert into t values (%d)", i)) testKit.MustExec(fmt.Sprintf("insert into t1 values (%d)", i)) } - h.HandleDDLEvent(<-h.DDLEventCh()) - h.HandleDDLEvent(<-h.DDLEventCh()) + err := h.HandleDDLEvent(<-h.DDLEventCh()) + c.Assert(err, IsNil) + err = h.HandleDDLEvent(<-h.DDLEventCh()) + c.Assert(err, IsNil) c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) testKit.MustExec("analyze table t, t1 with 3 buckets") for i := 30; i < 40; i++ { @@ -2045,7 +2064,8 @@ func (s *testStatsSuite) TestDisableFeedback(c *C) { }() statistics.FeedbackProbability.Store(0.0) oldNum := &dto.Metric{} - metrics.StoreQueryFeedbackCounter.WithLabelValues(metrics.LblOK).Write(oldNum) + err := metrics.StoreQueryFeedbackCounter.WithLabelValues(metrics.LblOK).Write(oldNum) + c.Assert(err, IsNil) testKit.MustExec("use test") testKit.MustExec("create table t (a int, b int, index idx_a(a))") testKit.MustExec("insert into t values (1, 1), (2, 2), (3, 3), (5, 5)") @@ -2055,7 +2075,8 @@ func (s *testStatsSuite) TestDisableFeedback(c *C) { } newNum := &dto.Metric{} - metrics.StoreQueryFeedbackCounter.WithLabelValues(metrics.LblOK).Write(newNum) + err = metrics.StoreQueryFeedbackCounter.WithLabelValues(metrics.LblOK).Write(newNum) + c.Assert(err, IsNil) c.Assert(subtraction(newNum, oldNum), Equals, 0) } @@ -2069,7 +2090,8 @@ func (s *testStatsSuite) TestFeedbackCounter(c *C) { }() statistics.FeedbackProbability.Store(1) oldNum := &dto.Metric{} - metrics.StoreQueryFeedbackCounter.WithLabelValues(metrics.LblOK).Write(oldNum) + err := metrics.StoreQueryFeedbackCounter.WithLabelValues(metrics.LblOK).Write(oldNum) + c.Assert(err, IsNil) testKit.MustExec("use test") testKit.MustExec("create table t (a int, b int, index idx_a(a))") testKit.MustExec("insert into t values (1, 1), (2, 2), (3, 3), (5, 5)") @@ -2079,7 +2101,8 @@ func (s *testStatsSuite) TestFeedbackCounter(c *C) { } newNum := &dto.Metric{} - metrics.StoreQueryFeedbackCounter.WithLabelValues(metrics.LblOK).Write(newNum) + err = metrics.StoreQueryFeedbackCounter.WithLabelValues(metrics.LblOK).Write(newNum) + c.Assert(err, IsNil) c.Assert(subtraction(newNum, oldNum), Equals, 20) } diff --git a/statistics/row_sampler.go b/statistics/row_sampler.go new file mode 100644 index 0000000000000..e78895e19da13 --- /dev/null +++ b/statistics/row_sampler.go @@ -0,0 +1,293 @@ +// 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 statistics + +import ( + "container/heap" + "context" + "math/rand" + + "github.com/pingcap/tidb/sessionctx/stmtctx" + "github.com/pingcap/tidb/tablecodec" + "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/collate" + "github.com/pingcap/tidb/util/sqlexec" + "github.com/pingcap/tipb/go-tipb" +) + +// RowSampleCollector collects the samples from the source and organize the samples by row. +// It will maintain the following things: +// Row samples. +// FM sketches(To calculate the NDV). +// Null counts. +// The data sizes. +// The number of rows. +// It uses weighted reservoir sampling(A-Res) to do the sampling. +type RowSampleCollector struct { + Samples WeightedRowSampleHeap + NullCount []int64 + FMSketches []*FMSketch + TotalSizes []int64 + Count int64 + MaxSampleSize int +} + +// RowSampleItem is the item for the RowSampleCollector. The weight is needed for the sampling algorithm. +type RowSampleItem struct { + Columns []types.Datum + Weight int64 +} + +// WeightedRowSampleHeap implements the Heap interface. +type WeightedRowSampleHeap []*RowSampleItem + +// Len implements the Heap interface. +func (h WeightedRowSampleHeap) Len() int { + return len(h) +} + +// Swap implements the Heap interface. +func (h WeightedRowSampleHeap) Swap(i, j int) { + h[i], h[j] = h[j], h[i] +} + +// Less implements the Heap interface. +func (h WeightedRowSampleHeap) Less(i, j int) bool { + return h[i].Weight < h[j].Weight +} + +// Push implements the Heap interface. +func (h *WeightedRowSampleHeap) Push(i interface{}) { + *h = append(*h, i.(*RowSampleItem)) +} + +// Pop implements the Heap interface. +func (h *WeightedRowSampleHeap) Pop() interface{} { + old := *h + n := len(old) + item := old[n-1] + *h = old[:n-1] + return item +} + +// RowSampleBuilder is used to construct the RowSampleCollector to get the samples. +type RowSampleBuilder struct { + Sc *stmtctx.StatementContext + RecordSet sqlexec.RecordSet + ColsFieldType []*types.FieldType + Collators []collate.Collator + ColGroups [][]int64 + MaxSampleSize int + MaxFMSketchSize int + Rng *rand.Rand +} + +// Collect first builds the collector. Then maintain the null count, FM sketch and the data size for each column and +// column group. +// Then use the weighted reservoir sampling to collect the samples. +func (s *RowSampleBuilder) Collect() (*RowSampleCollector, error) { + collector := &RowSampleCollector{ + Samples: make(WeightedRowSampleHeap, 0, s.MaxSampleSize), + NullCount: make([]int64, len(s.ColsFieldType)+len(s.ColGroups)), + FMSketches: make([]*FMSketch, 0, len(s.ColsFieldType)+len(s.ColGroups)), + TotalSizes: make([]int64, len(s.ColsFieldType)+len(s.ColGroups)), + MaxSampleSize: s.MaxSampleSize, + } + for i := 0; i < len(s.ColsFieldType)+len(s.ColGroups); i++ { + collector.FMSketches = append(collector.FMSketches, NewFMSketch(s.MaxFMSketchSize)) + } + ctx := context.TODO() + chk := s.RecordSet.NewChunk() + it := chunk.NewIterator4Chunk(chk) + for { + err := s.RecordSet.Next(ctx, chk) + if err != nil { + return nil, err + } + if chk.NumRows() == 0 { + return collector, nil + } + collector.Count += int64(chk.NumRows()) + for row := it.Begin(); row != it.End(); row = it.Next() { + datums := RowToDatums(row, s.RecordSet.Fields()) + for i, val := range datums { + // For string values, we use the collation key instead of the original value. + if s.Collators[i] != nil && !val.IsNull() { + decodedVal, err := tablecodec.DecodeColumnValue(val.GetBytes(), s.ColsFieldType[i], s.Sc.TimeZone) + if err != nil { + return nil, err + } + decodedVal.SetBytesAsString(s.Collators[i].Key(decodedVal.GetString()), decodedVal.Collation(), uint32(decodedVal.Length())) + encodedKey, err := tablecodec.EncodeValue(s.Sc, nil, decodedVal) + if err != nil { + return nil, err + } + val.SetBytes(encodedKey) + } + } + err := collector.collectColumns(s.Sc, datums) + if err != nil { + return nil, err + } + err = collector.collectColumnGroups(s.Sc, datums, s.ColGroups) + if err != nil { + return nil, err + } + weight := s.Rng.Int63() + newCols := make([]types.Datum, len(datums)) + for i := range datums { + datums[i].Copy(&newCols[i]) + } + item := &RowSampleItem{ + Columns: newCols, + Weight: weight, + } + collector.sampleZippedRow(item) + } + } +} + +func (s *RowSampleCollector) collectColumns(sc *stmtctx.StatementContext, cols []types.Datum) error { + for i, col := range cols { + if col.IsNull() { + s.NullCount[i]++ + continue + } + s.TotalSizes[i] += int64(len(col.GetBytes())) - 1 + // Minus one is to remove the flag byte. + err := s.FMSketches[i].InsertValue(sc, col) + if err != nil { + return err + } + } + return nil +} + +func (s *RowSampleCollector) collectColumnGroups(sc *stmtctx.StatementContext, cols []types.Datum, colGroups [][]int64) error { + colLen := len(cols) + datumBuffer := make([]types.Datum, 0, len(cols)) + for i, group := range colGroups { + datumBuffer = datumBuffer[:0] + hasNull := true + for _, c := range group { + datumBuffer = append(datumBuffer, cols[c]) + hasNull = hasNull && cols[c].IsNull() + s.TotalSizes[colLen+i] += int64(len(cols[c].GetBytes())) - 1 + } + // We don't maintain the null counts information for the multi-column group + if hasNull && len(group) == 1 { + s.NullCount[colLen+i]++ + continue + } + err := s.FMSketches[colLen+i].InsertRowValue(sc, datumBuffer) + if err != nil { + return err + } + } + return nil +} + +func (s *RowSampleCollector) sampleZippedRow(sample *RowSampleItem) { + if len(s.Samples) < s.MaxSampleSize { + s.Samples = append(s.Samples, sample) + if len(s.Samples) == s.MaxSampleSize { + heap.Init(&s.Samples) + } + return + } + if s.Samples[0].Weight < sample.Weight { + s.Samples[0] = sample + heap.Fix(&s.Samples, 0) + } +} + +// ToProto converts the collector to proto struct. +func (s *RowSampleCollector) ToProto() *tipb.RowSampleCollector { + pbFMSketches := make([]*tipb.FMSketch, 0, len(s.FMSketches)) + for _, sketch := range s.FMSketches { + pbFMSketches = append(pbFMSketches, FMSketchToProto(sketch)) + } + collector := &tipb.RowSampleCollector{ + Samples: RowSamplesToProto(s.Samples), + NullCounts: s.NullCount, + Count: s.Count, + FmSketch: pbFMSketches, + TotalSize: s.TotalSizes, + } + return collector +} + +// FromProto constructs the collector from the proto struct. +func (s *RowSampleCollector) FromProto(pbCollector *tipb.RowSampleCollector) { + s.Count = pbCollector.Count + s.NullCount = pbCollector.NullCounts + s.FMSketches = make([]*FMSketch, 0, len(pbCollector.FmSketch)) + for _, pbSketch := range pbCollector.FmSketch { + s.FMSketches = append(s.FMSketches, FMSketchFromProto(pbSketch)) + } + s.TotalSizes = pbCollector.TotalSize + s.Samples = make(WeightedRowSampleHeap, 0, len(pbCollector.Samples)) + for _, pbSample := range pbCollector.Samples { + data := make([]types.Datum, 0, len(pbSample.Row)) + for _, col := range pbSample.Row { + b := make([]byte, len(col)) + copy(b, col) + data = append(data, types.NewBytesDatum(b)) + } + // The samples collected from regions are also organized by binary heap. So we can just copy the slice. + // No need to maintain the heap again. + s.Samples = append(s.Samples, &RowSampleItem{ + Columns: data, + Weight: pbSample.Weight, + }) + } +} + +// MergeCollector merges the collectors to a final one. +func (s *RowSampleCollector) MergeCollector(subCollector *RowSampleCollector) { + s.Count += subCollector.Count + for i := range subCollector.FMSketches { + s.FMSketches[i].MergeFMSketch(subCollector.FMSketches[i]) + } + for i := range subCollector.NullCount { + s.NullCount[i] += subCollector.NullCount[i] + } + for i := range subCollector.TotalSizes { + s.TotalSizes[i] += subCollector.TotalSizes[i] + } + for _, sample := range subCollector.Samples { + s.sampleZippedRow(sample) + } +} + +// RowSamplesToProto converts the samp slice to the pb struct. +func RowSamplesToProto(samples WeightedRowSampleHeap) []*tipb.RowSample { + if len(samples) == 0 { + return nil + } + rows := make([]*tipb.RowSample, 0, len(samples)) + colLen := len(samples[0].Columns) + for _, sample := range samples { + pbRow := &tipb.RowSample{ + Row: make([][]byte, 0, colLen), + Weight: sample.Weight, + } + for _, c := range sample.Columns { + pbRow.Row = append(pbRow.Row, c.GetBytes()) + } + rows = append(rows, pbRow) + } + return rows +} diff --git a/statistics/sample_test.go b/statistics/sample_test.go index 1a9647505b547..0183a2a57dac2 100644 --- a/statistics/sample_test.go +++ b/statistics/sample_test.go @@ -14,6 +14,7 @@ package statistics import ( + "math/rand" "time" . "github.com/pingcap/check" @@ -135,3 +136,121 @@ func (s *testSampleSuite) TestCollectorProtoConversion(c *C) { c.Assert(len(collector.Samples), Equals, len(s.Samples)) } } + +func (s *testSampleSuite) recordSetForWeightSamplingTest(size int) *recordSet { + r := &recordSet{ + data: make([]types.Datum, 0, size), + count: size, + } + for i := 0; i < size; i++ { + r.data = append(r.data, types.NewIntDatum(int64(i))) + } + r.setFields(mysql.TypeLonglong) + return r +} + +func (s *testSampleSuite) recordSetForDistributedSamplingTest(size, batch int) []*recordSet { + sets := make([]*recordSet, 0, batch) + batchSize := size / batch + for i := 0; i < batch; i++ { + r := &recordSet{ + data: make([]types.Datum, 0, batchSize), + count: batchSize, + } + for j := 0; j < size/batch; j++ { + r.data = append(r.data, types.NewIntDatum(int64(j+batchSize*i))) + } + r.setFields(mysql.TypeLonglong) + sets = append(sets, r) + } + return sets +} + +func (s *testSampleSuite) TestWeightedSampling(c *C) { + sampleNum := int64(20) + rowNum := 100 + loopCnt := 1000 + rs := s.recordSetForWeightSamplingTest(rowNum) + sc := mock.NewContext().GetSessionVars().StmtCtx + // The loop which is commented out is used for stability test. + // This test can run 800 times in a row without any failure. + // for x := 0; x < 800; x++ { + itemCnt := make([]int, rowNum) + for loopI := 0; loopI < loopCnt; loopI++ { + builder := &RowSampleBuilder{ + Sc: sc, + RecordSet: rs, + ColsFieldType: []*types.FieldType{types.NewFieldType(mysql.TypeLonglong)}, + Collators: make([]collate.Collator, 1), + ColGroups: nil, + MaxSampleSize: int(sampleNum), + MaxFMSketchSize: 1000, + Rng: rand.New(rand.NewSource(time.Now().UnixNano())), + } + collector, err := builder.Collect() + c.Assert(err, IsNil) + for i := 0; i < collector.MaxSampleSize; i++ { + a := collector.Samples[i].Columns[0].GetInt64() + itemCnt[a]++ + } + c.Assert(rs.Close(), IsNil) + } + expFrequency := float64(sampleNum) * float64(loopCnt) / float64(rowNum) + delta := 0.5 + for _, cnt := range itemCnt { + if float64(cnt) < expFrequency/(1+delta) || float64(cnt) > expFrequency*(1+delta) { + c.Assert(false, IsTrue, Commentf("The frequency %v is exceed the Chernoff Bound", cnt)) + } + } + // } +} + +func (s *testSampleSuite) TestDistributedWeightedSampling(c *C) { + sampleNum := int64(10) + rowNum := 100 + loopCnt := 1500 + batch := 5 + sets := s.recordSetForDistributedSamplingTest(rowNum, batch) + sc := mock.NewContext().GetSessionVars().StmtCtx + // The loop which is commented out is used for stability test. + // This test can run 800 times in a row without any failure. + // for x := 0; x < 800; x++ { + itemCnt := make([]int, rowNum) + for loopI := 1; loopI < loopCnt; loopI++ { + rootRowCollector := &RowSampleCollector{ + NullCount: make([]int64, 1), + FMSketches: make([]*FMSketch, 0, 1), + TotalSizes: make([]int64, 1), + Samples: make(WeightedRowSampleHeap, 0, sampleNum), + MaxSampleSize: int(sampleNum), + } + rootRowCollector.FMSketches = append(rootRowCollector.FMSketches, NewFMSketch(1000)) + for i := 0; i < batch; i++ { + builder := &RowSampleBuilder{ + Sc: sc, + RecordSet: sets[i], + ColsFieldType: []*types.FieldType{types.NewFieldType(mysql.TypeLonglong)}, + Collators: make([]collate.Collator, 1), + ColGroups: nil, + MaxSampleSize: int(sampleNum), + MaxFMSketchSize: 1000, + Rng: rand.New(rand.NewSource(time.Now().UnixNano())), + } + collector, err := builder.Collect() + c.Assert(err, IsNil) + rootRowCollector.MergeCollector(collector) + c.Assert(sets[i].Close(), IsNil) + } + for _, sample := range rootRowCollector.Samples { + itemCnt[sample.Columns[0].GetInt64()]++ + } + } + expFrequency := float64(sampleNum) * float64(loopCnt) / float64(rowNum) + delta := 0.5 + for _, cnt := range itemCnt { + if float64(cnt) < expFrequency/(1+delta) || float64(cnt) > expFrequency*(1+delta) { + c.Assert(false, IsTrue, Commentf("the frequency %v is exceed the Chernoff Bound", cnt)) + } + } + // } +} diff --git a/statistics/scalar_test.go b/statistics/scalar_test.go index 63b04c171a3e8..bf895e908dcf2 100644 --- a/statistics/scalar_test.go +++ b/statistics/scalar_test.go @@ -25,7 +25,10 @@ const eps = 1e-9 func getDecimal(value float64) *types.MyDecimal { dec := &types.MyDecimal{} - dec.FromFloat64(value) + err := dec.FromFloat64(value) + if err != nil { + panic(err) + } return dec } diff --git a/statistics/selectivity_test.go b/statistics/selectivity_test.go index 36f9b9fdebb9f..32ed9fcb9650b 100644 --- a/statistics/selectivity_test.go +++ b/statistics/selectivity_test.go @@ -557,8 +557,12 @@ func BenchmarkSelectivity(b *testing.B) { file, err := os.Create("cpu.profile") c.Assert(err, IsNil) - defer file.Close() - pprof.StartCPUProfile(file) + defer func() { + err := file.Close() + c.Assert(err, IsNil) + }() + err = pprof.StartCPUProfile(file) + c.Assert(err, IsNil) b.Run("Selectivity", func(b *testing.B) { b.ResetTimer() diff --git a/statistics/statistics_test.go b/statistics/statistics_test.go index 255914c3caf4a..e74e04f7596ac 100644 --- a/statistics/statistics_test.go +++ b/statistics/statistics_test.go @@ -465,7 +465,10 @@ func (s *testStatisticsSuite) TestPseudoTable(c *C) { func buildCMSketch(values []types.Datum) *CMSketch { cms := NewCMSketch(8, 2048) for _, val := range values { - cms.insert(&val) + err := cms.insert(&val) + if err != nil { + panic(err) + } } return cms } diff --git a/statistics/table.go b/statistics/table.go index 9a7f5606bdd27..7628e018e25a5 100644 --- a/statistics/table.go +++ b/statistics/table.go @@ -62,6 +62,12 @@ type Table struct { Version uint64 Name string ExtendedStats *ExtendedStatsColl + // TblInfoUpdateTS is the UpdateTS of the TableInfo used when filling this struct. + // It is the schema version of the corresponding table. It is used to skip redundant + // loading of stats, i.e, if the cached stats is already update-to-date with mysql.stats_xxx tables, + // and the schema of the table does not change, we don't need to load the stats for this + // table again. + TblInfoUpdateTS uint64 } // ExtendedStatsItem is the cached item of a mysql.stats_extended record. @@ -136,9 +142,10 @@ func (t *Table) Copy() *Table { newHistColl.Indices[id] = idx } nt := &Table{ - HistColl: newHistColl, - Version: t.Version, - Name: t.Name, + HistColl: newHistColl, + Version: t.Version, + Name: t.Name, + TblInfoUpdateTS: t.TblInfoUpdateTS, } if t.ExtendedStats != nil { newExtStatsColl := &ExtendedStatsColl{ diff --git a/store/copr/batch_coprocessor.go b/store/copr/batch_coprocessor.go index aa2e21e89a0c6..f851f5b4bee14 100644 --- a/store/copr/batch_coprocessor.go +++ b/store/copr/batch_coprocessor.go @@ -170,7 +170,7 @@ func buildBatchCopTasks(bo *tikv.Backoffer, cache *tikv.RegionCache, ranges *tik } } -func (c *CopClient) sendBatch(ctx context.Context, req *kv.Request, vars *kv.Variables) kv.Response { +func (c *CopClient) sendBatch(ctx context.Context, req *kv.Request, vars *tikv.Variables) kv.Response { if req.KeepOrder || req.Desc { return copErrorResponse{errors.New("batch coprocessor cannot prove keep order or desc property")} } @@ -209,7 +209,7 @@ type batchCopIterator struct { // Batch results are stored in respChan. respChan chan *batchCopResponse - vars *kv.Variables + vars *tikv.Variables memTracker *memory.Tracker diff --git a/store/copr/coprocessor.go b/store/copr/coprocessor.go index 484f6b078e3c8..2d42ecbbbe68b 100644 --- a/store/copr/coprocessor.go +++ b/store/copr/coprocessor.go @@ -63,7 +63,11 @@ type CopClient struct { } // Send builds the request and gets the coprocessor iterator response. -func (c *CopClient) Send(ctx context.Context, req *kv.Request, vars *kv.Variables, sessionMemTracker *memory.Tracker, enabledRateLimitAction bool) kv.Response { +func (c *CopClient) Send(ctx context.Context, req *kv.Request, variables interface{}, sessionMemTracker *memory.Tracker, enabledRateLimitAction bool) kv.Response { + vars, ok := variables.(*tikv.Variables) + if !ok { + return copErrorResponse{errors.Errorf("unsupported variables:%+v", variables)} + } if req.StoreType == kv.TiFlash && req.BatchCop { logutil.BgLogger().Debug("send batch requests") return c.sendBatch(ctx, req, vars) @@ -241,7 +245,7 @@ type copIterator struct { // Otherwise, results are stored in respChan. respChan chan *copResponse - vars *kv.Variables + vars *tikv.Variables memTracker *memory.Tracker @@ -268,7 +272,7 @@ type copIteratorWorker struct { req *kv.Request respChan chan<- *copResponse finishCh <-chan struct{} - vars *kv.Variables + vars *tikv.Variables *tikv.ClientHelper memTracker *memory.Tracker @@ -912,8 +916,8 @@ func (worker *copIteratorWorker) handleCopResponse(bo *tikv.Backoffer, rpcCtx *t resp.detail.CalleeAddress = rpcCtx.Addr } resp.respTime = costTime - sd := &execdetails.ScanDetail{} - td := execdetails.TimeDetail{} + sd := &util.ScanDetail{} + td := util.TimeDetail{} if pbDetails := resp.pbResp.ExecDetailsV2; pbDetails != nil { // Take values in `ExecDetailsV2` first. if timeDetail := pbDetails.TimeDetail; timeDetail != nil { diff --git a/store/copr/coprocessor_cache_test.go b/store/copr/coprocessor_cache_test.go index 45ee3ef0ddfac..14084f978d20a 100644 --- a/store/copr/coprocessor_cache_test.go +++ b/store/copr/coprocessor_cache_test.go @@ -85,10 +85,6 @@ func (s *testCoprocessorSuite) TestDisable(c *C) { v = cache.CheckResponseAdmission(1024, time.Second*5) c.Assert(v, Equals, false) - cache, err = newCoprCache(&config.CoprocessorCache{CapacityMB: 0.1, AdmissionMaxResultMB: 1}) - c.Assert(err, NotNil) - c.Assert(cache, IsNil) - cache, err = newCoprCache(&config.CoprocessorCache{CapacityMB: 0.001}) c.Assert(err, NotNil) c.Assert(cache, IsNil) diff --git a/store/copr/coprocessor_test.go b/store/copr/coprocessor_test.go index fbd029f72f016..73950f80390af 100644 --- a/store/copr/coprocessor_test.go +++ b/store/copr/coprocessor_test.go @@ -15,6 +15,7 @@ package copr import ( "context" + "testing" . "github.com/pingcap/check" "github.com/pingcap/tidb/kv" @@ -23,6 +24,11 @@ import ( "github.com/pingcap/tidb/store/tikv/mockstore/mocktikv" ) +func TestT(t *testing.T) { + CustomVerboseFlag = true + TestingT(t) +} + type testCoprocessorSuite struct { } diff --git a/store/copr/mpp.go b/store/copr/mpp.go index 71c246775d521..bb9a14f08547a 100644 --- a/store/copr/mpp.go +++ b/store/copr/mpp.go @@ -132,7 +132,7 @@ type mppIterator struct { closed uint32 - vars *kv.Variables + vars *tikv.Variables mu sync.Mutex } @@ -436,7 +436,8 @@ func (m *mppIterator) Next(ctx context.Context) (kv.ResultSubset, error) { } // DispatchMPPTasks dispatches all the mpp task and waits for the responses. -func (c *MPPClient) DispatchMPPTasks(ctx context.Context, vars *kv.Variables, dispatchReqs []*kv.MPPDispatchRequest) kv.Response { +func (c *MPPClient) DispatchMPPTasks(ctx context.Context, variables interface{}, dispatchReqs []*kv.MPPDispatchRequest) kv.Response { + vars := variables.(*tikv.Variables) ctxChild, cancelFunc := context.WithCancel(ctx) iter := &mppIterator{ store: c.store, diff --git a/store/driver/txn/txn_driver.go b/store/driver/txn/txn_driver.go index 94d43c32c3041..feec81c6c9abe 100644 --- a/store/driver/txn/txn_driver.go +++ b/store/driver/txn/txn_driver.go @@ -115,6 +115,17 @@ func (txn *tikvTxn) SetOption(opt int, val interface{}) { } } +// SetVars sets variables to the transaction. +func (txn *tikvTxn) SetVars(vars interface{}) { + if vs, ok := vars.(*tikv.Variables); ok { + txn.KVTxn.SetVars(vs) + } +} + +func (txn *tikvTxn) GetVars() interface{} { + return txn.KVTxn.GetVars() +} + func (txn *tikvTxn) extractKeyErr(err error) error { if e, ok := errors.Cause(err).(*tikvstore.ErrKeyExist); ok { return txn.extractKeyExistsErr(e.GetKey()) diff --git a/store/driver/util_test.go b/store/driver/util_test.go index 62477bd5bf202..f46a482bd823e 100644 --- a/store/driver/util_test.go +++ b/store/driver/util_test.go @@ -18,6 +18,7 @@ import ( "flag" "fmt" "sync" + "testing" . "github.com/pingcap/check" "github.com/pingcap/errors" @@ -27,6 +28,11 @@ import ( "github.com/pingcap/tidb/store/tikv" ) +func TestT(t *testing.T) { + CustomVerboseFlag = true + TestingT(t) +} + var ( withTiKVGlobalLock sync.RWMutex WithTiKV = flag.Bool("with-tikv", false, "run tests with TiKV cluster started. (not use the mock server)") diff --git a/store/tikv/mockstore/mocktikv/cluster_test.go b/store/mockstore/cluster_test.go similarity index 93% rename from store/tikv/mockstore/mocktikv/cluster_test.go rename to store/mockstore/cluster_test.go index 14fdb4106e3e4..2e34edad55266 100644 --- a/store/tikv/mockstore/mocktikv/cluster_test.go +++ b/store/mockstore/cluster_test.go @@ -11,7 +11,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package mocktikv_test +package mockstore import ( "bytes" @@ -75,7 +75,8 @@ func (s *testClusterSuite) TestClusterSplit(c *C) { c.Assert(err, IsNil) // Split Table into 10 regions. - cluster.SplitTable(tblID, 10) + tableStart := tablecodec.GenTableRecordPrefix(tblID) + cluster.SplitKeys(tableStart, tableStart.PrefixNext(), 10) // 10 table regions and first region and last region. regions := cluster.GetAllRegions() @@ -99,7 +100,8 @@ func (s *testClusterSuite) TestClusterSplit(c *C) { } c.Assert(allKeysMap, HasLen, 1000) - cluster.SplitIndex(tblID, idxID, 10) + indexStart := tablecodec.EncodeTableIndexPrefix(tblID, idxID) + cluster.SplitKeys(indexStart, indexStart.PrefixNext(), 10) allIndexMap := make(map[string]bool) indexPrefix := tablecodec.EncodeTableIndexPrefix(tblID, idxID) diff --git a/store/mockstore/mockcopr/executor_test.go b/store/mockstore/mockcopr/executor_test.go index c442b0cbac767..39dd945d23f89 100644 --- a/store/mockstore/mockcopr/executor_test.go +++ b/store/mockstore/mockcopr/executor_test.go @@ -15,6 +15,7 @@ package mockcopr_test import ( "context" + "testing" . "github.com/pingcap/check" "github.com/pingcap/errors" @@ -32,6 +33,11 @@ import ( "github.com/pingcap/tidb/util/testkit" ) +func TestT(t *testing.T) { + CustomVerboseFlag = true + TestingT(t) +} + var _ = Suite(&testExecutorSuite{}) type testExecutorSuite struct { diff --git a/store/mockstore/mocktikv/puppet.go b/store/mockstore/mocktikv/puppet.go deleted file mode 100644 index 36e3bc6d6201f..0000000000000 --- a/store/mockstore/mocktikv/puppet.go +++ /dev/null @@ -1,45 +0,0 @@ -// 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 mocktikv - -import "github.com/pingcap/tidb/store/tikv/mockstore/mocktikv" - -// Type alias to keep br working. Remove it after BR updates. -type ( - // MVCCStore is a mvcc key-value storage. - MVCCStore = mocktikv.MVCCStore - // Cluster simulates a TiKV cluster. It focuses on management and the change of - // meta data. A Cluster mainly includes following 3 kinds of meta data: - // 1) Region: A Region is a fragment of TiKV's data whose range is [start, end). - // The data of a Region is duplicated to multiple Peers and distributed in - // multiple Stores. - // 2) Peer: A Peer is a replica of a Region's data. All peers of a Region form - // a group, each group elects a Leader to provide services. - // 3) Store: A Store is a storage/service node. Try to think it as a TiKV server - // process. Only the store with request's Region's leader Peer could respond - // to client's request. - Cluster = mocktikv.Cluster -) - -// Variable alias to keep br working. Remove it after BR updates. -var ( - // MustNewMVCCStore is used for testing, use NewMVCCLevelDB instead. - MustNewMVCCStore = mocktikv.MustNewMVCCStore - // NewCluster creates an empty cluster. It needs to be bootstrapped before - // providing service. - NewCluster = mocktikv.NewCluster - // NewPDClient creates a mock pd.Client that uses local timestamp and meta data - // from a Cluster. - NewPDClient = mocktikv.NewPDClient -) diff --git a/store/mockstore/unistore/cophandler/analyze.go b/store/mockstore/unistore/cophandler/analyze.go index e4ba35614647f..4f309027edc4a 100644 --- a/store/mockstore/unistore/cophandler/analyze.go +++ b/store/mockstore/unistore/cophandler/analyze.go @@ -16,6 +16,7 @@ package cophandler import ( "bytes" "math" + "math/rand" "sort" "time" @@ -68,8 +69,10 @@ func handleCopAnalyzeRequest(dbReader *dbreader.DBReader, req *coprocessor.Reque resp, err = handleAnalyzeCommonHandleReq(dbReader, ranges, analyzeReq, req.StartTs) } else if analyzeReq.Tp == tipb.AnalyzeType_TypeColumn { resp, err = handleAnalyzeColumnsReq(dbReader, ranges, analyzeReq, req.StartTs) - } else { + } else if analyzeReq.Tp == tipb.AnalyzeType_TypeMixed { resp, err = handleAnalyzeMixedReq(dbReader, ranges, analyzeReq, req.StartTs) + } else { + resp, err = handleAnalyzeFullSamplingReq(dbReader, ranges, analyzeReq, req.StartTs) } if err != nil { resp = &coprocessor.Response{ @@ -364,6 +367,83 @@ func handleAnalyzeColumnsReq(dbReader *dbreader.DBReader, rans []kv.KeyRange, an return &coprocessor.Response{Data: data}, nil } +func handleAnalyzeFullSamplingReq( + dbReader *dbreader.DBReader, + rans []kv.KeyRange, + analyzeReq *tipb.AnalyzeReq, + startTS uint64, +) (*coprocessor.Response, error) { + sc := flagsToStatementContext(analyzeReq.Flags) + sc.TimeZone = time.FixedZone("UTC", int(analyzeReq.TimeZoneOffset)) + evalCtx := &evalContext{sc: sc} + columns := analyzeReq.ColReq.ColumnsInfo + evalCtx.setColumnInfo(columns) + if len(analyzeReq.ColReq.PrimaryColumnIds) > 0 { + evalCtx.primaryCols = analyzeReq.ColReq.PrimaryColumnIds + } + decoder, err := newRowDecoder(evalCtx.columnInfos, evalCtx.fieldTps, evalCtx.primaryCols, evalCtx.sc.TimeZone) + if err != nil { + return nil, err + } + e := &analyzeColumnsExec{ + reader: dbReader, + seekKey: rans[0].StartKey, + endKey: rans[0].EndKey, + ranges: rans, + curRan: 0, + startTS: startTS, + chk: chunk.NewChunkWithCapacity(evalCtx.fieldTps, 1), + decoder: decoder, + evalCtx: evalCtx, + } + e.fields = make([]*ast.ResultField, len(columns)) + for i := range e.fields { + rf := new(ast.ResultField) + rf.Column = new(model.ColumnInfo) + rf.Column.FieldType = types.FieldType{Tp: mysql.TypeBlob, Flen: mysql.MaxBlobWidth, Charset: charset.CharsetUTF8, Collate: charset.CollationUTF8} + e.fields[i] = rf + } + + numCols := len(columns) + collators := make([]collate.Collator, numCols) + fts := make([]*types.FieldType, numCols) + for i, col := range columns { + ft := fieldTypeFromPBColumn(col) + fts[i] = ft + if ft.EvalType() == types.ETString { + collators[i] = collate.GetCollator(ft.Collate) + } + } + colGroups := make([][]int64, 0, len(analyzeReq.ColReq.ColumnGroups)) + for _, group := range analyzeReq.ColReq.ColumnGroups { + colOffsets := make([]int64, len(group.ColumnOffsets)) + copy(colOffsets, group.ColumnOffsets) + colGroups = append(colGroups, colOffsets) + } + colReq := analyzeReq.ColReq + builder := &statistics.RowSampleBuilder{ + Sc: sc, + RecordSet: e, + ColsFieldType: fts, + Collators: collators, + ColGroups: colGroups, + MaxSampleSize: int(colReq.SampleSize), + MaxFMSketchSize: int(colReq.SketchSize), + Rng: rand.New(rand.NewSource(time.Now().UnixNano())), + } + collector, err := builder.Collect() + if err != nil { + return nil, err + } + colResp := &tipb.AnalyzeColumnsResp{} + colResp.RowCollector = collector.ToProto() + data, err := colResp.Marshal() + if err != nil { + return nil, err + } + return &coprocessor.Response{Data: data}, nil +} + // Fields implements the sqlexec.RecordSet Fields interface. func (e *analyzeColumnsExec) Fields() []*ast.ResultField { return e.fields diff --git a/store/mockstore/unistore/cophandler/cop_handler.go b/store/mockstore/unistore/cophandler/cop_handler.go index f3a73c99ed46a..fa3b5ec694aa5 100644 --- a/store/mockstore/unistore/cophandler/cop_handler.go +++ b/store/mockstore/unistore/cophandler/cop_handler.go @@ -203,30 +203,6 @@ func (e *evalContext) setColumnInfo(cols []*tipb.ColumnInfo) { } } -func (e *evalContext) fillColumnInfo(fieldTypes []*tipb.FieldType) { - e.columnInfos = make([]*tipb.ColumnInfo, 0, len(fieldTypes)) - e.fieldTps = make([]*types.FieldType, 0, len(fieldTypes)) - for i, pbType := range fieldTypes { - e.columnInfos = append(e.columnInfos, &tipb.ColumnInfo{ColumnId: int64(i), - Tp: pbType.Tp, Collation: pbType.Collate, ColumnLen: pbType.Flen, - Decimal: pbType.Decimal, Flag: int32(pbType.Flag)}) - // todo fill collate and charset field - e.fieldTps = append(e.fieldTps, &types.FieldType{Tp: byte(pbType.Tp), - Flag: uint(pbType.Flag), Flen: int(pbType.Flen), Decimal: int(pbType.Decimal)}) - } -} - -func (e *evalContext) fillColumnInfoFromTPs(fieldTypes []*types.FieldType) { - e.columnInfos = make([]*tipb.ColumnInfo, 0, len(fieldTypes)) - e.fieldTps = append(e.fieldTps, fieldTypes...) - for i, fieldType := range fieldTypes { - pbType := expression.ToPBFieldType(fieldType) - e.columnInfos = append(e.columnInfos, &tipb.ColumnInfo{ColumnId: int64(i), - Tp: pbType.Tp, Collation: pbType.Collate, ColumnLen: pbType.Flen, - Decimal: pbType.Decimal, Flag: int32(pbType.Flag)}) - } -} - func newRowDecoder(columnInfos []*tipb.ColumnInfo, fieldTps []*types.FieldType, primaryCols []int64, timeZone *time.Location) (*rowcodec.ChunkDecoder, error) { var ( pkCols []int64 diff --git a/store/mockstore/unistore/cophandler/mpp.go b/store/mockstore/unistore/cophandler/mpp.go index 1395ce21b2b7b..7f5dfdc9e8707 100644 --- a/store/mockstore/unistore/cophandler/mpp.go +++ b/store/mockstore/unistore/cophandler/mpp.go @@ -15,7 +15,6 @@ package cophandler import ( "context" - "fmt" "sync" "time" @@ -408,10 +407,6 @@ type ExchangerTunnel struct { ErrCh chan error } -func (tunnel *ExchangerTunnel) debugString() string { - return fmt.Sprintf("(%d->%d)", tunnel.sourceTask.TaskId, tunnel.targetTask.TaskId) -} - // RecvChunk recive tipb chunk func (tunnel *ExchangerTunnel) RecvChunk() (tipbChunk *tipb.Chunk, err error) { tipbChunk = <-tunnel.DataCh diff --git a/store/mockstore/unistore/lockstore/lockstore.go b/store/mockstore/unistore/lockstore/lockstore.go index ebe6bc3786fe8..d609e409fd5a9 100644 --- a/store/mockstore/unistore/lockstore/lockstore.go +++ b/store/mockstore/unistore/lockstore/lockstore.go @@ -36,8 +36,8 @@ type MemStore struct { } const ( - maxHeight = 16 - nodeHeadrSize = int(unsafe.Sizeof(nodeHeader{addr: 0, height: 0, keyLen: 0, valLen: 0})) + maxHeight = 16 + nodeHeaderSize = int(unsafe.Sizeof(nodeHeader{addr: 0, height: 0, keyLen: 0, valLen: 0})) ) type nodeHeader struct { @@ -69,10 +69,6 @@ func (e *entry) getValue(arena *arena) []byte { return nil } -func (n *node) nexts(level int) uint64 { - return *n.nextsAddr(level) -} - func (n *node) setNexts(level int, val uint64) { *n.nextsAddr(level) = val } @@ -90,12 +86,8 @@ func (n *node) setNextAddr(level int, addr arenaAddr) { atomic.StoreUint64(n.nextsAddr(level), uint64(addr)) } -func (n *node) entryLen() int { - return n.nodeLen() + int(n.keyLen) + int(n.valLen) -} - func (n *node) nodeLen() int { - return int(n.height)*8 + nodeHeadrSize + return int(n.height)*8 + nodeHeaderSize } func (n *node) getKey(a *arena) []byte { @@ -152,7 +144,7 @@ func (n *node) getNextNode(arena *arena, level int) *node { if addr == nullArenaAddr { return nil } - data := arena.get(addr, nodeHeadrSize) + data := arena.get(addr, nodeHeaderSize) return (*node)(unsafe.Pointer(&data[0])) } @@ -162,7 +154,7 @@ func (ls *MemStore) getNext(n *node, level int) (e entry) { return } arena := ls.getArena() - data := arena.get(addr, nodeHeadrSize) + data := arena.get(addr, nodeHeaderSize) e.node = (*node)(unsafe.Pointer(&data[0])) e.key = e.node.getKey(arena) return e @@ -177,7 +169,7 @@ func (ls *MemStore) findGreater(key []byte, allowEqual bool) (entry, bool) { addr := prev.getNextAddr(level) if addr != nullArenaAddr { arena := ls.getArena() - data := arena.get(addr, nodeHeadrSize) + data := arena.get(addr, nodeHeaderSize) next.node = (*node)(unsafe.Pointer(&data[0])) next.key = next.node.getKey(arena) cmp := bytes.Compare(next.key, key) @@ -247,7 +239,7 @@ func (ls *MemStore) findSpliceForLevel(arena *arena, key []byte, before *node, l if nextAddr == nullArenaAddr { return before, nil, false } - data := arena.get(nextAddr, nodeHeadrSize) + data := arena.get(nextAddr, nodeHeaderSize) next := (*node)(unsafe.Pointer(&data[0])) nextKey := next.getKey(arena) cmp := bytes.Compare(nextKey, key) @@ -282,7 +274,7 @@ func (ls *MemStore) findLast() entry { } func (ls *MemStore) getNode(arena *arena, addr arenaAddr) *node { - data := arena.get(addr, nodeHeadrSize) + data := arena.get(addr, nodeHeaderSize) return (*node)(unsafe.Pointer(&data[0])) } @@ -362,7 +354,7 @@ func (ls *MemStore) replace(key, v []byte, hint *Hint, old *node) { func (ls *MemStore) newNode(arena *arena, key []byte, v []byte, height int) *node { // The base level is already allocated in the node struct. - nodeSize := nodeHeadrSize + height*8 + len(key) + len(v) + nodeSize := nodeHeaderSize + height*8 + len(key) + len(v) addr := arena.alloc(nodeSize) if addr == nullArenaAddr { arena = arena.grow() diff --git a/store/mockstore/unistore/tikv/mvcc_test.go b/store/mockstore/unistore/tikv/mvcc_test.go index 323e50aea9f89..8819fe21ab398 100644 --- a/store/mockstore/unistore/tikv/mvcc_test.go +++ b/store/mockstore/unistore/tikv/mvcc_test.go @@ -390,12 +390,6 @@ func MustPrewritePessimisticPutErr(pk []byte, key []byte, value []byte, startTs store.c.Assert(err, NotNil) } -func MustPrewritePessimisticErr(pk []byte, key []byte, value []byte, startTs uint64, lockTTL uint64, - isPessimisticLock []bool, forUpdateTs uint64, store *TestStore) { - err := PrewritePessimistic(pk, key, value, startTs, lockTTL, isPessimisticLock, forUpdateTs, store) - store.c.Assert(err, NotNil) -} - func MustCommitKeyPut(key, val []byte, startTs, commitTs uint64, store *TestStore) { err := store.MvccStore.Commit(store.newReqCtx(), [][]byte{key}, startTs, commitTs) store.c.Assert(err, IsNil) diff --git a/store/mockstore/unistore/tikv/region.go b/store/mockstore/unistore/tikv/region.go index 21ea7ab5e40da..372b39f6dd667 100644 --- a/store/mockstore/unistore/tikv/region.go +++ b/store/mockstore/unistore/tikv/region.go @@ -191,18 +191,10 @@ func (ri *regionCtx) decodeRawEndKey() []byte { return rawKey } -func (ri *regionCtx) lessThanStartKey(key []byte) bool { - return bytes.Compare(key, ri.rawStartKey) < 0 -} - func (ri *regionCtx) greaterEqualEndKey(key []byte) bool { return len(ri.rawEndKey) > 0 && bytes.Compare(key, ri.rawEndKey) >= 0 } -func (ri *regionCtx) greaterThanEndKey(key []byte) bool { - return len(ri.rawEndKey) > 0 && bytes.Compare(key, ri.rawEndKey) > 0 -} - func newPeerMeta(peerID, storeID uint64) *metapb.Peer { return &metapb.Peer{ Id: peerID, diff --git a/store/mockstore/unistore/tikv/server.go b/store/mockstore/unistore/tikv/server.go index a8434d42e9c27..66abc424084b1 100644 --- a/store/mockstore/unistore/tikv/server.go +++ b/store/mockstore/unistore/tikv/server.go @@ -59,19 +59,6 @@ func NewServer(rm RegionManager, store *MVCCStore, innerServer InnerServer) *Ser } } -const requestMaxSize = 6 * 1024 * 1024 - -func (svr *Server) checkRequestSize(size int) *errorpb.Error { - // TiKV has a limitation on raft log size. - // mocktikv has no raft inside, so we check the request's size instead. - if size >= requestMaxSize { - return &errorpb.Error{ - RaftEntryTooLarge: &errorpb.RaftEntryTooLarge{}, - } - } - return nil -} - // Stop stops the server. func (svr *Server) Stop() { atomic.StoreInt32(&svr.stopped, 1) @@ -629,28 +616,6 @@ func (svr *Server) BatchCoprocessor(req *coprocessor.BatchRequest, batchCopServe return nil } -func (mrm *MockRegionManager) getMPPTaskHandler(rpcClient client.Client, meta *mpp.TaskMeta, createdIfNotExist bool, storeID uint64) (*cophandler.MPPTaskHandler, bool, error) { - set := mrm.getMPPTaskSet(storeID) - if set == nil { - return nil, false, errors.New("cannot find mpp task set for store") - } - set.mu.Lock() - defer set.mu.Unlock() - if handler, ok := set.taskHandlers[meta.TaskId]; ok { - return handler, false, nil - } - if createdIfNotExist { - handler := &cophandler.MPPTaskHandler{ - TunnelSet: make(map[int64]*cophandler.ExchangerTunnel), - Meta: meta, - RPCClient: rpcClient, - } - set.taskHandlers[meta.TaskId] = handler - return handler, true, nil - } - return nil, false, nil -} - func (mrm *MockRegionManager) removeMPPTaskHandler(taskID int64, storeID uint64) error { set := mrm.getMPPTaskSet(storeID) if set == nil { diff --git a/store/tikv/2pc.go b/store/tikv/2pc.go index e39eabb71399c..6d738e2761d19 100644 --- a/store/tikv/2pc.go +++ b/store/tikv/2pc.go @@ -924,7 +924,7 @@ func (c *twoPhaseCommitter) cleanup(ctx context.Context) { logutil.Logger(ctx).Info("2PC cleanup failed", zap.Error(err), zap.Uint64("txnStartTS", c.startTS), zap.Bool("isPessimistic", c.isPessimistic), zap.Bool("isOnePC", c.isOnePC())) } else { - logutil.Logger(ctx).Info("2PC clean up done", + logutil.Logger(ctx).Debug("2PC clean up done", zap.Uint64("txnStartTS", c.startTS), zap.Bool("isPessimistic", c.isPessimistic), zap.Bool("isOnePC", c.isOnePC())) } @@ -1116,13 +1116,6 @@ func (c *twoPhaseCommitter) execute(ctx context.Context) (err error) { logutil.SetTag(ctx, "commitTs", commitTS) } - if c.sessionID > 0 { - failpoint.Inject("beforeSchemaCheck", func() { - c.ttlManager.close() - failpoint.Return() - }) - } - if !c.isAsyncCommit() { tryAmend := c.isPessimistic && c.sessionID > 0 && c.txn.schemaAmender != nil if !tryAmend { @@ -1146,7 +1139,7 @@ func (c *twoPhaseCommitter) execute(ctx context.Context) (err error) { if err != nil { logutil.Logger(ctx).Info("schema check after amend failed, it means the schema version changed again", zap.Uint64("startTS", c.startTS), - zap.Uint64("amendTS", c.commitTS), + zap.Uint64("amendTS", commitTS), zap.Int64("amendedSchemaVersion", relatedSchemaChange.LatestInfoSchema.SchemaMetaVersion()), zap.Uint64("newCommitTS", newCommitTS)) return errors.Trace(err) diff --git a/store/tikv/backoff.go b/store/tikv/backoff.go index cee31f7fdfa01..45ee15bf16447 100644 --- a/store/tikv/backoff.go +++ b/store/tikv/backoff.go @@ -25,7 +25,6 @@ import ( "github.com/opentracing/opentracing-go" "github.com/pingcap/errors" "github.com/pingcap/log" - tidbkv "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/logutil" "github.com/pingcap/tidb/store/tikv/metrics" @@ -134,7 +133,7 @@ const ( boMaxTsNotSynced ) -func (t BackoffType) createFn(vars *tidbkv.Variables) func(context.Context, int) int { +func (t BackoffType) createFn(vars *kv.Variables) func(context.Context, int) int { if vars.Hook != nil { vars.Hook(t.String(), vars) } @@ -252,7 +251,7 @@ type Backoffer struct { totalSleep int errors []error types []fmt.Stringer - vars *tidbkv.Variables + vars *kv.Variables noop bool backoffSleepMS map[BackoffType]int @@ -269,12 +268,12 @@ func NewBackoffer(ctx context.Context, maxSleep int) *Backoffer { return &Backoffer{ ctx: ctx, maxSleep: maxSleep, - vars: tidbkv.DefaultVars, + vars: kv.DefaultVars, } } -// NewBackofferWithVars creates a Backoffer with maximum sleep time(in ms) and tidbkv.Variables. -func NewBackofferWithVars(ctx context.Context, maxSleep int, vars *tidbkv.Variables) *Backoffer { +// NewBackofferWithVars creates a Backoffer with maximum sleep time(in ms) and kv.Variables. +func NewBackofferWithVars(ctx context.Context, maxSleep int, vars *kv.Variables) *Backoffer { return NewBackoffer(ctx, maxSleep).withVars(vars) } @@ -283,8 +282,8 @@ func NewNoopBackoff(ctx context.Context) *Backoffer { return &Backoffer{ctx: ctx, noop: true} } -// withVars sets the tidbkv.Variables to the Backoffer and return it. -func (b *Backoffer) withVars(vars *tidbkv.Variables) *Backoffer { +// withVars sets the kv.Variables to the Backoffer and return it. +func (b *Backoffer) withVars(vars *kv.Variables) *Backoffer { if vars != nil { b.vars = vars } @@ -415,7 +414,7 @@ func (b *Backoffer) Fork() (*Backoffer, context.CancelFunc) { } // GetVars returns the binded vars. -func (b *Backoffer) GetVars() *tidbkv.Variables { +func (b *Backoffer) GetVars() *kv.Variables { return b.vars } diff --git a/store/tikv/config/config.go b/store/tikv/config/config.go index 04ec5c7178f04..a8337cdcedf8c 100644 --- a/store/tikv/config/config.go +++ b/store/tikv/config/config.go @@ -22,6 +22,7 @@ import ( "sync/atomic" "github.com/pingcap/errors" + "github.com/pingcap/failpoint" "github.com/pingcap/tidb/store/tikv/logutil" "go.uber.org/zap" ) @@ -50,6 +51,7 @@ type Config struct { OpenTracingEnable bool Path string EnableForwarding bool + TxnScope string } // DefaultConfig returns the default configuration. @@ -65,6 +67,7 @@ func DefaultConfig() Config { OpenTracingEnable: false, Path: "", EnableForwarding: false, + TxnScope: "", } } @@ -133,6 +136,26 @@ func UpdateGlobal(f func(conf *Config)) func() { return restore } +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) + }) + + if kvcfg := GetGlobalConfig(); kvcfg != nil && len(kvcfg.TxnScope) > 0 { + return false, kvcfg.TxnScope + } + return true, globalTxnScope +} + // ParsePath parses this path. // Path example: tikv://etcd-node1:port,etcd-node2:port?cluster=1&disableGC=false func ParsePath(path string) (etcdAddrs []string, disableGC bool, err error) { diff --git a/store/tikv/config/config_test.go b/store/tikv/config/config_test.go index c8fc674fcd973..f79f2d09c22c8 100644 --- a/store/tikv/config/config_test.go +++ b/store/tikv/config/config_test.go @@ -15,6 +15,7 @@ package config import ( . "github.com/pingcap/check" + "github.com/pingcap/failpoint" ) var _ = SerialSuites(&testConfigSuite{}) @@ -31,3 +32,22 @@ func (s *testConfigSuite) TestParsePath(c *C) { c.Assert(err, IsNil) c.Assert(disableGC, IsTrue) } + +func (s *testConfigSuite) TestTxnScopeValue(c *C) { + + failpoint.Enable("github.com/pingcap/tidb/store/tikv/config/injectTxnScope", `return("bj")`) + isGlobal, v := GetTxnScopeFromConfig() + c.Assert(isGlobal, IsFalse) + c.Assert(v, Equals, "bj") + failpoint.Disable("github.com/pingcap/tidb/store/tikv/config/injectTxnScope") + failpoint.Enable("github.com/pingcap/tidb/store/tikv/config/injectTxnScope", `return("")`) + isGlobal, v = GetTxnScopeFromConfig() + c.Assert(isGlobal, IsTrue) + c.Assert(v, Equals, "global") + failpoint.Disable("github.com/pingcap/tidb/store/tikv/config/injectTxnScope") + failpoint.Enable("github.com/pingcap/tidb/store/tikv/config/injectTxnScope", `return("global")`) + isGlobal, v = GetTxnScopeFromConfig() + c.Assert(isGlobal, IsFalse) + c.Assert(v, Equals, "global") + failpoint.Disable("github.com/pingcap/tidb/store/tikv/config/injectTxnScope") +} diff --git a/store/tikv/kv.go b/store/tikv/kv.go index d95ab34d1e0b0..4e0bea5213b32 100644 --- a/store/tikv/kv.go +++ b/store/tikv/kv.go @@ -347,3 +347,6 @@ func (s *KVStore) SetTiKVClient(client Client) { func (s *KVStore) GetTiKVClient() (client Client) { return s.client } + +// Variables defines the variables used by TiKV storage. +type Variables = kv.Variables diff --git a/store/tikv/kv/variables.go b/store/tikv/kv/variables.go new file mode 100644 index 0000000000000..b722023bcae08 --- /dev/null +++ b/store/tikv/kv/variables.go @@ -0,0 +1,50 @@ +// 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 kv + +// Variables defines the variables used by KV storage. +type Variables struct { + // BackoffLockFast specifies the LockFast backoff base duration in milliseconds. + BackoffLockFast int + + // BackOffWeight specifies the weight of the max back off time duration. + BackOffWeight int + + // Hook is used for test to verify the variable take effect. + Hook func(name string, vars *Variables) + + // Pointer to SessionVars.Killed + // Killed is a flag to indicate that this query is killed. + Killed *uint32 +} + +// NewVariables create a new Variables instance with default values. +func NewVariables(killed *uint32) *Variables { + return &Variables{ + BackoffLockFast: DefBackoffLockFast, + BackOffWeight: DefBackOffWeight, + Killed: killed, + } +} + +var ignoreKill uint32 + +// DefaultVars is the default variables instance. +var DefaultVars = NewVariables(&ignoreKill) + +// Default values +const ( + DefBackoffLockFast = 100 + DefBackOffWeight = 2 +) diff --git a/store/tikv/mockstore/cluster/cluster.go b/store/tikv/mockstore/cluster/cluster.go index d1fda9483d4da..ef6f20953715f 100644 --- a/store/tikv/mockstore/cluster/cluster.go +++ b/store/tikv/mockstore/cluster/cluster.go @@ -35,10 +35,6 @@ type Cluster interface { Split(regionID, newRegionID uint64, key []byte, peerIDs []uint64, leaderPeerID uint64) // SplitRaw splits a Region at the key (not encoded) and creates new Region. SplitRaw(regionID, newRegionID uint64, rawKey []byte, peerIDs []uint64, leaderPeerID uint64) *metapb.Region - // SplitTable evenly splits the data in table into count regions. - SplitTable(tableID int64, count int) - // SplitIndex evenly splits the data in index into count regions. - SplitIndex(tableID, indexID int64, count int) // SplitKeys evenly splits the start, end key into "count" regions. SplitKeys(start, end []byte, count int) // AddStore adds a new Store to the cluster. diff --git a/store/tikv/mockstore/mocktikv/cluster.go b/store/tikv/mockstore/mocktikv/cluster.go index abeade418cd69..5d962564568f3 100644 --- a/store/tikv/mockstore/mocktikv/cluster.go +++ b/store/tikv/mockstore/mocktikv/cluster.go @@ -24,7 +24,6 @@ import ( "github.com/golang/protobuf/proto" "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/kvproto/pkg/metapb" - "github.com/pingcap/tidb/tablecodec" pd "github.com/tikv/pd/client" ) @@ -410,22 +409,6 @@ func (c *Cluster) Merge(regionID1, regionID2 uint64) { delete(c.regions, regionID2) } -// SplitTable evenly splits the data in table into count regions. -// Only works for single store. -func (c *Cluster) SplitTable(tableID int64, count int) { - tableStart := tablecodec.GenTableRecordPrefix(tableID) - tableEnd := tableStart.PrefixNext() - c.splitRange(c.mvccStore, NewMvccKey(tableStart), NewMvccKey(tableEnd), count) -} - -// SplitIndex evenly splits the data in index into count regions. -// Only works for single store. -func (c *Cluster) SplitIndex(tableID, indexID int64, count int) { - indexStart := tablecodec.EncodeTableIndexPrefix(tableID, indexID) - indexEnd := indexStart.PrefixNext() - c.splitRange(c.mvccStore, NewMvccKey(indexStart), NewMvccKey(indexEnd), count) -} - // SplitKeys evenly splits the start, end key into "count" regions. // Only works for single store. func (c *Cluster) SplitKeys(start, end []byte, count int) { diff --git a/store/tikv/oracle/oracle.go b/store/tikv/oracle/oracle.go index 78f7ab0164ea3..b642e96da30b6 100644 --- a/store/tikv/oracle/oracle.go +++ b/store/tikv/oracle/oracle.go @@ -18,7 +18,7 @@ import ( "time" "github.com/pingcap/failpoint" - "github.com/pingcap/tidb/config" + "github.com/pingcap/tidb/store/tikv/config" "github.com/pingcap/tidb/store/tikv/logutil" "go.uber.org/zap" ) diff --git a/store/tikv/oracle/oracles/export_test.go b/store/tikv/oracle/oracles/export_test.go index 50d3bf076230c..a8911f920c585 100644 --- a/store/tikv/oracle/oracles/export_test.go +++ b/store/tikv/oracle/oracles/export_test.go @@ -33,14 +33,6 @@ func SetOracleHookCurrentTime(oc oracle.Oracle, t time.Time) { } } -// ClearOracleHook exports localOracle's clear hook method -func ClearOracleHook(oc oracle.Oracle) { - switch o := oc.(type) { - case *localOracle: - o.hook = nil - } -} - // NewEmptyPDOracle exports pdOracle struct to test func NewEmptyPDOracle() oracle.Oracle { return &pdOracle{} diff --git a/store/tikv/snapshot.go b/store/tikv/snapshot.go index 904b8653093ac..83b3ce2e0c3aa 100644 --- a/store/tikv/snapshot.go +++ b/store/tikv/snapshot.go @@ -32,11 +32,9 @@ import ( "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/logutil" "github.com/pingcap/tidb/store/tikv/metrics" - "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/store/tikv/tikvrpc" "github.com/pingcap/tidb/store/tikv/unionstore" "github.com/pingcap/tidb/store/tikv/util" - "github.com/pingcap/tidb/util/execdetails" "go.uber.org/zap" ) @@ -55,7 +53,7 @@ type KVSnapshot struct { notFillCache bool syncLog bool keyOnly bool - vars *tidbkv.Variables + vars *kv.Variables replicaReadSeed uint32 resolvedLocks *util.TSSet @@ -93,7 +91,7 @@ func newTiKVSnapshot(store *KVStore, ts uint64, replicaReadSeed uint32) *KVSnaps store: store, version: ts, priority: pb.CommandPri_Normal, - vars: tidbkv.DefaultVars, + vars: kv.DefaultVars, replicaReadSeed: replicaReadSeed, resolvedLocks: util.NewTSSet(5), } @@ -415,14 +413,6 @@ func (s *KVSnapshot) get(ctx context.Context, bo *Backoffer, k []byte) ([]byte, cli := NewClientHelper(s.store, s.resolvedLocks) - // Secondary locks or async commit locks cannot be ignored when getting using the max version. - // So we concurrently get a TS from PD and use it in retries to avoid unnecessary blocking. - var tsFuture oracle.Future - if s.version == maxTimestamp { - tsFuture = s.store.oracle.GetTimestampAsync(ctx, &oracle.Option{TxnScope: s.txnScope}) - } - failpoint.Inject("snapshotGetTSAsync", nil) - isStaleness := false var matchStoreLabels []*metapb.StoreLabel s.mu.RLock() @@ -451,7 +441,10 @@ func (s *KVSnapshot) get(ctx context.Context, bo *Backoffer, k []byte) ([]byte, if len(matchStoreLabels) > 0 { ops = append(ops, WithMatchLabels(matchStoreLabels)) } + + var firstLock *Lock for { + failpoint.Inject("beforeSendPointGet", nil) loc, err := s.store.regionCache.LocateKey(bo, k) if err != nil { return nil, errors.Trace(err) @@ -484,25 +477,17 @@ func (s *KVSnapshot) get(ctx context.Context, bo *Backoffer, k []byte) ([]byte, if err != nil { return nil, errors.Trace(err) } - - snapVer := s.version - if s.version == maxTimestamp { - newTS, err := tsFuture.Wait() - if err != nil { - return nil, errors.Trace(err) - } - s.version = newTS - req.Req.(*pb.GetRequest).Version = newTS - // skip lock resolving and backoff if the lock does not block the read - if newTS < lock.TxnID || newTS < lock.MinCommitTS { - continue - } + if firstLock == nil { + firstLock = lock + } else if s.version == maxTimestamp && firstLock.TxnID != lock.TxnID { + // If it is an autocommit point get, it needs to be blocked only + // by the first lock it meets. During retries, if the encountered + // lock is different from the first one, we can omit it. + cli.resolvedLocks.Put(lock.TxnID) + continue } - // Use the original snapshot version to resolve locks so we can use MaxUint64 - // as the callerStartTS if it's an auto-commit point get. This could save us - // one write at TiKV by not pushing forward the minCommitTS. - msBeforeExpired, err := cli.ResolveLocks(bo, snapVer, []*Lock{lock}) + msBeforeExpired, err := cli.ResolveLocks(bo, s.version, []*Lock{lock}) if err != nil { return nil, errors.Trace(err) } @@ -525,10 +510,10 @@ func (s *KVSnapshot) mergeExecDetail(detail *pb.ExecDetailsV2) { return } if s.mu.stats.scanDetail == nil { - s.mu.stats.scanDetail = &execdetails.ScanDetail{} + s.mu.stats.scanDetail = &util.ScanDetail{} } if s.mu.stats.timeDetail == nil { - s.mu.stats.timeDetail = &execdetails.TimeDetail{} + s.mu.stats.timeDetail = &util.TimeDetail{} } s.mu.stats.scanDetail.MergeFromScanDetailV2(detail.ScanDetailV2) s.mu.stats.timeDetail.MergeFromTimeDetail(detail.TimeDetail) @@ -706,17 +691,12 @@ type SnapshotRuntimeStats struct { rpcStats RegionRequestRuntimeStats backoffSleepMS map[BackoffType]int backoffTimes map[BackoffType]int - scanDetail *execdetails.ScanDetail - timeDetail *execdetails.TimeDetail -} - -// Tp implements the RuntimeStats interface. -func (rs *SnapshotRuntimeStats) Tp() int { - return execdetails.TpSnapshotRuntimeStats + scanDetail *util.ScanDetail + timeDetail *util.TimeDetail } // Clone implements the RuntimeStats interface. -func (rs *SnapshotRuntimeStats) Clone() execdetails.RuntimeStats { +func (rs *SnapshotRuntimeStats) Clone() *SnapshotRuntimeStats { newRs := SnapshotRuntimeStats{rpcStats: NewRegionRequestRuntimeStats()} if rs.rpcStats.Stats != nil { for k, v := range rs.rpcStats.Stats { @@ -737,28 +717,24 @@ func (rs *SnapshotRuntimeStats) Clone() execdetails.RuntimeStats { } // Merge implements the RuntimeStats interface. -func (rs *SnapshotRuntimeStats) Merge(other execdetails.RuntimeStats) { - tmp, ok := other.(*SnapshotRuntimeStats) - if !ok { - return - } - if tmp.rpcStats.Stats != nil { +func (rs *SnapshotRuntimeStats) Merge(other *SnapshotRuntimeStats) { + if other.rpcStats.Stats != nil { if rs.rpcStats.Stats == nil { - rs.rpcStats.Stats = make(map[tikvrpc.CmdType]*RPCRuntimeStats, len(tmp.rpcStats.Stats)) + rs.rpcStats.Stats = make(map[tikvrpc.CmdType]*RPCRuntimeStats, len(other.rpcStats.Stats)) } - rs.rpcStats.Merge(tmp.rpcStats) + rs.rpcStats.Merge(other.rpcStats) } - if len(tmp.backoffSleepMS) > 0 { + if len(other.backoffSleepMS) > 0 { if rs.backoffSleepMS == nil { rs.backoffSleepMS = make(map[BackoffType]int) } if rs.backoffTimes == nil { rs.backoffTimes = make(map[BackoffType]int) } - for k, v := range tmp.backoffSleepMS { + for k, v := range other.backoffSleepMS { rs.backoffSleepMS[k] += v } - for k, v := range tmp.backoffTimes { + for k, v := range other.backoffTimes { rs.backoffTimes[k] += v } } diff --git a/store/tikv/tests/snapshot_fail_test.go b/store/tikv/tests/snapshot_fail_test.go index 75d716e35ac5c..654eaaa53df8c 100644 --- a/store/tikv/tests/snapshot_fail_test.go +++ b/store/tikv/tests/snapshot_fail_test.go @@ -140,38 +140,65 @@ func (s *testSnapshotFailSuite) TestScanResponseKeyError(c *C) { c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/mockstore/unistore/rpcScanResult"), IsNil) } -func (s *testSnapshotFailSuite) TestRetryPointGetWithTS(c *C) { +func (s *testSnapshotFailSuite) TestRetryMaxTsPointGetSkipLock(c *C) { defer s.cleanup(c) + // Prewrite k1 and k2 with async commit but don't commit them + txn, err := s.store.Begin() + c.Assert(err, IsNil) + err = txn.Set([]byte("k1"), []byte("v1")) + c.Assert(err, IsNil) + err = txn.Set([]byte("k2"), []byte("v2")) + c.Assert(err, IsNil) + txn.SetOption(kv.EnableAsyncCommit, true) + + c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/asyncCommitDoNothing", "return"), IsNil) + c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/twoPCShortLockTTL", "return"), IsNil) + committer, err := txn.NewCommitter(1) + c.Assert(err, IsNil) + err = committer.Execute(context.Background()) + c.Assert(err, IsNil) + c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/twoPCShortLockTTL"), IsNil) + snapshot := s.store.GetSnapshot(math.MaxUint64) - c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/snapshotGetTSAsync", `pause`), IsNil) - ch := make(chan error) + getCh := make(chan []byte) go func() { - _, err := snapshot.Get(context.Background(), []byte("k4")) - ch <- err + // Sleep a while to make the TTL of the first txn expire, then we make sure we resolve lock by this get + time.Sleep(200 * time.Millisecond) + c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/beforeSendPointGet", "1*off->pause"), IsNil) + res, err := snapshot.Get(context.Background(), []byte("k2")) + c.Assert(err, IsNil) + getCh <- res }() + // The get should be blocked by the failpoint. But the lock should have been resolved. + select { + case res := <-getCh: + c.Errorf("too early %s", string(res)) + case <-time.After(1 * time.Second): + } - txn, err := s.store.Begin() - c.Assert(err, IsNil) - err = txn.Set([]byte("k4"), []byte("v4")) + // Prewrite k1 and k2 again without committing them + txn, err = s.store.Begin() c.Assert(err, IsNil) txn.SetOption(kv.EnableAsyncCommit, true) - txn.SetOption(kv.Enable1PC, false) - txn.SetOption(kv.GuaranteeLinearizability, false) - // Prewrite an async-commit lock and do not commit it. - c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/asyncCommitDoNothing", `return`), IsNil) - committer, err := txn.NewCommitter(1) + err = txn.Set([]byte("k1"), []byte("v3")) + c.Assert(err, IsNil) + err = txn.Set([]byte("k2"), []byte("v4")) + c.Assert(err, IsNil) + committer, err = txn.NewCommitter(1) c.Assert(err, IsNil) - // Sets its minCommitTS to one second later, so the lock will be ignored by point get. - committer.SetMinCommitTS(committer.GetStartTS() + (1000 << 18)) err = committer.Execute(context.Background()) c.Assert(err, IsNil) - c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/snapshotGetTSAsync"), IsNil) - err = <-ch - c.Assert(err, ErrorMatches, ".*key not exist") + c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/beforeSendPointGet"), IsNil) - c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/asyncCommitDoNothing"), IsNil) + // After disabling the failpoint, the get request should bypass the new locks and read the old result + select { + case res := <-getCh: + c.Assert(res, DeepEquals, []byte("v2")) + case <-time.After(1 * time.Second): + c.Errorf("get timeout") + } } func (s *testSnapshotFailSuite) TestRetryPointGetResolveTS(c *C) { diff --git a/store/tikv/tests/split_test.go b/store/tikv/tests/split_test.go index 43ebd9a21be77..8c6c2a5225d9e 100644 --- a/store/tikv/tests/split_test.go +++ b/store/tikv/tests/split_test.go @@ -121,12 +121,6 @@ type mockPDClient struct { stop bool } -func (c *mockPDClient) enable() { - c.Lock() - defer c.Unlock() - c.stop = false -} - func (c *mockPDClient) disable() { c.Lock() defer c.Unlock() diff --git a/store/tikv/txn.go b/store/tikv/txn.go index fb3e4d17b2ef9..202e38c309f83 100644 --- a/store/tikv/txn.go +++ b/store/tikv/txn.go @@ -59,7 +59,7 @@ type KVTxn struct { commitTS uint64 mu sync.Mutex // For thread-safe LockKeys function. setCnt int64 - vars *tidbkv.Variables + vars *kv.Variables committer *twoPhaseCommitter lockedCnt int @@ -94,7 +94,7 @@ func newTiKVTxnWithStartTS(store *KVStore, txnScope string, startTS uint64, repl startTS: startTS, startTime: time.Now(), valid: true, - vars: tidbkv.DefaultVars, + vars: kv.DefaultVars, } newTiKVTxn.SetOption(kv.TxnScope, txnScope) return newTiKVTxn, nil @@ -113,7 +113,7 @@ func newTiKVTxnWithExactStaleness(store *KVStore, txnScope string, prevSec uint6 var SetSuccess = false // SetVars sets variables to the transaction. -func (txn *KVTxn) SetVars(vars *tidbkv.Variables) { +func (txn *KVTxn) SetVars(vars *kv.Variables) { txn.vars = vars txn.snapshot.vars = vars failpoint.Inject("probeSetVars", func(val failpoint.Value) { @@ -124,7 +124,7 @@ func (txn *KVTxn) SetVars(vars *tidbkv.Variables) { } // GetVars gets variables from the transaction. -func (txn *KVTxn) GetVars() *tidbkv.Variables { +func (txn *KVTxn) GetVars() *kv.Variables { return txn.vars } @@ -654,6 +654,9 @@ func (txn *KVTxn) GetSnapshot() *KVSnapshot { // SetBinlogExecutor sets the method to perform binlong synchronization. func (txn *KVTxn) SetBinlogExecutor(binlog BinlogExecutor) { txn.binlog = binlog + if txn.committer != nil { + txn.committer.binlog = binlog + } } // GetClusterID returns store's cluster id. diff --git a/store/tikv/unionstore/memdb_test.go b/store/tikv/unionstore/memdb_test.go index 13a940f144678..a24c45857f799 100644 --- a/store/tikv/unionstore/memdb_test.go +++ b/store/tikv/unionstore/memdb_test.go @@ -25,7 +25,7 @@ import ( . "github.com/pingcap/check" leveldb "github.com/pingcap/goleveldb/leveldb/memdb" "github.com/pingcap/tidb/store/tikv/kv" - "github.com/pingcap/tidb/util/testleak" + "github.com/pingcap/tidb/store/tikv/util/testleak" ) type KeyFlags = kv.KeyFlags diff --git a/store/tikv/unionstore/union_store_test.go b/store/tikv/unionstore/union_store_test.go index 67bf1361f191b..931514af63c83 100644 --- a/store/tikv/unionstore/union_store_test.go +++ b/store/tikv/unionstore/union_store_test.go @@ -18,7 +18,7 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/util/testleak" + "github.com/pingcap/tidb/store/tikv/util/testleak" ) var _ = Suite(&testUnionStoreSuite{}) diff --git a/store/tikv/util/execdetails.go b/store/tikv/util/execdetails.go index ee84a88fc6172..eeaaf92da6b27 100644 --- a/store/tikv/util/execdetails.go +++ b/store/tikv/util/execdetails.go @@ -14,10 +14,15 @@ package util import ( + "bytes" "fmt" "math" + "strconv" "sync" + "sync/atomic" "time" + + "github.com/pingcap/kvproto/pkg/kvrpcpb" ) type commitDetailCtxKeyType struct{} @@ -179,3 +184,119 @@ func getUnit(d time.Duration) time.Duration { } return time.Nanosecond } + +// ScanDetail contains coprocessor scan detail information. +type ScanDetail struct { + // TotalKeys is the approximate number of MVCC keys meet during scanning. It includes + // deleted versions, but does not include RocksDB tombstone keys. + TotalKeys int64 + // ProcessedKeys is the number of user keys scanned from the storage. + // It does not include deleted version or RocksDB tombstone keys. + // For Coprocessor requests, it includes keys that has been filtered out by Selection. + ProcessedKeys int64 + // RocksdbDeleteSkippedCount is the total number of deletes and single deletes skipped over during + // iteration, i.e. how many RocksDB tombstones are skipped. + RocksdbDeleteSkippedCount uint64 + // RocksdbKeySkippedCount it the total number of internal keys skipped over during iteration. + RocksdbKeySkippedCount uint64 + // RocksdbBlockCacheHitCount is the total number of RocksDB block cache hits. + RocksdbBlockCacheHitCount uint64 + // RocksdbBlockReadCount is the total number of block reads (with IO). + RocksdbBlockReadCount uint64 + // RocksdbBlockReadByte is the total number of bytes from block reads. + RocksdbBlockReadByte uint64 +} + +// Merge merges scan detail execution details into self. +func (sd *ScanDetail) Merge(scanDetail *ScanDetail) { + atomic.AddInt64(&sd.TotalKeys, scanDetail.TotalKeys) + atomic.AddInt64(&sd.ProcessedKeys, scanDetail.ProcessedKeys) + atomic.AddUint64(&sd.RocksdbDeleteSkippedCount, scanDetail.RocksdbDeleteSkippedCount) + atomic.AddUint64(&sd.RocksdbKeySkippedCount, scanDetail.RocksdbKeySkippedCount) + atomic.AddUint64(&sd.RocksdbBlockCacheHitCount, scanDetail.RocksdbBlockCacheHitCount) + atomic.AddUint64(&sd.RocksdbBlockReadCount, scanDetail.RocksdbBlockReadCount) + atomic.AddUint64(&sd.RocksdbBlockReadByte, scanDetail.RocksdbBlockReadByte) +} + +var zeroScanDetail = ScanDetail{} + +// String implements the fmt.Stringer interface. +func (sd *ScanDetail) String() string { + if sd == nil || *sd == zeroScanDetail { + return "" + } + buf := bytes.NewBuffer(make([]byte, 0, 16)) + buf.WriteString("scan_detail: {") + buf.WriteString("total_process_keys: ") + buf.WriteString(strconv.FormatInt(sd.ProcessedKeys, 10)) + buf.WriteString(", total_keys: ") + buf.WriteString(strconv.FormatInt(sd.TotalKeys, 10)) + buf.WriteString(", rocksdb: {") + buf.WriteString("delete_skipped_count: ") + buf.WriteString(strconv.FormatUint(sd.RocksdbDeleteSkippedCount, 10)) + buf.WriteString(", key_skipped_count: ") + buf.WriteString(strconv.FormatUint(sd.RocksdbKeySkippedCount, 10)) + buf.WriteString(", block: {") + buf.WriteString("cache_hit_count: ") + buf.WriteString(strconv.FormatUint(sd.RocksdbBlockCacheHitCount, 10)) + buf.WriteString(", read_count: ") + buf.WriteString(strconv.FormatUint(sd.RocksdbBlockReadCount, 10)) + buf.WriteString(", read_byte: ") + buf.WriteString(FormatBytes(int64(sd.RocksdbBlockReadByte))) + buf.WriteString("}}}") + return buf.String() +} + +// MergeFromScanDetailV2 merges scan detail from pb into itself. +func (sd *ScanDetail) MergeFromScanDetailV2(scanDetail *kvrpcpb.ScanDetailV2) { + if scanDetail != nil { + sd.TotalKeys += int64(scanDetail.TotalVersions) + sd.ProcessedKeys += int64(scanDetail.ProcessedVersions) + sd.RocksdbDeleteSkippedCount += scanDetail.RocksdbDeleteSkippedCount + sd.RocksdbKeySkippedCount += scanDetail.RocksdbKeySkippedCount + sd.RocksdbBlockCacheHitCount += scanDetail.RocksdbBlockCacheHitCount + sd.RocksdbBlockReadCount += scanDetail.RocksdbBlockReadCount + sd.RocksdbBlockReadByte += scanDetail.RocksdbBlockReadByte + } +} + +// TimeDetail contains coprocessor time detail information. +type TimeDetail struct { + // WaitWallTimeMs is the off-cpu wall time which is elapsed in TiKV side. Usually this includes queue waiting time and + // other kind of waitings in series. + ProcessTime time.Duration + // Off-cpu and on-cpu wall time elapsed to actually process the request payload. It does not + // include `wait_wall_time`. + // This field is very close to the CPU time in most cases. Some wait time spend in RocksDB + // cannot be excluded for now, like Mutex wait time, which is included in this field, so that + // this field is called wall time instead of CPU time. + WaitTime time.Duration +} + +// String implements the fmt.Stringer interface. +func (td *TimeDetail) String() string { + if td == nil { + return "" + } + buf := bytes.NewBuffer(make([]byte, 0, 16)) + if td.ProcessTime > 0 { + buf.WriteString("total_process_time: ") + buf.WriteString(FormatDuration(td.ProcessTime)) + } + if td.WaitTime > 0 { + if buf.Len() > 0 { + buf.WriteString(", ") + } + buf.WriteString("total_wait_time: ") + buf.WriteString(FormatDuration(td.WaitTime)) + } + return buf.String() +} + +// MergeFromTimeDetail merges time detail from pb into itself. +func (td *TimeDetail) MergeFromTimeDetail(timeDetail *kvrpcpb.TimeDetail) { + if timeDetail != nil { + td.WaitTime += time.Duration(timeDetail.WaitWallTimeMs) * time.Millisecond + td.ProcessTime += time.Duration(timeDetail.ProcessWallTimeMs) * time.Millisecond + } +} diff --git a/store/tikv/util/misc.go b/store/tikv/util/misc.go index 05a2692aa61a6..6b56b974d65fb 100644 --- a/store/tikv/util/misc.go +++ b/store/tikv/util/misc.go @@ -15,6 +15,8 @@ package util import ( "context" + "fmt" + "strconv" "strings" "time" @@ -74,3 +76,60 @@ var SessionID = sessionIDCtxKey{} func SetSessionID(ctx context.Context, sessionID uint64) context.Context { return context.WithValue(ctx, SessionID, sessionID) } + +const ( + byteSizeGB = int64(1 << 30) + byteSizeMB = int64(1 << 20) + byteSizeKB = int64(1 << 10) + byteSizeBB = int64(1) +) + +// FormatBytes uses to format bytes, this function will prune precision before format bytes. +func FormatBytes(numBytes int64) string { + if numBytes <= byteSizeKB { + return BytesToString(numBytes) + } + unit, unitStr := getByteUnit(numBytes) + if unit == byteSizeBB { + return BytesToString(numBytes) + } + v := float64(numBytes) / float64(unit) + decimal := 1 + if numBytes%unit == 0 { + decimal = 0 + } else if v < 10 { + decimal = 2 + } + return fmt.Sprintf("%v %s", strconv.FormatFloat(v, 'f', decimal, 64), unitStr) +} + +func getByteUnit(b int64) (int64, string) { + if b > byteSizeGB { + return byteSizeGB, "GB" + } else if b > byteSizeMB { + return byteSizeMB, "MB" + } else if b > byteSizeKB { + return byteSizeKB, "KB" + } + return byteSizeBB, "Bytes" +} + +// BytesToString converts the memory consumption to a readable string. +func BytesToString(numBytes int64) string { + GB := float64(numBytes) / float64(byteSizeGB) + if GB > 1 { + return fmt.Sprintf("%v GB", GB) + } + + MB := float64(numBytes) / float64(byteSizeMB) + if MB > 1 { + return fmt.Sprintf("%v MB", MB) + } + + KB := float64(numBytes) / float64(byteSizeKB) + if KB > 1 { + return fmt.Sprintf("%v KB", KB) + } + + return fmt.Sprintf("%v Bytes", numBytes) +} diff --git a/store/tikv/util/testleak/add-leaktest.sh b/store/tikv/util/testleak/add-leaktest.sh new file mode 100755 index 0000000000000..18b23d6da4f58 --- /dev/null +++ b/store/tikv/util/testleak/add-leaktest.sh @@ -0,0 +1,33 @@ +#!/bin/sh +# Copyright 2019 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. +# +# Usage: add-leaktest.sh pkg/*_test.go + +set -eu + +sed -i'~' -e ' + /^func (s \*test.*Suite) Test.*(c \*C) {/ { + n + /testleak.AfterTest/! i\ + defer testleak.AfterTest(c)() + } +' $@ + +for i in $@; do + if ! cmp -s $i $i~ ; then + goimports -w $i + fi +echo $i + rm -f $i~ +done diff --git a/store/tikv/util/testleak/check-leaktest.sh b/store/tikv/util/testleak/check-leaktest.sh new file mode 100755 index 0000000000000..62083b9014363 --- /dev/null +++ b/store/tikv/util/testleak/check-leaktest.sh @@ -0,0 +1,50 @@ +#!/bin/sh +# Copyright 2019 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. +# +# Usage: check-leaktest.sh +# It needs to run under the github.com/pingcap/tidb directory. + +set -e + +pkgs=$(git grep 'Suite' |grep -vE "Godeps|tags" |awk -F: '{print $1}' | xargs -n1 dirname | sort |uniq) +echo $pkgs +for pkg in ${pkgs}; do + if [ -z "$(ls ${pkg}/*_test.go 2>/dev/null)" ]; then + continue + fi + awk -F'[(]' ' +/func \(s .*Suite\) Test.*C\) {/ { + test = $1"("$2 + next +} + +/defer testleak.AfterTest/ { + test = 0 + next +} + +{ + if (test && (FILENAME != "./tidb_test.go")) { + printf "%s: %s: missing defer testleak.AfterTest\n", FILENAME, test + test = 0 + code = 1 + } +} + +END { + exit code +} + +' ${pkg}/*_test.go +done diff --git a/store/tikv/util/testleak/fake.go b/store/tikv/util/testleak/fake.go new file mode 100644 index 0000000000000..5d4ad573f69d6 --- /dev/null +++ b/store/tikv/util/testleak/fake.go @@ -0,0 +1,37 @@ +// Copyright 2017 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. +// +build !leak + +package testleak + +import ( + "testing" + + "github.com/pingcap/check" +) + +// BeforeTest is a dummy implementation when build tag 'leak' is not set. +func BeforeTest() { +} + +// AfterTest is a dummy implementation when build tag 'leak' is not set. +func AfterTest(c *check.C) func() { + return func() { + } +} + +// AfterTestT is used after all the test cases is finished. +func AfterTestT(t *testing.T) func() { + return func() { + } +} diff --git a/store/tikv/util/testleak/leaktest.go b/store/tikv/util/testleak/leaktest.go new file mode 100644 index 0000000000000..035eedc34bd4a --- /dev/null +++ b/store/tikv/util/testleak/leaktest.go @@ -0,0 +1,173 @@ +// Copyright 2013 The Go Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +// Copyright 2016 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. +// +build leak + +package testleak + +import ( + "runtime" + "sort" + "strings" + "testing" + "time" + + "github.com/pingcap/check" +) + +func interestingGoroutines() (gs []string) { + buf := make([]byte, 2<<20) + buf = buf[:runtime.Stack(buf, true)] + ignoreList := []string{ + "testing.RunTests", + "check.(*resultTracker).start", + "check.(*suiteRunner).runFunc", + "check.(*suiteRunner).parallelRun", + "localstore.(*dbStore).scheduler", + "testing.(*T).Run", + "testing.Main(", + "runtime.goexit", + "created by runtime.gc", + "interestingGoroutines", + "runtime.MHeap_Scavenger", + "created by os/signal.init", + "gopkg.in/natefinch/lumberjack%2ev2.(*Logger).millRun", + // these go routines are async terminated, so they may still alive after test end, thus cause + // false positive leak failures + "google.golang.org/grpc.(*addrConn).resetTransport", + "google.golang.org/grpc.(*ccBalancerWrapper).watcher", + "github.com/pingcap/goleveldb/leveldb/util.(*BufferPool).drain", + "github.com/pingcap/goleveldb/leveldb.(*DB).compactionError", + "github.com/pingcap/goleveldb/leveldb.(*DB).mpoolDrain", + "go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop", + "go.etcd.io/etcd/v3/pkg/logutil.(*MergeLogger).outputLoop", + "oracles.(*pdOracle).updateTS", + "tikv.(*KVStore).runSafePointChecker", + "tikv.(*RegionCache).asyncCheckAndResolveLoop", + "github.com/pingcap/badger", + "github.com/ngaut/unistore/tikv.(*MVCCStore).runUpdateSafePointLoop", + } + shouldIgnore := func(stack string) bool { + if stack == "" { + return true + } + for _, ident := range ignoreList { + if strings.Contains(stack, ident) { + return true + } + } + return false + } + for _, g := range strings.Split(string(buf), "\n\n") { + sl := strings.SplitN(g, "\n", 2) + if len(sl) != 2 { + continue + } + stack := strings.TrimSpace(sl[1]) + if shouldIgnore(stack) { + continue + } + gs = append(gs, stack) + } + sort.Strings(gs) + return +} + +var beforeTestGoroutines = map[string]bool{} +var testGoroutinesInited bool + +// BeforeTest gets the current goroutines. +// It's used for check.Suite.SetUpSuite() function. +// Now it's only used in the tidb_test.go. +// Note: it's not accurate, consider the following function: +// func loop() { +// for { +// select { +// case <-ticker.C: +// DoSomething() +// } +// } +// } +// If this loop step into DoSomething() during BeforeTest(), the stack for this goroutine will contain DoSomething(). +// Then if this loop jumps out of DoSomething during AfterTest(), the stack for this goroutine will not contain DoSomething(). +// Resulting in false-positive leak reports. +func BeforeTest() { + for _, g := range interestingGoroutines() { + beforeTestGoroutines[g] = true + } + testGoroutinesInited = true +} + +const defaultCheckCnt = 50 + +func checkLeakAfterTest(errorFunc func(cnt int, g string)) func() { + // After `BeforeTest`, `beforeTestGoroutines` may still be empty, in this case, + // we shouldn't init it again. + if !testGoroutinesInited && len(beforeTestGoroutines) == 0 { + for _, g := range interestingGoroutines() { + beforeTestGoroutines[g] = true + } + } + + cnt := defaultCheckCnt + return func() { + defer func() { + beforeTestGoroutines = map[string]bool{} + testGoroutinesInited = false + }() + + var leaked []string + for i := 0; i < cnt; i++ { + leaked = leaked[:0] + for _, g := range interestingGoroutines() { + if !beforeTestGoroutines[g] { + leaked = append(leaked, g) + } + } + // Bad stuff found, but goroutines might just still be + // shutting down, so give it some time. + if len(leaked) != 0 { + time.Sleep(50 * time.Millisecond) + continue + } + + return + } + for _, g := range leaked { + errorFunc(cnt, g) + } + } +} + +// AfterTest gets the current goroutines and runs the returned function to +// get the goroutines at that time to contrast whether any goroutines leaked. +// Usage: defer testleak.AfterTest(c)() +// It can call with BeforeTest() at the beginning of check.Suite.TearDownSuite() or +// call alone at the beginning of each test. +func AfterTest(c *check.C) func() { + errorFunc := func(cnt int, g string) { + c.Errorf("Test %s check-count %d appears to have leaked: %v", c.TestName(), cnt, g) + } + return checkLeakAfterTest(errorFunc) +} + +// AfterTestT is used after all the test cases is finished. +func AfterTestT(t *testing.T) func() { + errorFunc := func(cnt int, g string) { + t.Errorf("Test %s check-count %d appears to have leaked: %v", t.Name(), cnt, g) + } + return checkLeakAfterTest(errorFunc) +} diff --git a/table/tables/index.go b/table/tables/index.go index d6197de894b43..8b4630d47f70d 100644 --- a/table/tables/index.go +++ b/table/tables/index.go @@ -105,10 +105,6 @@ func NeedRestoredData(idxCols []*model.IndexColumn, colInfos []*model.ColumnInfo return false } -func (c *index) checkNeedRestoredData() bool { - return NeedRestoredData(c.idxInfo.Columns, c.tblInfo.Columns) -} - // NewIndex builds a new Index object. func NewIndex(physicalID int64, tblInfo *model.TableInfo, indexInfo *model.IndexInfo) table.Index { // The prefix can't encode from tblInfo.ID, because table partition may change the id to partition id. diff --git a/telemetry/data_feature_usage.go b/telemetry/data_feature_usage.go index 81831e933e392..ff3e3c672cff0 100644 --- a/telemetry/data_feature_usage.go +++ b/telemetry/data_feature_usage.go @@ -79,11 +79,11 @@ var initialTxnCommitCounter metrics.TxnCommitCounter func GetTxnUsageInfo(ctx sessionctx.Context) *TxnUsage { asyncCommitUsed := false if val, err := variable.GetGlobalSystemVar(ctx.GetSessionVars(), variable.TiDBEnableAsyncCommit); err == nil { - asyncCommitUsed = val == variable.BoolOn + asyncCommitUsed = val == variable.On } onePCUsed := false if val, err := variable.GetGlobalSystemVar(ctx.GetSessionVars(), variable.TiDBEnable1PC); err == nil { - onePCUsed = val == variable.BoolOn + onePCUsed = val == variable.On } curr := metrics.GetTxnCommitCounter() diff := curr.Sub(initialTxnCommitCounter) diff --git a/telemetry/telemetry.go b/telemetry/telemetry.go index 0aee303a03d5d..55d6dfb215d34 100644 --- a/telemetry/telemetry.go +++ b/telemetry/telemetry.go @@ -17,6 +17,7 @@ import ( "bytes" "context" "encoding/json" + "fmt" "net/http" "time" @@ -24,7 +25,9 @@ import ( "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/util/logutil" "go.etcd.io/etcd/clientv3" + "go.uber.org/zap" ) const ( @@ -127,6 +130,7 @@ func reportUsageData(ctx sessionctx.Context, etcdClient *clientv3.Client) (bool, } req.Header.Add("Content-Type", "application/json") + logutil.BgLogger().Info(fmt.Sprintf("Uploading telemetry data to %s", apiEndpoint)) resp, err := http.DefaultClient.Do(req) if err != nil { return false, errors.Trace(err) @@ -159,3 +163,15 @@ func ReportUsageData(ctx sessionctx.Context, etcdClient *clientv3.Client) error return updateTelemetryStatus(s, etcdClient) } + +// InitialRun reports the Telmetry configuration and trigger an initial run +func InitialRun(ctx sessionctx.Context, etcdClient *clientv3.Client) error { + enabled, err := IsTelemetryEnabled(ctx) + if err != nil { + return err + } + + logutil.BgLogger().Info("Telemetry configuration", zap.String("endpoint", apiEndpoint), zap.Duration("report_interval", ReportInterval), zap.Bool("enabled", enabled)) + + return ReportUsageData(ctx, etcdClient) +} diff --git a/util/collate/collate.go b/util/collate/collate.go index 062b624c50e04..78540ea48f220 100644 --- a/util/collate/collate.go +++ b/util/collate/collate.go @@ -193,6 +193,21 @@ func CollationName2ID(name string) int { return mysql.DefaultCollationID } +// SubstituteMissingCollationToDefault will switch to the default collation if +// new collations are enabled and the specified collation is not supported. +func SubstituteMissingCollationToDefault(co string) string { + var err error + if _, err = GetCollationByName(co); err == nil { + return co + } + logutil.BgLogger().Warn(err.Error()) + var coll *charset.Collation + if coll, err = GetCollationByName(charset.CollationUTF8MB4); err != nil { + logutil.BgLogger().Warn(err.Error()) + } + return coll.Name +} + // GetCollationByName wraps charset.GetCollationByName, it checks the collation. func GetCollationByName(name string) (coll *charset.Collation, err error) { if coll, err = charset.GetCollationByName(name); err != nil { diff --git a/util/execdetails/execdetails.go b/util/execdetails/execdetails.go index 64351c359433c..ca045352dbd33 100644 --- a/util/execdetails/execdetails.go +++ b/util/execdetails/execdetails.go @@ -24,9 +24,7 @@ import ( "sync/atomic" "time" - "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/tidb/store/tikv/util" - "github.com/pingcap/tidb/util/memory" "github.com/pingcap/tipb/go-tipb" "go.uber.org/zap" ) @@ -42,8 +40,8 @@ type ExecDetails struct { RequestCount int CommitDetail *util.CommitDetails LockKeysDetail *util.LockKeysDetails - ScanDetail *ScanDetail - TimeDetail TimeDetail + ScanDetail *util.ScanDetail + TimeDetail util.TimeDetail } type stmtExecDetailKeyType struct{} @@ -56,122 +54,6 @@ type StmtExecDetails struct { WriteSQLRespDuration time.Duration } -// TimeDetail contains coprocessor time detail information. -type TimeDetail struct { - // WaitWallTimeMs is the off-cpu wall time which is elapsed in TiKV side. Usually this includes queue waiting time and - // other kind of waitings in series. - ProcessTime time.Duration - // Off-cpu and on-cpu wall time elapsed to actually process the request payload. It does not - // include `wait_wall_time`. - // This field is very close to the CPU time in most cases. Some wait time spend in RocksDB - // cannot be excluded for now, like Mutex wait time, which is included in this field, so that - // this field is called wall time instead of CPU time. - WaitTime time.Duration -} - -// String implements the fmt.Stringer interface. -func (td *TimeDetail) String() string { - if td == nil { - return "" - } - buf := bytes.NewBuffer(make([]byte, 0, 16)) - if td.ProcessTime > 0 { - buf.WriteString("total_process_time: ") - buf.WriteString(FormatDuration(td.ProcessTime)) - } - if td.WaitTime > 0 { - if buf.Len() > 0 { - buf.WriteString(", ") - } - buf.WriteString("total_wait_time: ") - buf.WriteString(FormatDuration(td.WaitTime)) - } - return buf.String() -} - -// MergeFromTimeDetail merges time detail from pb into itself. -func (td *TimeDetail) MergeFromTimeDetail(timeDetail *kvrpcpb.TimeDetail) { - if timeDetail != nil { - td.WaitTime += time.Duration(timeDetail.WaitWallTimeMs) * time.Millisecond - td.ProcessTime += time.Duration(timeDetail.ProcessWallTimeMs) * time.Millisecond - } -} - -// ScanDetail contains coprocessor scan detail information. -type ScanDetail struct { - // TotalKeys is the approximate number of MVCC keys meet during scanning. It includes - // deleted versions, but does not include RocksDB tombstone keys. - TotalKeys int64 - // ProcessedKeys is the number of user keys scanned from the storage. - // It does not include deleted version or RocksDB tombstone keys. - // For Coprocessor requests, it includes keys that has been filtered out by Selection. - ProcessedKeys int64 - // RocksdbDeleteSkippedCount is the total number of deletes and single deletes skipped over during - // iteration, i.e. how many RocksDB tombstones are skipped. - RocksdbDeleteSkippedCount uint64 - // RocksdbKeySkippedCount it the total number of internal keys skipped over during iteration. - RocksdbKeySkippedCount uint64 - // RocksdbBlockCacheHitCount is the total number of RocksDB block cache hits. - RocksdbBlockCacheHitCount uint64 - // RocksdbBlockReadCount is the total number of block reads (with IO). - RocksdbBlockReadCount uint64 - // RocksdbBlockReadByte is the total number of bytes from block reads. - RocksdbBlockReadByte uint64 -} - -// Merge merges scan detail execution details into self. -func (sd *ScanDetail) Merge(scanDetail *ScanDetail) { - atomic.AddInt64(&sd.TotalKeys, scanDetail.TotalKeys) - atomic.AddInt64(&sd.ProcessedKeys, scanDetail.ProcessedKeys) - atomic.AddUint64(&sd.RocksdbDeleteSkippedCount, scanDetail.RocksdbDeleteSkippedCount) - atomic.AddUint64(&sd.RocksdbKeySkippedCount, scanDetail.RocksdbKeySkippedCount) - atomic.AddUint64(&sd.RocksdbBlockCacheHitCount, scanDetail.RocksdbBlockCacheHitCount) - atomic.AddUint64(&sd.RocksdbBlockReadCount, scanDetail.RocksdbBlockReadCount) - atomic.AddUint64(&sd.RocksdbBlockReadByte, scanDetail.RocksdbBlockReadByte) -} - -var zeroScanDetail = ScanDetail{} - -// String implements the fmt.Stringer interface. -func (sd *ScanDetail) String() string { - if sd == nil || *sd == zeroScanDetail { - return "" - } - buf := bytes.NewBuffer(make([]byte, 0, 16)) - buf.WriteString("scan_detail: {") - buf.WriteString("total_process_keys: ") - buf.WriteString(strconv.FormatInt(sd.ProcessedKeys, 10)) - buf.WriteString(", total_keys: ") - buf.WriteString(strconv.FormatInt(sd.TotalKeys, 10)) - buf.WriteString(", rocksdb: {") - buf.WriteString("delete_skipped_count: ") - buf.WriteString(strconv.FormatUint(sd.RocksdbDeleteSkippedCount, 10)) - buf.WriteString(", key_skipped_count: ") - buf.WriteString(strconv.FormatUint(sd.RocksdbKeySkippedCount, 10)) - buf.WriteString(", block: {") - buf.WriteString("cache_hit_count: ") - buf.WriteString(strconv.FormatUint(sd.RocksdbBlockCacheHitCount, 10)) - buf.WriteString(", read_count: ") - buf.WriteString(strconv.FormatUint(sd.RocksdbBlockReadCount, 10)) - buf.WriteString(", read_byte: ") - buf.WriteString(memory.FormatBytes(int64(sd.RocksdbBlockReadByte))) - buf.WriteString("}}}") - return buf.String() -} - -// MergeFromScanDetailV2 merges scan detail from pb into itself. -func (sd *ScanDetail) MergeFromScanDetailV2(scanDetail *kvrpcpb.ScanDetailV2) { - if scanDetail != nil { - sd.TotalKeys += int64(scanDetail.TotalVersions) - sd.ProcessedKeys += int64(scanDetail.ProcessedVersions) - sd.RocksdbDeleteSkippedCount += scanDetail.RocksdbDeleteSkippedCount - sd.RocksdbKeySkippedCount += scanDetail.RocksdbKeySkippedCount - sd.RocksdbBlockCacheHitCount += scanDetail.RocksdbBlockCacheHitCount - sd.RocksdbBlockReadCount += scanDetail.RocksdbBlockReadCount - sd.RocksdbBlockReadByte += scanDetail.RocksdbBlockReadByte - } -} - const ( // CopTimeStr represents the sum of cop-task time spend in TiDB distSQL. CopTimeStr = "Cop_time" @@ -435,7 +317,7 @@ type CopRuntimeStats struct { // same tikv-server instance. We have to use a list to maintain all tasks // executed on each instance. stats map[string][]*basicCopRuntimeStats - scanDetail *ScanDetail + scanDetail *util.ScanDetail // do not use kv.StoreType because it will meet cycle import error storeType string } @@ -739,7 +621,7 @@ func (e *RuntimeStatsColl) GetOrCreateCopStats(planID int, storeType string) *Co if !ok { copStats = &CopRuntimeStats{ stats: make(map[string][]*basicCopRuntimeStats), - scanDetail: &ScanDetail{}, + scanDetail: &util.ScanDetail{}, storeType: storeType, } e.copStats[planID] = copStats @@ -769,7 +651,7 @@ func (e *RuntimeStatsColl) RecordOneCopTask(planID int, storeType string, addres } // RecordScanDetail records a specific cop tasks's cop detail. -func (e *RuntimeStatsColl) RecordScanDetail(planID int, storeType string, detail *ScanDetail) { +func (e *RuntimeStatsColl) RecordScanDetail(planID int, storeType string, detail *util.ScanDetail) { copStats := e.GetOrCreateCopStats(planID, storeType) copStats.scanDetail.Merge(detail) } diff --git a/util/execdetails/execdetails_test.go b/util/execdetails/execdetails_test.go index 5684912fbf8ae..371d06006051f 100644 --- a/util/execdetails/execdetails_test.go +++ b/util/execdetails/execdetails_test.go @@ -58,7 +58,7 @@ func TestString(t *testing.T) { PrewriteRegionNum: 1, TxnRetry: 1, }, - ScanDetail: &ScanDetail{ + ScanDetail: &util.ScanDetail{ ProcessedKeys: 10, TotalKeys: 100, RocksdbDeleteSkippedCount: 1, @@ -67,7 +67,7 @@ func TestString(t *testing.T) { RocksdbBlockReadCount: 1, RocksdbBlockReadByte: 100, }, - TimeDetail: TimeDetail{ + TimeDetail: util.TimeDetail{ ProcessTime: 2*time.Second + 5*time.Millisecond, WaitTime: time.Second, }, @@ -103,7 +103,7 @@ func TestCopRuntimeStats(t *testing.T) { stats.RecordOneCopTask(tableScanID, "tikv", "8.8.8.9", mockExecutorExecutionSummary(2, 2, 2)) stats.RecordOneCopTask(aggID, "tikv", "8.8.8.8", mockExecutorExecutionSummary(3, 3, 3)) stats.RecordOneCopTask(aggID, "tikv", "8.8.8.9", mockExecutorExecutionSummary(4, 4, 4)) - scanDetail := &ScanDetail{ + scanDetail := &util.ScanDetail{ TotalKeys: 15, ProcessedKeys: 10, RocksdbDeleteSkippedCount: 5, @@ -151,7 +151,7 @@ func TestCopRuntimeStats(t *testing.T) { t.Fatalf(cop.String()) } - zeroScanDetail := ScanDetail{} + zeroScanDetail := util.ScanDetail{} if zeroScanDetail.String() != "" { t.Fatalf(zeroScanDetail.String()) } @@ -166,7 +166,7 @@ func TestCopRuntimeStatsForTiFlash(t *testing.T) { stats.RecordOneCopTask(aggID, "tiflash", "8.8.8.9", mockExecutorExecutionSummaryForTiFlash(2, 2, 2, 1, "tablescan_"+strconv.Itoa(tableScanID))) stats.RecordOneCopTask(tableScanID, "tiflash", "8.8.8.8", mockExecutorExecutionSummaryForTiFlash(3, 3, 3, 1, "aggregation_"+strconv.Itoa(aggID))) stats.RecordOneCopTask(tableScanID, "tiflash", "8.8.8.9", mockExecutorExecutionSummaryForTiFlash(4, 4, 4, 1, "aggregation_"+strconv.Itoa(aggID))) - scanDetail := &ScanDetail{ + scanDetail := &util.ScanDetail{ TotalKeys: 10, ProcessedKeys: 10, RocksdbDeleteSkippedCount: 10, diff --git a/util/fastrand/random_test.go b/util/fastrand/random_test.go index 673784193a7d1..8bacb1e69d00a 100644 --- a/util/fastrand/random_test.go +++ b/util/fastrand/random_test.go @@ -21,6 +21,11 @@ import ( . "github.com/pingcap/check" ) +func TestT(t *testing.T) { + CustomVerboseFlag = true + TestingT(t) +} + var _ = Suite(&testRandSuite{}) type testRandSuite struct { diff --git a/util/gcutil/gcutil.go b/util/gcutil/gcutil.go index e307d876604e4..5cf8c742b6d1e 100644 --- a/util/gcutil/gcutil.go +++ b/util/gcutil/gcutil.go @@ -40,12 +40,12 @@ func CheckGCEnable(ctx sessionctx.Context) (enable bool, err error) { // DisableGC will disable GC enable variable. func DisableGC(ctx sessionctx.Context) error { - return ctx.GetSessionVars().GlobalVarsAccessor.SetGlobalSysVar(variable.TiDBGCEnable, variable.BoolOff) + return ctx.GetSessionVars().GlobalVarsAccessor.SetGlobalSysVar(variable.TiDBGCEnable, variable.Off) } // EnableGC will enable GC enable variable. func EnableGC(ctx sessionctx.Context) error { - return ctx.GetSessionVars().GlobalVarsAccessor.SetGlobalSysVar(variable.TiDBGCEnable, variable.BoolOn) + return ctx.GetSessionVars().GlobalVarsAccessor.SetGlobalSysVar(variable.TiDBGCEnable, variable.On) } // ValidateSnapshot checks that the newly set snapshot time is after GC safe point time. diff --git a/util/mock/client.go b/util/mock/client.go index 56ec53336d2e4..72a8422be7509 100644 --- a/util/mock/client.go +++ b/util/mock/client.go @@ -28,6 +28,6 @@ type Client struct { } // Send implement kv.Client interface. -func (c *Client) Send(ctx context.Context, req *kv.Request, kv *kv.Variables, sessionMemTracker *memory.Tracker, enabledRateLimit bool) kv.Response { +func (c *Client) Send(ctx context.Context, req *kv.Request, kv interface{}, sessionMemTracker *memory.Tracker, enabledRateLimit bool) kv.Response { return c.MockResponse } diff --git a/util/sem/sem.go b/util/sem/sem.go index cf59b14899283..0dde650b9d765 100644 --- a/util/sem/sem.go +++ b/util/sem/sem.go @@ -67,14 +67,14 @@ var ( // Dynamic configuration by users may be a security risk. func Enable() { atomic.StoreInt32(&semEnabled, 1) - variable.SetSysVar(variable.TiDBEnableEnhancedSecurity, variable.BoolOn) + variable.SetSysVar(variable.TiDBEnableEnhancedSecurity, variable.On) } // Disable disables SEM. This is intended to be used by the test-suite. // Dynamic configuration by users may be a security risk. func Disable() { atomic.StoreInt32(&semEnabled, 0) - variable.SetSysVar(variable.TiDBEnableEnhancedSecurity, variable.BoolOff) + variable.SetSysVar(variable.TiDBEnableEnhancedSecurity, variable.Off) } // IsEnabled checks if Security Enhanced Mode (SEM) is enabled diff --git a/util/stmtsummary/statement_summary_test.go b/util/stmtsummary/statement_summary_test.go index 4562ad877f934..5971e83e7980f 100644 --- a/util/stmtsummary/statement_summary_test.go +++ b/util/stmtsummary/statement_summary_test.go @@ -200,7 +200,7 @@ func (s *testStmtSummarySuite) TestAddStatement(c *C) { PrewriteRegionNum: 100, TxnRetry: 10, }, - ScanDetail: &execdetails.ScanDetail{ + ScanDetail: &util.ScanDetail{ TotalKeys: 6000, ProcessedKeys: 1500, RocksdbDeleteSkippedCount: 100, @@ -209,7 +209,7 @@ func (s *testStmtSummarySuite) TestAddStatement(c *C) { RocksdbBlockReadCount: 10, RocksdbBlockReadByte: 1000, }, - TimeDetail: execdetails.TimeDetail{ + TimeDetail: util.TimeDetail{ ProcessTime: 1500, WaitTime: 150, }, @@ -327,7 +327,7 @@ func (s *testStmtSummarySuite) TestAddStatement(c *C) { PrewriteRegionNum: 10, TxnRetry: 1, }, - ScanDetail: &execdetails.ScanDetail{ + ScanDetail: &util.ScanDetail{ TotalKeys: 600, ProcessedKeys: 150, RocksdbDeleteSkippedCount: 100, @@ -336,7 +336,7 @@ func (s *testStmtSummarySuite) TestAddStatement(c *C) { RocksdbBlockReadCount: 10, RocksdbBlockReadByte: 1000, }, - TimeDetail: execdetails.TimeDetail{ + TimeDetail: util.TimeDetail{ ProcessTime: 150, WaitTime: 15, }, @@ -583,7 +583,7 @@ func generateAnyExecInfo() *StmtExecInfo { PrewriteRegionNum: 20, TxnRetry: 2, }, - ScanDetail: &execdetails.ScanDetail{ + ScanDetail: &util.ScanDetail{ TotalKeys: 1000, ProcessedKeys: 500, RocksdbDeleteSkippedCount: 100, @@ -592,7 +592,7 @@ func generateAnyExecInfo() *StmtExecInfo { RocksdbBlockReadCount: 10, RocksdbBlockReadByte: 1000, }, - TimeDetail: execdetails.TimeDetail{ + TimeDetail: util.TimeDetail{ ProcessTime: 500, WaitTime: 50, },