diff --git a/cmd/explaintest/r/explain_easy.result b/cmd/explaintest/r/explain_easy.result index a33fe5a791bbc..ad77afcf869e2 100644 --- a/cmd/explaintest/r/explain_easy.result +++ b/cmd/explaintest/r/explain_easy.result @@ -194,29 +194,29 @@ test t4 1 expr_idx 1 NULL NULL (`a` + `b` + 1) 2 YES NO explain format = 'brief' select count(1) from (select count(1) from (select * from t1 where c3 = 100) k) k2; id estRows task access object operator info StreamAgg 1.00 root funcs:count(1)->Column#5 -└─StreamAgg 1.00 root funcs:firstrow(Column#9)->Column#7 +└─StreamAgg 1.00 root funcs:firstrow(Column#13)->Column#11 └─TableReader 1.00 root data:StreamAgg - └─StreamAgg 1.00 cop[tikv] funcs:firstrow(1)->Column#9 + └─StreamAgg 1.00 cop[tikv] funcs:firstrow(1)->Column#13 └─Selection 10.00 cop[tikv] eq(test.t1.c3, 100) └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo explain format = 'brief' select 1 from (select count(c2), count(c3) from t1) k; id estRows task access object operator info Projection 1.00 root 1->Column#6 -└─StreamAgg 1.00 root funcs:firstrow(Column#14)->Column#9 +└─StreamAgg 1.00 root funcs:firstrow(Column#16)->Column#11 └─TableReader 1.00 root data:StreamAgg - └─StreamAgg 1.00 cop[tikv] funcs:firstrow(1)->Column#14 + └─StreamAgg 1.00 cop[tikv] funcs:firstrow(1)->Column#16 └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo explain format = 'brief' select count(1) from (select max(c2), count(c3) as m from t1) k; id estRows task access object operator info StreamAgg 1.00 root funcs:count(1)->Column#6 -└─StreamAgg 1.00 root funcs:firstrow(Column#13)->Column#8 +└─StreamAgg 1.00 root funcs:firstrow(Column#15)->Column#10 └─TableReader 1.00 root data:StreamAgg - └─StreamAgg 1.00 cop[tikv] funcs:firstrow(1)->Column#13 + └─StreamAgg 1.00 cop[tikv] funcs:firstrow(1)->Column#15 └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo explain format = 'brief' select count(1) from (select count(c2) from t1 group by c3) k; id estRows task access object operator info StreamAgg 1.00 root funcs:count(1)->Column#5 -└─HashAgg 8000.00 root group by:test.t1.c3, funcs:firstrow(1)->Column#7 +└─HashAgg 8000.00 root group by:test.t1.c3, funcs:firstrow(1)->Column#9 └─TableReader 10000.00 root data:TableFullScan └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo set @@session.tidb_opt_insubq_to_join_and_agg=0; @@ -498,8 +498,8 @@ PRIMARY KEY (`id`) explain format = 'brief' SELECT COUNT(1) FROM (SELECT COALESCE(b.region_name, '不详') region_name, SUM(a.registration_num) registration_num FROM (SELECT stat_date, show_date, region_id, 0 registration_num FROM test01 WHERE period = 1 AND stat_date >= 20191202 AND stat_date <= 20191202 UNION ALL SELECT stat_date, show_date, region_id, registration_num registration_num FROM test01 WHERE period = 1 AND stat_date >= 20191202 AND stat_date <= 20191202) a LEFT JOIN test02 b ON a.region_id = b.id WHERE registration_num > 0 AND a.stat_date >= '20191202' AND a.stat_date <= '20191202' GROUP BY a.stat_date , a.show_date , COALESCE(b.region_name, '不详') ) JLS; id estRows task access object operator info StreamAgg 1.00 root funcs:count(1)->Column#22 -└─HashAgg 8000.00 root group by:Column#32, Column#33, Column#34, funcs:firstrow(1)->Column#31 - └─Projection 10000.01 root Column#14, Column#15, coalesce(test.test02.region_name, 不详)->Column#34 +└─HashAgg 8000.00 root group by:Column#34, Column#35, Column#36, funcs:firstrow(1)->Column#33 + └─Projection 10000.01 root Column#14, Column#15, coalesce(test.test02.region_name, 不详)->Column#36 └─HashJoin 10000.01 root left outer join, equal:[eq(Column#16, test.test02.id)] ├─TableReader(Build) 10000.00 root data:TableFullScan │ └─TableFullScan 10000.00 cop[tikv] table:b keep order:false, stats:pseudo diff --git a/cmd/explaintest/r/explain_indexmerge.result b/cmd/explaintest/r/explain_indexmerge.result index 4a048b40da7f7..83bc89a593e7c 100644 --- a/cmd/explaintest/r/explain_indexmerge.result +++ b/cmd/explaintest/r/explain_indexmerge.result @@ -146,7 +146,7 @@ KEY `aid_c2` (`aid`,`c2`) ); desc select /*+ USE_INDEX_MERGE(t, aid_c1, aid_c2) */ * from t where (aid = 1 and c1='aaa') or (aid = 2 and c2='bbb'); id estRows task access object operator info -IndexMerge_8 269.49 root +IndexMerge_8 8.08 root ├─IndexRangeScan_5(Build) 0.10 cop[tikv] table:t, index:aid_c1(aid, c1) range:[1 "aaa",1 "aaa"], keep order:false, stats:pseudo ├─IndexRangeScan_6(Build) 0.10 cop[tikv] table:t, index:aid_c2(aid, c2) range:[2 "bbb",2 "bbb"], keep order:false, stats:pseudo -└─TableRowIDScan_7(Probe) 269.49 cop[tikv] table:t keep order:false, stats:pseudo +└─TableRowIDScan_7(Probe) 8.08 cop[tikv] table:t keep order:false, stats:pseudo diff --git a/cmd/explaintest/r/explain_join_stats.result b/cmd/explaintest/r/explain_join_stats.result index 15e68179c5085..1480805628875 100644 --- a/cmd/explaintest/r/explain_join_stats.result +++ b/cmd/explaintest/r/explain_join_stats.result @@ -7,13 +7,14 @@ load stats 's/explain_join_stats_lo.json'; explain format = 'brief' select count(*) from e, lo where lo.a=e.a and e.b=22336; id estRows task access object operator info StreamAgg 1.00 root funcs:count(1)->Column#5 -└─HashJoin 19977.00 root inner join, equal:[eq(test.lo.a, test.e.a)] - ├─TableReader(Build) 250.00 root data:TableFullScan - │ └─TableFullScan 250.00 cop[tikv] table:lo keep order:false - └─IndexLookUp(Probe) 19977.00 root - ├─IndexRangeScan(Build) 19977.00 cop[tikv] table:e, index:idx_b(b) range:[22336,22336], keep order:false - └─Selection(Probe) 19977.00 cop[tikv] not(isnull(test.e.a)) - └─TableRowIDScan 19977.00 cop[tikv] table:e keep order:false +└─Projection 19977.00 root 1->Column#6 + └─HashJoin 19977.00 root inner join, equal:[eq(test.lo.a, test.e.a)] + ├─TableReader(Build) 250.00 root data:TableFullScan + │ └─TableFullScan 250.00 cop[tikv] table:lo keep order:false + └─IndexLookUp(Probe) 19977.00 root + ├─IndexRangeScan(Build) 19977.00 cop[tikv] table:e, index:idx_b(b) range:[22336,22336], keep order:false + └─Selection(Probe) 19977.00 cop[tikv] not(isnull(test.e.a)) + └─TableRowIDScan 19977.00 cop[tikv] table:e keep order:false explain format = 'brief' select /*+ TIDB_INLJ(e) */ count(*) from e, lo where lo.a=e.a and e.b=22336; id estRows task access object operator info StreamAgg 1.00 root funcs:count(1)->Column#5 diff --git a/config/config.go b/config/config.go index 4275ed8151e44..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"], } } @@ -419,7 +420,7 @@ type Performance struct { TCPNoDelay bool `toml:"tcp-no-delay" json:"tcp-no-delay"` CrossJoin bool `toml:"cross-join" json:"cross-join"` RunAutoAnalyze bool `toml:"run-auto-analyze" json:"run-auto-analyze"` - DistinctAggPushDown bool `toml:"distinct-agg-push-down" json:"agg-push-down-join"` + DistinctAggPushDown bool `toml:"distinct-agg-push-down" json:"distinct-agg-push-down"` CommitterConcurrency int `toml:"committer-concurrency" json:"committer-concurrency"` MaxTxnTTL uint64 `toml:"max-txn-ttl" json:"max-txn-ttl"` MemProfileInterval string `toml:"mem-profile-interval" json:"mem-profile-interval"` diff --git a/config/config_test.go b/config/config_test.go index caeb8ef7e4ce5..9533b8bf52975 100644 --- a/config/config_test.go +++ b/config/config_test.go @@ -18,6 +18,7 @@ import ( "os" "os/user" "path/filepath" + "reflect" "runtime" "testing" @@ -423,6 +424,14 @@ xkNuJ2BlEGkwWLiRbKy1lNBBFUXKuhh3L/EIY10WTnr3TQzeL6H1 // is recycled when the reference count drops to 0. c.Assert(os.Remove(certFile), IsNil) c.Assert(os.Remove(keyFile), IsNil) + + // test for config `toml` and `json` tag names + c1 := Config{} + st := reflect.TypeOf(c1) + for i := 0; i < st.NumField(); i++ { + field := st.Field(i) + c.Assert(field.Tag.Get("toml"), Equals, field.Tag.Get("json")) + } } func (s *testConfigSuite) TestOOMActionValid(c *C) { 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 e0c9249e8869a..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.LabelConstraint{ + 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 @@ -6085,7 +6088,7 @@ func (d *ddl) AlterTableAlterPartition(ctx sessionctx.Context, ident ast.Ident, Count: cnt, StartKeyHex: startKey, EndKeyHex: endKey, - LabelConstraints: []placement.LabelConstraint{{ + LabelConstraints: []placement.Constraint{{ Op: placement.NotIn, Key: placement.EngineLabelKey, Values: []string{placement.EngineLabelTiFlash}, 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/constraint.go b/ddl/placement/constraint.go new file mode 100644 index 0000000000000..9eae461ffe0f9 --- /dev/null +++ b/ddl/placement/constraint.go @@ -0,0 +1,146 @@ +// 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" +) + +// ConstraintOp defines how a Constraint matches a store. +type ConstraintOp string + +const ( + // In restricts the store label value should in the value list. + // If label does not exist, `in` is always false. + In ConstraintOp = "in" + // NotIn restricts the store label value should not in the value list. + // If label does not exist, `notIn` is always true. + NotIn ConstraintOp = "notIn" + // Exists restricts the store should have the label. + Exists ConstraintOp = "exists" + // NotExists restricts the store should not have the label. + NotExists ConstraintOp = "notExists" +) + +// Constraint is used to filter store when trying to place peer of a region. +type Constraint struct { + Key string `json:"key,omitempty"` + Op ConstraintOp `json:"op,omitempty"` + Values []string `json:"values,omitempty"` +} + +// NewConstraint will create a Constraint from a string. +func NewConstraint(label string) (Constraint, error) { + r := Constraint{} + + if len(label) < 4 { + return r, fmt.Errorf("%w: %s", ErrInvalidConstraintFormat, label) + } + + var op ConstraintOp + switch label[0] { + case '+': + op = In + case '-': + op = NotIn + default: + return r, fmt.Errorf("%w: %s", ErrInvalidConstraintFormat, label) + } + + kv := strings.Split(label[1:], "=") + if len(kv) != 2 { + return r, fmt.Errorf("%w: %s", ErrInvalidConstraintFormat, label) + } + + key := strings.TrimSpace(kv[0]) + if key == "" { + return r, fmt.Errorf("%w: %s", ErrInvalidConstraintFormat, label) + } + + val := strings.TrimSpace(kv[1]) + if val == "" { + return r, fmt.Errorf("%w: %s", ErrInvalidConstraintFormat, label) + } + + if op == In && key == EngineLabelKey && strings.ToLower(val) == EngineLabelTiFlash { + return r, fmt.Errorf("%w: %s", ErrUnsupportedConstraint, label) + } + + r.Key = key + r.Op = op + r.Values = []string{val} + return r, nil +} + +// Restore converts a Constraint to a string. +func (c *Constraint) Restore() (string, error) { + var sb strings.Builder + if len(c.Values) != 1 { + return "", fmt.Errorf("%w: constraint should have exactly one label value, got %v", ErrInvalidConstraintFormat, c.Values) + } + switch c.Op { + case In: + sb.WriteString("+") + case NotIn: + sb.WriteString("-") + default: + return "", fmt.Errorf("%w: disallowed operation '%s'", ErrInvalidConstraintFormat, c.Op) + } + sb.WriteString(c.Key) + sb.WriteString("=") + sb.WriteString(c.Values[0]) + return sb.String(), nil +} + +// ConstraintCompatibility is the return type of CompatibleWith. +type ConstraintCompatibility byte + +const ( + // ConstraintCompatible indicates two constraints are compatible. + ConstraintCompatible ConstraintCompatibility = iota + // ConstraintIncompatible indicates two constraints are incompatible. + ConstraintIncompatible + // ConstraintDuplicated indicates two constraints are duplicated. + ConstraintDuplicated +) + +// CompatibleWith will check if two constraints are compatible. +// Return (compatible, duplicated). +func (c *Constraint) CompatibleWith(o *Constraint) ConstraintCompatibility { + sameKey := c.Key == o.Key + if !sameKey { + return ConstraintCompatible + } + + sameOp := c.Op == o.Op + sameVal := true + for i := range c.Values { + if i < len(o.Values) && c.Values[i] != o.Values[i] { + sameVal = false + break + } + } + // no following cases: + // 1. duplicated constraint, skip it + // 2. no instance can meet: +dc=sh, -dc=sh + // 3. can not match multiple instances: +dc=sh, +dc=bj + if sameOp && sameVal { + return ConstraintDuplicated + } else if (!sameOp && sameVal) || (sameOp && !sameVal && c.Op == In) { + return ConstraintIncompatible + } + + return ConstraintCompatible +} diff --git a/ddl/placement/constraint_test.go b/ddl/placement/constraint_test.go new file mode 100644 index 0000000000000..4c97cb9d58f41 --- /dev/null +++ b/ddl/placement/constraint_test.go @@ -0,0 +1,259 @@ +// 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(&testConstraintSuite{}) + +type testConstraintSuite struct{} + +func (t *testConstraintSuite) TestNew(c *C) { + type TestCase struct { + name string + input string + label Constraint + err error + } + tests := []TestCase{ + { + name: "normal", + input: "+zone=bj", + label: Constraint{ + Key: "zone", + Op: In, + Values: []string{"bj"}, + }, + }, + { + name: "normal with spaces", + input: "- dc = sh ", + label: Constraint{ + Key: "dc", + Op: NotIn, + Values: []string{"sh"}, + }, + }, + { + name: "not tiflash", + input: "-engine = tiflash ", + label: Constraint{ + Key: "engine", + Op: NotIn, + Values: []string{"tiflash"}, + }, + }, + { + name: "disallow tiflash", + input: "+engine=Tiflash", + err: ErrUnsupportedConstraint, + }, + // invalid + { + name: "invalid length", + input: ",,,", + err: ErrInvalidConstraintFormat, + }, + { + name: "invalid, lack = 1", + input: "+ ", + err: ErrInvalidConstraintFormat, + }, + { + name: "invalid, lack = 2", + input: "+000", + err: ErrInvalidConstraintFormat, + }, + { + name: "invalid op", + input: "0000", + err: ErrInvalidConstraintFormat, + }, + { + name: "empty key 1", + input: "+ =zone1", + err: ErrInvalidConstraintFormat, + }, + { + name: "empty key 2", + input: "+ = z", + err: ErrInvalidConstraintFormat, + }, + { + name: "empty value 1", + input: "+zone=", + err: ErrInvalidConstraintFormat, + }, + { + name: "empty value 2", + input: "+z = ", + err: ErrInvalidConstraintFormat, + }, + } + + for _, t := range tests { + label, err := NewConstraint(t.input) + comment := Commentf("%s: %v", t.name, err) + if t.err == nil { + c.Assert(err, IsNil, comment) + c.Assert(label, DeepEquals, t.label, comment) + } else { + c.Assert(errors.Is(err, t.err), IsTrue, comment) + } + } +} + +func (t *testConstraintSuite) TestRestore(c *C) { + type TestCase struct { + name string + input Constraint + output string + err error + } + var tests []TestCase + + input, err := NewConstraint("+zone=bj") + c.Assert(err, IsNil) + tests = append(tests, TestCase{ + name: "normal, op in", + input: input, + output: "+zone=bj", + }) + + input, err = NewConstraint("+ zone = bj ") + c.Assert(err, IsNil) + tests = append(tests, TestCase{ + name: "normal with spaces, op in", + input: input, + output: "+zone=bj", + }) + + input, err = NewConstraint("- zone = bj ") + c.Assert(err, IsNil) + tests = append(tests, TestCase{ + name: "normal with spaces, op not in", + input: input, + output: "-zone=bj", + }) + + tests = append(tests, TestCase{ + name: "no values", + input: Constraint{ + Op: In, + Key: "dc", + Values: []string{}, + }, + err: ErrInvalidConstraintFormat, + }) + + tests = append(tests, TestCase{ + name: "multiple values", + input: Constraint{ + Op: In, + Key: "dc", + Values: []string{"dc1", "dc2"}, + }, + err: ErrInvalidConstraintFormat, + }) + + tests = append(tests, TestCase{ + name: "invalid op", + input: Constraint{ + Op: "[", + Key: "dc", + Values: []string{}, + }, + err: ErrInvalidConstraintFormat, + }) + + for _, t := range tests { + output, err := t.input.Restore() + comment := Commentf("%s: %v", t.name, err) + if t.err == nil { + c.Assert(err, IsNil, comment) + c.Assert(output, Equals, t.output, comment) + } else { + c.Assert(errors.Is(err, t.err), IsTrue, comment) + } + } +} + +func (t *testConstraintSuite) TestCompatibleWith(c *C) { + type TestCase struct { + name string + i1 Constraint + i2 Constraint + output ConstraintCompatibility + } + var tests []TestCase + + i1, err := NewConstraint("+zone=sh") + c.Assert(err, IsNil) + i2, err := NewConstraint("-zone=sh") + c.Assert(err, IsNil) + tests = append(tests, TestCase{ + "case 2", + i1, i2, + ConstraintIncompatible, + }) + + i1, err = NewConstraint("+zone=bj") + c.Assert(err, IsNil) + i2, err = NewConstraint("+zone=sh") + c.Assert(err, IsNil) + tests = append(tests, TestCase{ + "case 3", + i1, i2, + ConstraintIncompatible, + }) + + i1, err = NewConstraint("+zone=sh") + c.Assert(err, IsNil) + i2, err = NewConstraint("+zone=sh") + c.Assert(err, IsNil) + tests = append(tests, TestCase{ + "case 1", + i1, i2, + ConstraintDuplicated, + }) + + i1, err = NewConstraint("+zone=sh") + c.Assert(err, IsNil) + i2, err = NewConstraint("+dc=sh") + c.Assert(err, IsNil) + tests = append(tests, TestCase{ + "normal 1", + i1, i2, + ConstraintCompatible, + }) + + i1, err = NewConstraint("-zone=sh") + c.Assert(err, IsNil) + i2, err = NewConstraint("-zone=bj") + c.Assert(err, IsNil) + tests = append(tests, TestCase{ + "normal 2", + i1, i2, + ConstraintCompatible, + }) + + for _, t := range tests { + comment := Commentf("%s", t.name) + c.Assert(t.i1.CompatibleWith(&t.i2), Equals, t.output, comment) + } +} 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 new file mode 100644 index 0000000000000..19797022a609c --- /dev/null +++ b/ddl/placement/errors.go @@ -0,0 +1,27 @@ +// 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" +) + +var ( + // ErrInvalidConstraintFormat is from constraint.go. + 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 0f17cd2f7f659..3bb9da96e3890 100644 --- a/ddl/placement/types.go +++ b/ddl/placement/types.go @@ -15,9 +15,6 @@ package placement import ( "encoding/json" - "strings" - - "github.com/pingcap/errors" ) // Refer to https://github.com/tikv/pd/issues/2701 . @@ -39,64 +36,19 @@ const ( Learner PeerRoleType = "learner" ) -// LabelConstraintOp defines how a LabelConstraint matches a store. -type LabelConstraintOp string - -const ( - // In restricts the store label value should in the value list. - // If label does not exist, `in` is always false. - In LabelConstraintOp = "in" - // NotIn restricts the store label value should not in the value list. - // If label does not exist, `notIn` is always true. - NotIn LabelConstraintOp = "notIn" - // Exists restricts the store should have the label. - Exists LabelConstraintOp = "exists" - // NotExists restricts the store should not have the label. - NotExists LabelConstraintOp = "notExists" -) - -// LabelConstraint is used to filter store when trying to place peer of a region. -type LabelConstraint struct { - Key string `json:"key,omitempty"` - Op LabelConstraintOp `json:"op,omitempty"` - Values []string `json:"values,omitempty"` -} - -// Restore converts the LabelConstraint to a string. -func (c *LabelConstraint) Restore() (string, error) { - var sb strings.Builder - for i, value := range c.Values { - switch c.Op { - case In: - sb.WriteString("+") - case NotIn: - sb.WriteString("-") - default: - return "", errors.Errorf("Unsupported label constraint operation: %s", c.Op) - } - sb.WriteString(c.Key) - sb.WriteString("=") - sb.WriteString(value) - if i < len(c.Values)-1 { - sb.WriteString(",") - } - } - return sb.String(), nil -} - // Rule is the placement rule. Check https://github.com/tikv/pd/blob/master/server/schedule/placement/rule.go. type Rule struct { - GroupID string `json:"group_id"` - ID string `json:"id"` - Index int `json:"index,omitempty"` - Override bool `json:"override,omitempty"` - StartKeyHex string `json:"start_key"` - EndKeyHex string `json:"end_key"` - Role PeerRoleType `json:"role"` - Count int `json:"count"` - LabelConstraints []LabelConstraint `json:"label_constraints,omitempty"` - LocationLabels []string `json:"location_labels,omitempty"` - IsolationLevel string `json:"isolation_level,omitempty"` + GroupID string `json:"group_id"` + ID string `json:"id"` + Index int `json:"index,omitempty"` + Override bool `json:"override,omitempty"` + StartKeyHex string `json:"start_key"` + EndKeyHex string `json:"end_key"` + Role PeerRoleType `json:"role"` + Count int `json:"count"` + LabelConstraints Constraints `json:"label_constraints,omitempty"` + LocationLabels []string `json:"location_labels,omitempty"` + IsolationLevel string `json:"isolation_level,omitempty"` } // Clone is used to duplicate a RuleOp for safe modification. diff --git a/ddl/placement/utils.go b/ddl/placement/utils.go index 785235e8a5e77..16c0a424dde53 100644 --- a/ddl/placement/utils.go +++ b/ddl/placement/utils.go @@ -23,91 +23,6 @@ import ( "github.com/pingcap/tidb/util/codec" ) -func checkLabelConstraint(label string) (LabelConstraint, error) { - r := LabelConstraint{} - - if len(label) < 4 { - return r, errors.Errorf("label constraint should be in format '{+|-}key=value', but got '%s'", label) - } - - var op LabelConstraintOp - switch label[0] { - case '+': - op = In - case '-': - op = NotIn - default: - return r, errors.Errorf("label constraint should be in format '{+|-}key=value', but got '%s'", label) - } - - kv := strings.Split(label[1:], "=") - if len(kv) != 2 { - return r, errors.Errorf("label constraint should be in format '{+|-}key=value', but got '%s'", label) - } - - key := strings.TrimSpace(kv[0]) - if key == "" { - return r, errors.Errorf("label constraint should be in format '{+|-}key=value', but got '%s'", label) - } - - val := strings.TrimSpace(kv[1]) - if val == "" { - return r, errors.Errorf("label constraint should be in format '{+|-}key=value', but got '%s'", label) - } - - if op == In && key == EngineLabelKey && strings.ToLower(val) == EngineLabelTiFlash { - return r, errors.Errorf("unsupported label constraint '%s'", label) - } - - r.Key = key - r.Op = op - r.Values = []string{val} - return r, nil -} - -// CheckLabelConstraints will check labels, and build LabelConstraints for rule. -func CheckLabelConstraints(labels []string) ([]LabelConstraint, error) { - constraints := make([]LabelConstraint, 0, len(labels)) - for _, str := range labels { - label, err := checkLabelConstraint(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. @@ -121,24 +36,6 @@ func ObjectIDFromGroupID(groupID string) (int64, error) { return id, nil } -// RestoreLabelConstraintList converts the label constraints to a readable string. -func RestoreLabelConstraintList(constraints []LabelConstraint) (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 19b09dab995db..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 []LabelConstraint - expectedResult string - expectErr bool - }{ - { - constraints: []LabelConstraint{}, - expectedResult: ``, - }, - { - constraints: []LabelConstraint{ - { - Key: "zone", - Op: "in", - Values: []string{"bj"}, - }, - }, - expectedResult: `"+zone=bj"`, - }, - { - constraints: []LabelConstraint{ - { - Key: "zone", - Op: "notIn", - Values: []string{"bj"}, - }, - }, - expectedResult: `"-zone=bj"`, - }, - { - constraints: []LabelConstraint{ - { - Key: "zone", - Op: "exists", - Values: []string{"bj"}, - }, - }, - expectErr: true, - }, - { - constraints: []LabelConstraint{ - { - Key: "zone", - Op: "in", - Values: []string{"bj", "sh"}, - }, - }, - expectedResult: `"+zone=bj,+zone=sh"`, - }, - { - constraints: []LabelConstraint{ - { - Key: "zone", - Op: "in", - Values: []string{"bj", "sh"}, - }, - { - Key: "disk", - Op: "in", - Values: []string{"ssd"}, - }, - }, - expectedResult: `"+zone=bj,+zone=sh","+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 @@ -134,7 +58,7 @@ func (t *testUtilsSuite) TestGetLeaderDCByBundle(c *C) { { ID: "12", Role: Leader, - LabelConstraints: []LabelConstraint{ + LabelConstraints: []Constraint{ { Key: "zone", Op: In, @@ -160,7 +84,7 @@ func (t *testUtilsSuite) TestGetLeaderDCByBundle(c *C) { { ID: "12", Role: Voter, - LabelConstraints: []LabelConstraint{ + LabelConstraints: []Constraint{ { Key: "zone", Op: In, @@ -186,7 +110,7 @@ func (t *testUtilsSuite) TestGetLeaderDCByBundle(c *C) { { ID: "11", Role: Leader, - LabelConstraints: []LabelConstraint{ + LabelConstraints: []Constraint{ { Key: "zone", Op: In, @@ -203,7 +127,7 @@ func (t *testUtilsSuite) TestGetLeaderDCByBundle(c *C) { { ID: "12", Role: Voter, - LabelConstraints: []LabelConstraint{ + LabelConstraints: []Constraint{ { Key: "zone", Op: In, @@ -229,7 +153,7 @@ func (t *testUtilsSuite) TestGetLeaderDCByBundle(c *C) { { ID: "11", Role: Leader, - LabelConstraints: []LabelConstraint{ + LabelConstraints: []Constraint{ { Key: "fake", Op: In, @@ -255,7 +179,7 @@ func (t *testUtilsSuite) TestGetLeaderDCByBundle(c *C) { { ID: "11", Role: Leader, - LabelConstraints: []LabelConstraint{ + LabelConstraints: []Constraint{ { Key: "zone", Op: NotIn, @@ -281,7 +205,7 @@ func (t *testUtilsSuite) TestGetLeaderDCByBundle(c *C) { { ID: "11", Role: Leader, - LabelConstraints: []LabelConstraint{ + LabelConstraints: []Constraint{ { Key: "zone", Op: In, diff --git a/ddl/placement_rule_test.go b/ddl/placement_rule_test.go index 16450b85dfc7e..b051092a776e9 100644 --- a/ddl/placement_rule_test.go +++ b/ddl/placement_rule_test.go @@ -52,33 +52,13 @@ func (s *testPlacementSuite) TestPlacementBuild(c *C) { { Role: placement.Voter, Count: 3, - LabelConstraints: []placement.LabelConstraint{ + LabelConstraints: []placement.Constraint{ {Key: "zone", Op: "in", Values: []string{"sh"}}, }, }, }, }, - { - 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, @@ -89,28 +69,8 @@ func (s *testPlacementSuite) TestPlacementBuild(c *C) { output: []*placement.Rule{{ Role: placement.Voter, Count: 3, - LabelConstraints: []placement.LabelConstraint{}, - }}, - }, - - { - 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"]`, + LabelConstraints: []placement.Constraint{}, }}, - err: ".*conflicting constraints.*", }, { @@ -123,38 +83,20 @@ func (s *testPlacementSuite) TestPlacementBuild(c *C) { { Role: placement.Voter, Count: 1, - LabelConstraints: []placement.LabelConstraint{ + LabelConstraints: []placement.Constraint{ {Key: "zone", Op: "in", Values: []string{"sh"}}, }, }, { Role: placement.Voter, Count: 2, - LabelConstraints: []placement.LabelConstraint{ + LabelConstraints: []placement.Constraint{ {Key: "zone", Op: "in", Values: []string{"sh"}}, }, }, }, }, - { - 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, @@ -166,7 +108,7 @@ func (s *testPlacementSuite) TestPlacementBuild(c *C) { { Role: placement.Voter, Count: 3, - LabelConstraints: []placement.LabelConstraint{ + LabelConstraints: []placement.Constraint{ {Key: "zone", Op: "notIn", Values: []string{"sh"}}, {Key: "zone", Op: "notIn", Values: []string{"bj"}}, }, @@ -185,7 +127,7 @@ func (s *testPlacementSuite) TestPlacementBuild(c *C) { { Role: placement.Voter, Count: 3, - LabelConstraints: []placement.LabelConstraint{ + LabelConstraints: []placement.Constraint{ {Key: "zone", Op: "in", Values: []string{"sh"}}, {Key: "zone", Op: "notIn", Values: []string{"bj"}}, }, @@ -212,7 +154,7 @@ func (s *testPlacementSuite) TestPlacementBuild(c *C) { { Role: placement.Voter, Count: 3, - LabelConstraints: []placement.LabelConstraint{ + LabelConstraints: []placement.Constraint{ {Key: "zone", Op: "in", Values: []string{"sh"}}, {Key: "zone", Op: "notIn", Values: []string{"bj"}}, }, @@ -220,7 +162,7 @@ func (s *testPlacementSuite) TestPlacementBuild(c *C) { { Role: placement.Follower, Count: 2, - LabelConstraints: []placement.LabelConstraint{ + LabelConstraints: []placement.Constraint{ {Key: "zone", Op: "notIn", Values: []string{"sh"}}, {Key: "zone", Op: "in", Values: []string{"bj"}}, }, @@ -247,7 +189,7 @@ func (s *testPlacementSuite) TestPlacementBuild(c *C) { { Role: placement.Voter, Count: 2, - LabelConstraints: []placement.LabelConstraint{ + LabelConstraints: []placement.Constraint{ {Key: "zone", Op: "notIn", Values: []string{"sh"}}, {Key: "zone", Op: "in", Values: []string{"bj"}}, }, @@ -274,12 +216,12 @@ func (s *testPlacementSuite) TestPlacementBuild(c *C) { { Role: placement.Voter, Count: 1, - LabelConstraints: []placement.LabelConstraint{{Key: "zone", Op: "notIn", Values: []string{"sh"}}}, + LabelConstraints: []placement.Constraint{{Key: "zone", Op: "notIn", Values: []string{"sh"}}}, }, { Role: placement.Voter, Count: 1, - LabelConstraints: []placement.LabelConstraint{{Key: "zone", Op: "in", Values: []string{"bj"}}}, + LabelConstraints: []placement.Constraint{{Key: "zone", Op: "in", Values: []string{"bj"}}}, }, { Role: placement.Voter, @@ -364,7 +306,7 @@ func (s *testPlacementSuite) TestPlacementBuild(c *C) { { Role: placement.Voter, Count: 3, - LabelConstraints: []placement.LabelConstraint{ + LabelConstraints: []placement.Constraint{ {Key: "zone", Op: "in", Values: []string{"sh"}}, {Key: "zone", Op: "notIn", Values: []string{"bj"}}, }, diff --git a/ddl/placement_sql_test.go b/ddl/placement_sql_test.go index 91c48528a986a..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)") @@ -473,7 +404,7 @@ PARTITION BY RANGE (c) ( GroupID: groupID, Role: placement.Leader, Count: 1, - LabelConstraints: []placement.LabelConstraint{ + LabelConstraints: []placement.Constraint{ { Key: placement.DCLabelKey, Op: placement.In, @@ -492,7 +423,7 @@ PARTITION BY RANGE (c) ( GroupID: groupID, Role: placement.Follower, Count: 3, - LabelConstraints: []placement.LabelConstraint{ + LabelConstraints: []placement.Constraint{ { Key: placement.DCLabelKey, Op: placement.In, @@ -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") } } @@ -688,7 +619,7 @@ PARTITION BY RANGE (c) ( GroupID: groupID, Role: placement.Leader, Count: 1, - LabelConstraints: []placement.LabelConstraint{ + LabelConstraints: []placement.Constraint{ { Key: placement.DCLabelKey, Op: placement.In, @@ -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/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/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/errno/errcode.go b/errno/errcode.go index cfe4259189259..2ed488242dd10 100644 --- a/errno/errcode.go +++ b/errno/errcode.go @@ -852,6 +852,7 @@ const ( ErrInvalidEncryptionOption = 3184 ErrTooLongValueForType = 3505 ErrPKIndexCantBeInvisible = 3522 + ErrGrantRole = 3523 ErrRoleNotGranted = 3530 ErrLockAcquireFailAndNoWaitSet = 3572 ErrWindowNoSuchWindow = 3579 diff --git a/errno/errname.go b/errno/errname.go index c5d52919118e3..5afdbbb91c4c0 100644 --- a/errno/errname.go +++ b/errno/errname.go @@ -399,6 +399,7 @@ var MySQLErrName = map[uint16]*mysql.ErrMessage{ ErrViewNoInsertFieldList: mysql.Message("Can not insert into join view '%-.192s.%-.192s' without fields list", nil), ErrViewDeleteMergeView: mysql.Message("Can not delete from join view '%-.192s.%-.192s'", nil), ErrCannotUser: mysql.Message("Operation %s failed for %.256s", nil), + ErrGrantRole: mysql.Message("Unknown authorization ID %.256s", nil), ErrXaerNota: mysql.Message("XAERNOTA: Unknown XID", nil), ErrXaerInval: mysql.Message("XAERINVAL: Invalid arguments (or unsupported command)", nil), ErrXaerRmfail: mysql.Message("XAERRMFAIL: The command cannot be executed when global transaction is in the %.64s state", nil), diff --git a/errors.toml b/errors.toml index ebed6e4054799..f09e33605293d 100644 --- a/errors.toml +++ b/errors.toml @@ -541,6 +541,11 @@ error = ''' The password hash doesn't have the expected format. Check if the correct password algorithm is being used with the PASSWORD() function. ''' +["executor:3523"] +error = ''' +Unknown authorization ID %.256s +''' + ["executor:3619"] error = ''' Illegal privilege level specified for %s 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.go b/executor/distsql.go index 385cf2cdd219f..1b65cee31b6f6 100644 --- a/executor/distsql.go +++ b/executor/distsql.go @@ -742,11 +742,11 @@ func (w *indexWorker) fetchHandles(ctx context.Context, result distsql.SelectRes func (w *indexWorker) extractTaskHandles(ctx context.Context, chk *chunk.Chunk, idxResult distsql.SelectResult, count uint64) ( handles []kv.Handle, retChk *chunk.Chunk, scannedKeys uint64, err error) { - var handleOffset []int numColsWithoutPid := chk.NumCols() if w.idxLookup.index.Global { numColsWithoutPid = numColsWithoutPid - 1 } + handleOffset := make([]int, 0, len(w.idxLookup.handleCols)) for i := range w.idxLookup.handleCols { handleOffset = append(handleOffset, numColsWithoutPid-len(w.idxLookup.handleCols)+i) } 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/errors.go b/executor/errors.go index 7793639249f5d..7f3345659e4f9 100644 --- a/executor/errors.go +++ b/executor/errors.go @@ -33,6 +33,7 @@ var ( ErrCantCreateUserWithGrant = dbterror.ClassExecutor.NewStd(mysql.ErrCantCreateUserWithGrant) ErrPasswordNoMatch = dbterror.ClassExecutor.NewStd(mysql.ErrPasswordNoMatch) ErrCannotUser = dbterror.ClassExecutor.NewStd(mysql.ErrCannotUser) + ErrGrantRole = dbterror.ClassExecutor.NewStd(mysql.ErrGrantRole) ErrPasswordFormat = dbterror.ClassExecutor.NewStd(mysql.ErrPasswordFormat) ErrCantChangeTxCharacteristics = dbterror.ClassExecutor.NewStd(mysql.ErrCantChangeTxCharacteristics) ErrPsManyParam = dbterror.ClassExecutor.NewStd(mysql.ErrPsManyParam) 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_test.go b/executor/executor_test.go index b03b90435a5e2..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") } @@ -8249,3 +8276,15 @@ func (s *testSuite) TestIssue23609(c *C) { tk.MustQuery("select * from t1 where a < b").Check(testkit.Rows()) c.Assert(tk.Se.GetSessionVars().StmtCtx.WarningCount(), Equals, uint16(0)) } + +func (s *testSuite1) TestIssue24091(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test;") + tk.MustExec("drop table if exists t;") + defer tk.MustExec("drop table if exists t;") + tk.MustExec("create table t(a int) partition by hash (a div 0) partitions 10;") + tk.MustExec("insert into t values (NULL);") + + tk.MustQuery("select null div 0;").Check(testkit.Rows("")) + tk.MustQuery("select * from t;").Check(testkit.Rows("")) +} diff --git a/executor/index_merge_reader.go b/executor/index_merge_reader.go index c1e780fc26e45..b1088cd22600f 100644 --- a/executor/index_merge_reader.go +++ b/executor/index_merge_reader.go @@ -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 c9f887d4f7d36..33db7c9fdd18b 100644 --- a/executor/infoschema_reader.go +++ b/executor/infoschema_reader.go @@ -1059,8 +1059,8 @@ func (e *memtableRetriever) setDataFromEngines() { } func (e *memtableRetriever) setDataFromCharacterSets() { - var rows [][]types.Datum charsets := charset.GetSupportedCharsets() + var rows = make([][]types.Datum, 0, len(charsets)) for _, charset := range charsets { rows = append(rows, types.MakeDatums(charset.Name, charset.DefaultCollation, charset.Desc, charset.Maxlen), @@ -1070,8 +1070,8 @@ func (e *memtableRetriever) setDataFromCharacterSets() { } func (e *memtableRetriever) setDataFromCollations() { - var rows [][]types.Datum collations := collate.GetSupportedCollations() + var rows = make([][]types.Datum, 0, len(collations)) for _, collation := range collations { isDefault := "" if collation.IsDefault { @@ -1085,8 +1085,8 @@ func (e *memtableRetriever) setDataFromCollations() { } func (e *memtableRetriever) dataForCollationCharacterSetApplicability() { - var rows [][]types.Datum collations := collate.GetSupportedCollations() + var rows = make([][]types.Datum, 0, len(collations)) for _, collation := range collations { rows = append(rows, types.MakeDatums(collation.Name, collation.CharsetName), @@ -1188,12 +1188,12 @@ func (e *memtableRetriever) setDataFromUserPrivileges(ctx sessionctx.Context) { } func (e *memtableRetriever) setDataForMetricTables(ctx sessionctx.Context) { - var rows [][]types.Datum tables := make([]string, 0, len(infoschema.MetricTableMap)) for name := range infoschema.MetricTableMap { tables = append(tables, name) } sort.Strings(tables) + rows := make([][]types.Datum, 0, len(tables)) for _, name := range tables { schema := infoschema.MetricTableMap[name] record := types.MakeDatums( @@ -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/inspection_result.go b/executor/inspection_result.go index b0350ae2890d5..4b8c4cb9753e0 100644 --- a/executor/inspection_result.go +++ b/executor/inspection_result.go @@ -856,14 +856,14 @@ func (thresholdCheckInspection) inspectThreshold1(ctx context.Context, sctx sess sql.Reset() if len(rule.configKey) > 0 { - fmt.Fprintf(sql, `select t1.status_address, t1.cpu, (t2.value * %[2]f) as threshold, t2.value from - (select status_address, max(sum_value) as cpu from (select instance as status_address, sum(value) as sum_value from metrics_schema.tikv_thread_cpu %[4]s and name like '%[1]s' group by instance, time) as tmp group by tmp.status_address) as t1 join - (select instance, value from information_schema.cluster_config where type='tikv' and %[5]s = '%[3]s') as t2 join + fmt.Fprintf(sql, `select t1.status_address, t1.cpu, (t2.value * %[2]f) as threshold, t2.value from + (select status_address, max(sum_value) as cpu from (select instance as status_address, sum(value) as sum_value from metrics_schema.tikv_thread_cpu %[4]s and name like '%[1]s' group by instance, time) as tmp group by tmp.status_address) as t1 join + (select instance, value from information_schema.cluster_config where type='tikv' and %[5]s = '%[3]s') as t2 join (select instance,status_address from information_schema.cluster_info where type='tikv') as t3 on t1.status_address=t3.status_address and t2.instance=t3.instance where t1.cpu > (t2.value * %[2]f)`, rule.component, rule.threshold, rule.configKey, condition, "`key`") } else { - fmt.Fprintf(sql, `select t1.instance, t1.cpu, %[2]f from - (select instance, max(value) as cpu from metrics_schema.tikv_thread_cpu %[3]s and name like '%[1]s' group by instance) as t1 + fmt.Fprintf(sql, `select t1.instance, t1.cpu, %[2]f from + (select instance, max(value) as cpu from metrics_schema.tikv_thread_cpu %[3]s and name like '%[1]s' group by instance) as t1 where t1.cpu > %[2]f;`, rule.component, rule.threshold, condition) } stmt, err := exec.ParseWithParams(ctx, sql.String()) diff --git a/executor/inspection_result_test.go b/executor/inspection_result_test.go index 90714fc8ba00d..c12fac1370b3a 100644 --- a/executor/inspection_result_test.go +++ b/executor/inspection_result_test.go @@ -479,7 +479,7 @@ func (s *inspectionResultSuite) TestCriticalErrorInspection(c *C) { } }() - var servers []string + var servers = make([]string, 0, len(testServers)) for _, s := range testServers { servers = append(servers, strings.Join([]string{s.typ, s.address, s.address}, ",")) } diff --git a/executor/join.go b/executor/join.go index 472df41ded441..47f1965579ba9 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 } diff --git a/executor/memtable_reader.go b/executor/memtable_reader.go index a5d39d1c7b21d..24e2001131580 100644 --- a/executor/memtable_reader.go +++ b/executor/memtable_reader.go @@ -395,8 +395,8 @@ func getServerInfoByGRPC(ctx context.Context, address string, tp diagnosticspb.S } func parseFailpointServerInfo(s string) []infoschema.ServerInfo { - var serversInfo []infoschema.ServerInfo servers := strings.Split(s, ";") + serversInfo := make([]infoschema.ServerInfo, 0, len(servers)) for _, server := range servers { parts := strings.Split(server, ",") serversInfo = append(serversInfo, infoschema.ServerInfo{ @@ -494,7 +494,7 @@ func (e *clusterLogRetriever) initialize(ctx context.Context, sctx sessionctx.Co nodeTypes := e.extractor.NodeTypes serversInfo = filterClusterServerInfo(serversInfo, nodeTypes, instances) - var levels []diagnosticspb.LogLevel + var levels = make([]diagnosticspb.LogLevel, 0, len(e.extractor.LogLevels)) for l := range e.extractor.LogLevels { levels = append(levels, sysutil.ParseLogLevel(l)) } diff --git a/executor/memtable_reader_test.go b/executor/memtable_reader_test.go index efd7f331cb0e6..53607dd80ef03 100644 --- a/executor/memtable_reader_test.go +++ b/executor/memtable_reader_test.go @@ -879,7 +879,7 @@ func (s *testMemTableReaderSuite) TestTiDBClusterLog(c *C) { }, } - var servers []string + var servers = make([]string, 0, len(testServers)) for _, s := range testServers { servers = append(servers, strings.Join([]string{s.typ, s.address, s.address}, ",")) } 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/sample.go b/executor/sample.go index 6deec4308fa0a..d9c2097a9d3cd 100644 --- a/executor/sample.go +++ b/executor/sample.go @@ -208,11 +208,11 @@ func splitIntoMultiRanges(store kv.Storage, startKey, endKey kv.Key) ([]kv.KeyRa maxSleep := 10000 // ms bo := tikv.NewBackofferWithVars(context.Background(), maxSleep, nil) - var ranges []kv.KeyRange regions, err := s.GetRegionCache().LoadRegionsInKeyRange(bo, startKey, endKey) if err != nil { return nil, errors.Trace(err) } + var ranges = make([]kv.KeyRange, 0, len(regions)) for _, r := range regions { start, end := r.StartKey(), r.EndKey() if kv.Key(start).Cmp(startKey) < 0 { 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_test.go b/executor/set_test.go index a4b92a5a6f35d..42f7dce134e9c 100644 --- a/executor/set_test.go +++ b/executor/set_test.go @@ -397,6 +397,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_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 820bb50415bc5..a490c694dea25 100644 --- a/executor/simple.go +++ b/executor/simple.go @@ -980,7 +980,7 @@ func (e *SimpleExec) executeGrantRole(s *ast.GrantRoleStmt) error { return err } if !exists { - return ErrCannotUser.GenWithStackByArgs("GRANT ROLE", role.String()) + return ErrGrantRole.GenWithStackByArgs(role.String()) } } for _, user := range s.Users { diff --git a/executor/simple_test.go b/executor/simple_test.go index 3399beb680798..8c09fc336f5d2 100644 --- a/executor/simple_test.go +++ b/executor/simple_test.go @@ -872,3 +872,15 @@ func (s *testSuite3) TestIssue17247(c *C) { _, err := tk1.Exec("ALTER USER USER() IDENTIFIED BY PASSWORD '*B50FBDB37F1256824274912F2A1CE648082C3F1F'") c.Assert(err, NotNil) } + +// Close issue #23649. +// See https://github.com/pingcap/tidb/issues/23649 +func (s *testSuite3) TestIssue23649(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("DROP USER IF EXISTS issue23649;") + tk.MustExec("CREATE USER issue23649;") + _, err := tk.Exec("GRANT bogusrole to issue23649;") + c.Assert(err.Error(), Equals, "[executor:3523]Unknown authorization ID `bogusrole`@`%`") + _, err = tk.Exec("GRANT bogusrole to nonexisting;") + c.Assert(err.Error(), Equals, "[executor:3523]Unknown authorization ID `bogusrole`@`%`") +} 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/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/builtin_arithmetic.go b/expression/builtin_arithmetic.go index c1a50abc5039e..57bb98ff40051 100644 --- a/expression/builtin_arithmetic.go +++ b/expression/builtin_arithmetic.go @@ -780,20 +780,19 @@ func (s *builtinArithmeticIntDivideIntSig) evalInt(row chunk.Row) (int64, bool, } func (s *builtinArithmeticIntDivideIntSig) evalIntWithCtx(sctx sessionctx.Context, row chunk.Row) (int64, bool, error) { - b, isNull, err := s.args[1].EvalInt(sctx, row) - if isNull || err != nil { - return 0, isNull, err + b, bIsNull, err := s.args[1].EvalInt(sctx, row) + if bIsNull || err != nil { + return 0, bIsNull, err + } + a, aIsNull, err := s.args[0].EvalInt(sctx, row) + if aIsNull || err != nil { + return 0, aIsNull, err } if b == 0 { return 0, true, handleDivisionByZeroError(sctx) } - a, isNull, err := s.args[0].EvalInt(sctx, row) - if isNull || err != nil { - return 0, isNull, err - } - var ( ret int64 val uint64 diff --git a/expression/integration_test.go b/expression/integration_test.go index 6a863561fdebb..ffe3f32d43a1f 100644 --- a/expression/integration_test.go +++ b/expression/integration_test.go @@ -8744,7 +8744,7 @@ PARTITION BY RANGE (c) ( GroupID: groupID, Role: placement.Leader, Count: 1, - LabelConstraints: []placement.LabelConstraint{ + LabelConstraints: []placement.Constraint{ { Key: placement.DCLabelKey, Op: placement.In, @@ -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/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/infoschema/tables.go b/infoschema/tables.go index 4021c2844ad9c..085bc6a96a77d 100644 --- a/infoschema/tables.go +++ b/infoschema/tables.go @@ -1453,11 +1453,11 @@ func GetTiDBServerInfo(ctx sessionctx.Context) ([]ServerInfo, error) { if err != nil { return nil, errors.Trace(err) } - var servers []ServerInfo var isDefaultVersion bool if len(config.GetGlobalConfig().ServerVersion) == 0 { isDefaultVersion = true } + var servers = make([]ServerInfo, 0, len(tidbNodes)) for _, node := range tidbNodes { servers = append(servers, ServerInfo{ ServerType: "tidb", @@ -1504,11 +1504,11 @@ func GetPDServerInfo(ctx sessionctx.Context) ([]ServerInfo, error) { if !ok { return nil, errors.Errorf("%T not an etcd backend", store) } - var servers []ServerInfo members, err := etcd.EtcdAddrs() if err != nil { return nil, errors.Trace(err) } + var servers = make([]ServerInfo, 0, len(members)) for _, addr := range members { // Get PD version url := fmt.Sprintf("%s://%s%s", util.InternalHTTPSchema(), addr, pdapi.ClusterVersion) diff --git a/infoschema/tables_test.go b/infoschema/tables_test.go index 3858aece8d96b..f30f25ba6abfa 100644 --- a/infoschema/tables_test.go +++ b/infoschema/tables_test.go @@ -1442,7 +1442,7 @@ func (s *testTableSuite) TestPlacementPolicy(c *C) { ID: "0", Role: "voter", Count: 3, - LabelConstraints: []placement.LabelConstraint{ + LabelConstraints: []placement.Constraint{ { Key: "zone", Op: "in", diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index 0ffd8937950ae..d4f2923b6220a 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -1965,6 +1965,9 @@ func (lt *LogicalTopN) getPhysTopN(prop *property.PhysicalProperty) []PhysicalPl if !lt.limitHints.preferLimitToCop { allTaskTypes = append(allTaskTypes, property.RootTaskType) } + if lt.ctx.GetSessionVars().AllowMPPExecution { + allTaskTypes = append(allTaskTypes, property.MppTaskType) + } ret := make([]PhysicalPlan, 0, len(allTaskTypes)) for _, tp := range allTaskTypes { resultProp := &property.PhysicalProperty{TaskTp: tp, ExpectedCnt: math.MaxFloat64} @@ -2041,7 +2044,7 @@ func (la *LogicalApply) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([ } disableAggPushDownToCop(la.children[0]) join := la.GetHashJoin(prop) - var columns []*expression.Column + var columns = make([]*expression.Column, 0, len(la.CorCols)) for _, colColumn := range la.CorCols { columns = append(columns, &colColumn.Column) } 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 fdaa3adfe0912..d861db8548f31 100644 --- a/planner/core/integration_test.go +++ b/planner/core/integration_test.go @@ -358,6 +358,8 @@ func (s *testIntegrationSerialSuite) TestSelPushDownTiFlash(c *C) { } tk.MustExec("set @@session.tidb_isolation_read_engines = 'tiflash'") + tk.MustExec("set @@session.tidb_allow_mpp = 0") + var input []string var output []struct { SQL string @@ -445,6 +447,7 @@ func (s *testIntegrationSerialSuite) TestPushDownToTiFlashWithKeepOrder(c *C) { } tk.MustExec("set @@session.tidb_isolation_read_engines = 'tiflash'") + tk.MustExec("set @@session.tidb_allow_mpp = 0") var input []string var output []struct { SQL string @@ -1688,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()) @@ -1704,44 +1706,217 @@ func (s *testIntegrationSerialSuite) TestIndexMerge(c *C) { tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t (a int, b int, unique key(a), unique key(b))") - tk.MustQuery("desc format='brief' select /*+ use_index_merge(t) */ * from t where a =1 or (b=1 and b+2>1)").Check(testkit.Rows( - "Projection 8000.00 root test.t.a, test.t.b", - "└─IndexMerge 2.00 root ", - " ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t, index:a(a) range:[1,1], keep order:false, stats:pseudo", - " ├─Selection(Build) 0.80 cop[tikv] 1", - " │ └─IndexRangeScan 1.00 cop[tikv] table:t, index:b(b) range:[1,1], keep order:false, stats:pseudo", - " └─TableRowIDScan(Probe) 2.00 cop[tikv] table:t keep order:false, stats:pseudo", - )) - tk.MustQuery("show warnings").Check(testkit.Rows()) + tk.MustExec("insert into t value (1, 5), (2, 4), (3, 3), (4, 2), (5, 1)") + tk.MustExec("insert into t value (6, 0), (7, -1), (8, -2), (9, -3), (10, -4)") + tk.MustExec("analyze table t") - tk.MustQuery("desc format='brief' select /*+ use_index_merge(t) */ * from t where a =1 or (b=1 and length(b)=1)").Check(testkit.Rows( - "Projection 1.80 root test.t.a, test.t.b", - "└─IndexMerge 2.00 root ", - " ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t, index:a(a) range:[1,1], keep order:false, stats:pseudo", - " ├─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, stats:pseudo", - " └─TableRowIDScan(Probe) 2.00 cop[tikv] table:t keep order:false, stats:pseudo")) - tk.MustQuery("show warnings").Check(testkit.Rows()) + var input []string + var output []struct { + SQL string + Plan []string + Warnings []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(tt).Rows()) + output[i].Warnings = s.testData.ConvertRowsToStrings(tk.MustQuery("show warnings").Rows()) + }) + tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) + tk.MustQuery("show warnings").Check(testkit.Rows(output[i].Warnings...)) + } +} - tk.MustQuery("desc format='brief' select /*+ use_index_merge(t) */ * from t where (a=1 and length(a)=1) or (b=1 and length(b)=1)").Check(testkit.Rows( - "Projection 1.60 root test.t.a, test.t.b", - "└─IndexMerge 2.00 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, stats:pseudo", - " ├─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, stats:pseudo", - " └─TableRowIDScan(Probe) 2.00 cop[tikv] table:t keep order:false, stats:pseudo")) - tk.MustQuery("show warnings").Check(testkit.Rows()) +func (s *testIntegrationSerialSuite) TestIndexMergePartialScansClusteredIndex(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test;") - tk.MustQuery("desc format='brief' select /*+ use_index_merge(t) */ * from t where (a=1 and length(b)=1) or (b=1 and length(a)=1)").Check(testkit.Rows( - "Projection 1.60 root test.t.a, test.t.b", - "└─IndexMerge 0.00 root ", - " ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t, index:a(a) range:[1,1], keep order:false, stats:pseudo", - " ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t, index:b(b) range:[1,1], keep order:false, stats:pseudo", - " └─Selection(Probe) 0.00 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 2.00 cop[tikv] table:t keep order:false, stats:pseudo", - )) - tk.MustQuery("show warnings").Check(testkit.Rows()) + 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) { @@ -1750,12 +1925,11 @@ func (s *testIntegrationSerialSuite) TestIssue16407(c *C) { 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()) @@ -1985,10 +2159,10 @@ func (s *testIntegrationSuite) TestAccessPathOnClusterIndex(c *C) { for i, tt := range input { s.testData.OnRecord(func() { output[i].SQL = tt - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery("explain " + tt).Rows()) + output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery("explain format='brief' " + tt).Rows()) output[i].Res = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Sort().Rows()) }) - tk.MustQuery("explain " + tt).Check(testkit.Rows(output[i].Plan...)) + tk.MustQuery("explain format='brief' " + tt).Check(testkit.Rows(output[i].Plan...)) tk.MustQuery(tt).Sort().Check(testkit.Rows(output[i].Res...)) } } @@ -3050,7 +3224,7 @@ func (s *testIntegrationSerialSuite) TestMppJoinDecimal(c *C) { } } -func (s *testIntegrationSerialSuite) TestMppAggWithJoin(c *C) { +func (s *testIntegrationSerialSuite) TestMppAggTopNWithJoin(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t") @@ -3110,6 +3284,31 @@ func (s *testIntegrationSerialSuite) TestLimitIndexLookUpKeepOrder(c *C) { } } +// Apply operator may got panic because empty Projection is eliminated. +func (s *testIntegrationSerialSuite) TestIssue23887(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);") + tk.MustExec("insert into t values(1, 2), (3, 4);") + var input []string + var output []struct { + SQL string + Plan []string + Res []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()) + output[i].Res = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Sort().Rows()) + }) + tk.MustQuery("explain format = 'brief' " + tt).Check(testkit.Rows(output[i].Plan...)) + tk.MustQuery(tt).Sort().Check(testkit.Rows(output[i].Res...)) + } +} + func (s *testIntegrationSuite) TestDecorrelateInnerJoinInSubquery(c *C) { tk := testkit.NewTestKit(c, s.store) @@ -3140,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", @@ -3329,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/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 5ca2600b733d1..7b66ce1c2eb87 100644 --- a/planner/core/rule_column_pruning.go +++ b/planner/core/rule_column_pruning.go @@ -71,6 +71,18 @@ func (p *LogicalProjection) PruneColumns(parentUsedCols []*expression.Column) er p.Exprs = append(p.Exprs[:i], p.Exprs[i+1:]...) } } + // Here we add a constant 1 to avoid Projection operator is eliminated. (#23887) + if len(p.Exprs) == 0 { + constOne := expression.NewOne() + p.schema.Append(&expression.Column{ + UniqueID: p.ctx.GetSessionVars().AllocPlanColumnID(), + RetType: constOne.GetType(), + }) + p.Exprs = append(p.Exprs, &expression.Constant{ + Value: constOne.Value, + RetType: constOne.GetType(), + }) + } selfUsedCols := make([]*expression.Column, 0, len(p.Exprs)) selfUsedCols = expression.ExtractColumnsFromExpressions(selfUsedCols, p.Exprs, nil) return child.PruneColumns(selfUsedCols) @@ -377,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 { @@ -387,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_eliminate_projection.go b/planner/core/rule_eliminate_projection.go index 5731495f9c2d2..3d3b550f4506d 100644 --- a/planner/core/rule_eliminate_projection.go +++ b/planner/core/rule_eliminate_projection.go @@ -60,9 +60,6 @@ func canProjectionBeEliminatedStrict(p *PhysicalProjection) bool { if p.CalculateNoDelay { return false } - if p.Schema().Len() == 0 { - return true - } child := p.Children()[0] if p.Schema().Len() != child.Schema().Len() { return false 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/stats.go b/planner/core/stats.go index 2e2d62fe9b714..0d81105d46c8b 100644 --- a/planner/core/stats.go +++ b/planner/core/stats.go @@ -455,7 +455,11 @@ func (ds *DataSource) generateIndexMergeOrPaths() error { accessConds := make([]expression.Expression, 0, len(partialPaths)) for _, p := range partialPaths { - accessConds = append(accessConds, p.AccessConds...) + indexCondsForP := p.AccessConds[:] + indexCondsForP = append(indexCondsForP, p.IndexFilters...) + if len(indexCondsForP) > 0 { + accessConds = append(accessConds, expression.ComposeCNFCondition(ds.ctx, indexCondsForP...)) + } } accessDNF := expression.ComposeDNFCondition(ds.ctx, accessConds...) sel, _, err := ds.tableStats.HistColl.Selectivity(ds.ctx, []expression.Expression{accessDNF}, nil) diff --git a/planner/core/task.go b/planner/core/task.go index 2ca6647506eca..68b6c627e9165 100644 --- a/planner/core/task.go +++ b/planner/core/task.go @@ -1164,12 +1164,12 @@ func (p *PhysicalTopN) GetCost(count float64, isRoot bool) float64 { } // canPushDown checks if this topN can be pushed down. If each of the expression can be converted to pb, it can be pushed. -func (p *PhysicalTopN) canPushDown(cop *copTask) bool { +func (p *PhysicalTopN) canPushDown(storeTp kv.StoreType) bool { exprs := make([]expression.Expression, 0, len(p.ByItems)) for _, item := range p.ByItems { exprs = append(exprs, item.Expr) } - return expression.CanExprsPushDown(p.ctx.GetSessionVars().StmtCtx, exprs, p.ctx.GetClient(), cop.getStoreType()) + return expression.CanExprsPushDown(p.ctx.GetSessionVars().StmtCtx, exprs, p.ctx.GetClient(), storeTp) } func (p *PhysicalTopN) allColsFromSchema(schema *expression.Schema) bool { @@ -1240,7 +1240,7 @@ func (p *PhysicalTopN) getPushedDownTopN(childPlan PhysicalPlan) *PhysicalTopN { func (p *PhysicalTopN) attach2Task(tasks ...task) task { t := tasks[0].copy() inputCount := t.count() - if copTask, ok := t.(*copTask); ok && p.canPushDown(copTask) && len(copTask.rootTaskConds) == 0 { + if copTask, ok := t.(*copTask); ok && p.canPushDown(copTask.getStoreType()) && len(copTask.rootTaskConds) == 0 { // If all columns in topN are from index plan, we push it to index plan, otherwise we finish the index plan and // push it to table plan. var pushedDownTopN *PhysicalTopN @@ -1253,6 +1253,9 @@ func (p *PhysicalTopN) attach2Task(tasks ...task) task { copTask.tablePlan = pushedDownTopN } copTask.addCost(pushedDownTopN.GetCost(inputCount, false)) + } else if mppTask, ok := t.(*mppTask); ok && p.canPushDown(kv.TiFlash) { + pushedDownTopN := p.getPushedDownTopN(mppTask.p) + mppTask.p = pushedDownTopN } rootTask := t.convertToRootTask(p.ctx) rootTask.addCost(p.GetCost(rootTask.count(), true)) diff --git a/planner/core/testdata/integration_serial_suite_in.json b/planner/core/testdata/integration_serial_suite_in.json index 39e3bb4992c2e..6026fdadd7976 100644 --- a/planner/core/testdata/integration_serial_suite_in.json +++ b/planner/core/testdata/integration_serial_suite_in.json @@ -264,7 +264,7 @@ ] }, { - "name": "TestMppAggWithJoin", + "name": "TestMppAggTopNWithJoin", "cases": [ "desc format = 'brief' select * from t join ( select count(*), id from t group by id) as A on A.id = t.id", "desc format = 'brief' select * from t join ( select count(*)+id as v from t group by id) as A on A.v = t.id", @@ -272,7 +272,19 @@ "desc format = 'brief' select * from t join ( select /*+ hash_agg()*/ count(*) as a from t) as A on A.a = t.id", "desc format = 'brief' select sum(b) from (select t.id, t1.id as b from t join t t1 on t.id=t1.id)A group by id", "desc format = 'brief' select * from (select id from t group by id) C join (select sum(value),id from t group by id)B on C.id=B.id", - "desc format = 'brief' select * from (select id from t group by id) C join (select sum(b),id from (select t.id, t1.id as b from t join (select id, count(*) as c from t group by id) t1 on t.id=t1.id)A group by id)B on C.id=b.id" + "desc format = 'brief' select * from (select id from t group by id) C join (select sum(b),id from (select t.id, t1.id as b from t join (select id, count(*) as c from t group by id) t1 on t.id=t1.id)A group by id)B on C.id=b.id", + "desc format = 'brief' select * from t join t t1 on t.id = t1.id order by t.value limit 1", + "desc format = 'brief' select * from t join t t1 on t.id = t1.id order by t.value % 100 limit 1", + "desc format = 'brief' select count(*) from (select t.id, t.value v1 from t join t t1 on t.id = t1.id order by t.value limit 20) v group by v.v1" + ] + }, + { + "name": "TestIndexMerge", + "cases": [ + "desc format='brief' select /*+ use_index_merge(t) */ * from t where a =1 or (b=1 and b+2>1)", + "desc format='brief' select /*+ use_index_merge(t) */ * from t where a =1 or (b=1 and length(b)=1)", + "desc format='brief' select /*+ use_index_merge(t) */ * from t where (a=1 and length(a)=1) or (b=1 and length(b)=1)", + "desc format='brief' select /*+ use_index_merge(t) */ * from t where (a=1 and length(b)=1) or (b=1 and length(a)=1)" ] }, { @@ -281,5 +293,11 @@ "desc format = 'brief' select * from t where a = 1 and b > 2 and b < 10 and d = 10 order by b,c limit 10", "desc format = 'brief' select * from t where a = 1 and b > 2 and b < 10 and d = 10 order by b desc, c desc limit 10" ] + }, + { + "name": "TestIssue23887", + "cases": [ + "select (2) in (select b from t) from (select t.a < (select t.a from t t1 limit 1) from t) t" + ] } ] diff --git a/planner/core/testdata/integration_serial_suite_out.json b/planner/core/testdata/integration_serial_suite_out.json index edacaa2109823..8499bfb15d103 100644 --- a/planner/core/testdata/integration_serial_suite_out.json +++ b/planner/core/testdata/integration_serial_suite_out.json @@ -85,18 +85,18 @@ "SQL": "explain format = 'verbose' select * from t3 order by a limit 1", "Plan": [ "TopN_7 1.00 13.22 root test.t3.a, offset:0, count:1", - "└─TableReader_15 1.00 10.22 root data:TopN_14", - " └─TopN_14 1.00 0.00 cop[tikv] test.t3.a, offset:0, count:1", - " └─TableFullScan_13 3.00 128.00 cop[tikv] table:t3 keep order:false" + "└─TableReader_16 1.00 10.22 root data:TopN_15", + " └─TopN_15 1.00 0.00 cop[tikv] test.t3.a, offset:0, count:1", + " └─TableFullScan_14 3.00 128.00 cop[tikv] table:t3 keep order:false" ] }, { "SQL": "explain format = 'verbose' select * from t3 order by b limit 1", "Plan": [ "TopN_7 1.00 13.22 root test.t3.b, offset:0, count:1", - "└─TableReader_15 1.00 10.22 root data:TopN_14", - " └─TopN_14 1.00 0.00 cop[tikv] test.t3.b, offset:0, count:1", - " └─TableFullScan_13 3.00 128.00 cop[tikv] table:t3 keep order:false" + "└─TableReader_16 1.00 10.22 root data:TopN_15", + " └─TopN_15 1.00 0.00 cop[tikv] test.t3.b, offset:0, count:1", + " └─TableFullScan_14 3.00 128.00 cop[tikv] table:t3 keep order:false" ] }, { @@ -185,19 +185,21 @@ { "SQL": "explain format = 'verbose' select (2) in (select count(*) from t1) from (select t.b < (select t.b from t2 limit 1 ) from t3 t) t", "Plan": [ - "HashJoin_19 3.00 113.61 root CARTESIAN left outer semi join", - "├─Selection_38(Build) 0.80 11.18 root eq(2, Column#18)", - "│ └─StreamAgg_59 1.00 69.50 root funcs:count(Column#27)->Column#18", - "│ └─TableReader_60 1.00 5.17 root data:StreamAgg_43", - "│ └─StreamAgg_43 1.00 8.18 batchCop[tiflash] funcs:count(1)->Column#27", - "│ └─TableFullScan_58 3.00 60.50 batchCop[tiflash] table:t1 keep order:false", - "└─Apply_21(Probe) 3.00 82.03 root CARTESIAN left outer join", - " ├─TableReader_23(Build) 3.00 10.16 root data:TableFullScan_22", - " │ └─TableFullScan_22 3.00 128.00 cop[tikv] table:t keep order:false", - " └─Limit_27(Probe) 1.00 5.36 root offset:0, count:1", - " └─TableReader_33 1.00 5.36 root data:Limit_32", - " └─Limit_32 1.00 56.00 cop[tikv] offset:0, count:1", - " └─TableFullScan_30 1.00 56.00 cop[tikv] table:t2 keep order:false" + "HashJoin_19 3.00 133.41 root CARTESIAN left outer semi join", + "├─Selection_39(Build) 0.80 11.18 root eq(2, Column#18)", + "│ └─StreamAgg_60 1.00 69.50 root funcs:count(Column#32)->Column#18", + "│ └─TableReader_61 1.00 5.17 root data:StreamAgg_44", + "│ └─StreamAgg_44 1.00 8.18 batchCop[tiflash] funcs:count(1)->Column#32", + "│ └─TableFullScan_59 3.00 60.50 batchCop[tiflash] table:t1 keep order:false", + "└─Projection_20(Probe) 3.00 101.83 root 1->Column#26", + " └─Apply_22 3.00 82.03 root CARTESIAN left outer join", + " ├─TableReader_24(Build) 3.00 10.16 root data:TableFullScan_23", + " │ └─TableFullScan_23 3.00 128.00 cop[tikv] table:t keep order:false", + " └─Projection_27(Probe) 1.00 23.96 root 1->Column#27", + " └─Limit_28 1.00 5.36 root offset:0, count:1", + " └─TableReader_34 1.00 5.36 root data:Limit_33", + " └─Limit_33 1.00 56.00 cop[tikv] offset:0, count:1", + " └─TableFullScan_31 1.00 56.00 cop[tikv] table:t2 keep order:false" ] }, { @@ -248,59 +250,62 @@ { "SQL": "explain format = 'brief' select count(*) from fact_t, d1_t where fact_t.d1_k = d1_t.d1_k", "Plan": [ - "HashAgg 1.00 root funcs:count(Column#12)->Column#11", + "HashAgg 1.00 root funcs:count(Column#13)->Column#11", "└─TableReader 1.00 root data:ExchangeSender", " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#12", - " └─HashJoin 8.00 batchCop[tiflash] inner join, equal:[eq(test.d1_t.d1_k, test.fact_t.d1_k)]", - " ├─ExchangeReceiver(Build) 2.00 batchCop[tiflash] ", - " │ └─ExchangeSender 2.00 batchCop[tiflash] ExchangeType: Broadcast", - " │ └─Selection 2.00 batchCop[tiflash] not(isnull(test.d1_t.d1_k))", - " │ └─TableFullScan 2.00 batchCop[tiflash] table:d1_t keep order:false", - " └─Selection(Probe) 8.00 batchCop[tiflash] not(isnull(test.fact_t.d1_k))", - " └─TableFullScan 8.00 batchCop[tiflash] table:fact_t keep order:false" + " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#13", + " └─Projection 8.00 batchCop[tiflash] 1->Column#12", + " └─HashJoin 8.00 batchCop[tiflash] inner join, equal:[eq(test.d1_t.d1_k, test.fact_t.d1_k)]", + " ├─ExchangeReceiver(Build) 2.00 batchCop[tiflash] ", + " │ └─ExchangeSender 2.00 batchCop[tiflash] ExchangeType: Broadcast", + " │ └─Selection 2.00 batchCop[tiflash] not(isnull(test.d1_t.d1_k))", + " │ └─TableFullScan 2.00 batchCop[tiflash] table:d1_t keep order:false", + " └─Selection(Probe) 8.00 batchCop[tiflash] not(isnull(test.fact_t.d1_k))", + " └─TableFullScan 8.00 batchCop[tiflash] table:fact_t keep order:false" ] }, { "SQL": "explain format = 'brief' select count(*) from fact_t, d1_t, d2_t, d3_t where fact_t.d1_k = d1_t.d1_k and fact_t.d2_k = d2_t.d2_k and fact_t.d3_k = d3_t.d3_k", "Plan": [ - "HashAgg 1.00 root funcs:count(Column#18)->Column#17", + "HashAgg 1.00 root funcs:count(Column#19)->Column#17", "└─TableReader 1.00 root data:ExchangeSender", " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#18", - " └─HashJoin 8.00 batchCop[tiflash] inner join, equal:[eq(test.fact_t.d3_k, test.d3_t.d3_k)]", - " ├─ExchangeReceiver(Build) 2.00 batchCop[tiflash] ", - " │ └─ExchangeSender 2.00 batchCop[tiflash] ExchangeType: Broadcast", - " │ └─Selection 2.00 batchCop[tiflash] not(isnull(test.d3_t.d3_k))", - " │ └─TableFullScan 2.00 batchCop[tiflash] table:d3_t keep order:false", - " └─HashJoin(Probe) 8.00 batchCop[tiflash] inner join, equal:[eq(test.fact_t.d2_k, test.d2_t.d2_k)]", + " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#19", + " └─Projection 8.00 batchCop[tiflash] 1->Column#18", + " └─HashJoin 8.00 batchCop[tiflash] inner join, equal:[eq(test.fact_t.d3_k, test.d3_t.d3_k)]", " ├─ExchangeReceiver(Build) 2.00 batchCop[tiflash] ", " │ └─ExchangeSender 2.00 batchCop[tiflash] ExchangeType: Broadcast", - " │ └─Selection 2.00 batchCop[tiflash] not(isnull(test.d2_t.d2_k))", - " │ └─TableFullScan 2.00 batchCop[tiflash] table:d2_t keep order:false", - " └─HashJoin(Probe) 8.00 batchCop[tiflash] inner join, equal:[eq(test.d1_t.d1_k, test.fact_t.d1_k)]", + " │ └─Selection 2.00 batchCop[tiflash] not(isnull(test.d3_t.d3_k))", + " │ └─TableFullScan 2.00 batchCop[tiflash] table:d3_t keep order:false", + " └─HashJoin(Probe) 8.00 batchCop[tiflash] inner join, equal:[eq(test.fact_t.d2_k, test.d2_t.d2_k)]", " ├─ExchangeReceiver(Build) 2.00 batchCop[tiflash] ", " │ └─ExchangeSender 2.00 batchCop[tiflash] ExchangeType: Broadcast", - " │ └─Selection 2.00 batchCop[tiflash] not(isnull(test.d1_t.d1_k))", - " │ └─TableFullScan 2.00 batchCop[tiflash] table:d1_t keep order:false", - " └─Selection(Probe) 8.00 batchCop[tiflash] not(isnull(test.fact_t.d1_k)), not(isnull(test.fact_t.d2_k)), not(isnull(test.fact_t.d3_k))", - " └─TableFullScan 8.00 batchCop[tiflash] table:fact_t keep order:false" + " │ └─Selection 2.00 batchCop[tiflash] not(isnull(test.d2_t.d2_k))", + " │ └─TableFullScan 2.00 batchCop[tiflash] table:d2_t keep order:false", + " └─HashJoin(Probe) 8.00 batchCop[tiflash] inner join, equal:[eq(test.d1_t.d1_k, test.fact_t.d1_k)]", + " ├─ExchangeReceiver(Build) 2.00 batchCop[tiflash] ", + " │ └─ExchangeSender 2.00 batchCop[tiflash] ExchangeType: Broadcast", + " │ └─Selection 2.00 batchCop[tiflash] not(isnull(test.d1_t.d1_k))", + " │ └─TableFullScan 2.00 batchCop[tiflash] table:d1_t keep order:false", + " └─Selection(Probe) 8.00 batchCop[tiflash] not(isnull(test.fact_t.d1_k)), not(isnull(test.fact_t.d2_k)), not(isnull(test.fact_t.d3_k))", + " └─TableFullScan 8.00 batchCop[tiflash] table:fact_t keep order:false" ] }, { "SQL": "explain format = 'brief' select count(*) from fact_t, d1_t where fact_t.d1_k = d1_t.d1_k", "Plan": [ - "HashAgg 1.00 root funcs:count(Column#12)->Column#11", + "HashAgg 1.00 root funcs:count(Column#13)->Column#11", "└─TableReader 1.00 root data:ExchangeSender", " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#12", - " └─HashJoin 8.00 batchCop[tiflash] inner join, equal:[eq(test.d1_t.d1_k, test.fact_t.d1_k)]", - " ├─ExchangeReceiver(Build) 2.00 batchCop[tiflash] ", - " │ └─ExchangeSender 2.00 batchCop[tiflash] ExchangeType: Broadcast", - " │ └─Selection 2.00 batchCop[tiflash] not(isnull(test.d1_t.d1_k))", - " │ └─TableFullScan 2.00 batchCop[tiflash] table:d1_t keep order:false", - " └─Selection(Probe) 8.00 batchCop[tiflash] not(isnull(test.fact_t.d1_k))", - " └─TableFullScan 8.00 batchCop[tiflash] table:fact_t keep order:false" + " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#13", + " └─Projection 8.00 batchCop[tiflash] 1->Column#12", + " └─HashJoin 8.00 batchCop[tiflash] inner join, equal:[eq(test.d1_t.d1_k, test.fact_t.d1_k)]", + " ├─ExchangeReceiver(Build) 2.00 batchCop[tiflash] ", + " │ └─ExchangeSender 2.00 batchCop[tiflash] ExchangeType: Broadcast", + " │ └─Selection 2.00 batchCop[tiflash] not(isnull(test.d1_t.d1_k))", + " │ └─TableFullScan 2.00 batchCop[tiflash] table:d1_t keep order:false", + " └─Selection(Probe) 8.00 batchCop[tiflash] not(isnull(test.fact_t.d1_k))", + " └─TableFullScan 8.00 batchCop[tiflash] table:fact_t keep order:false" ] }, { @@ -334,17 +339,18 @@ { "SQL": "explain format = 'brief' select count(*) from fact_t join d1_t on fact_t.d1_k = d1_t.d1_k and fact_t.col1 > d1_t.value", "Plan": [ - "HashAgg 1.00 root funcs:count(Column#12)->Column#11", + "HashAgg 1.00 root funcs:count(Column#13)->Column#11", "└─TableReader 1.00 root data:ExchangeSender", " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#12", - " └─HashJoin 8.00 batchCop[tiflash] inner join, equal:[eq(test.d1_t.d1_k, test.fact_t.d1_k)], other cond:gt(test.fact_t.col1, test.d1_t.value)", - " ├─ExchangeReceiver(Build) 2.00 batchCop[tiflash] ", - " │ └─ExchangeSender 2.00 batchCop[tiflash] ExchangeType: Broadcast", - " │ └─Selection 2.00 batchCop[tiflash] not(isnull(test.d1_t.d1_k)), not(isnull(test.d1_t.value))", - " │ └─TableFullScan 2.00 batchCop[tiflash] table:d1_t keep order:false", - " └─Selection(Probe) 8.00 batchCop[tiflash] not(isnull(test.fact_t.col1)), not(isnull(test.fact_t.d1_k))", - " └─TableFullScan 8.00 batchCop[tiflash] table:fact_t keep order:false" + " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#13", + " └─Projection 8.00 batchCop[tiflash] 1->Column#12", + " └─HashJoin 8.00 batchCop[tiflash] inner join, equal:[eq(test.d1_t.d1_k, test.fact_t.d1_k)], other cond:gt(test.fact_t.col1, test.d1_t.value)", + " ├─ExchangeReceiver(Build) 2.00 batchCop[tiflash] ", + " │ └─ExchangeSender 2.00 batchCop[tiflash] ExchangeType: Broadcast", + " │ └─Selection 2.00 batchCop[tiflash] not(isnull(test.d1_t.d1_k)), not(isnull(test.d1_t.value))", + " │ └─TableFullScan 2.00 batchCop[tiflash] table:d1_t keep order:false", + " └─Selection(Probe) 8.00 batchCop[tiflash] not(isnull(test.fact_t.col1)), not(isnull(test.fact_t.d1_k))", + " └─TableFullScan 8.00 batchCop[tiflash] table:fact_t keep order:false" ] }, { @@ -467,97 +473,101 @@ { "SQL": "explain format = 'brief' select count(*) from fact_t, d1_t where fact_t.d1_k = d1_t.d1_k", "Plan": [ - "HashAgg 1.00 root funcs:count(Column#12)->Column#11", + "HashAgg 1.00 root funcs:count(Column#13)->Column#11", "└─TableReader 1.00 root data:ExchangeSender", " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#12", - " └─HashJoin 32.00 batchCop[tiflash] inner join, equal:[eq(test.d1_t.d1_k, test.fact_t.d1_k)]", - " ├─ExchangeReceiver(Build) 4.00 batchCop[tiflash] ", - " │ └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.d1_t.d1_k", - " │ └─Selection 4.00 batchCop[tiflash] not(isnull(test.d1_t.d1_k))", - " │ └─TableFullScan 4.00 batchCop[tiflash] table:d1_t keep order:false", - " └─ExchangeReceiver(Probe) 16.00 batchCop[tiflash] ", - " └─ExchangeSender 16.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.fact_t.d1_k", - " └─Selection 16.00 batchCop[tiflash] not(isnull(test.fact_t.d1_k))", - " └─TableFullScan 16.00 batchCop[tiflash] table:fact_t keep order:false" + " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#13", + " └─Projection 32.00 batchCop[tiflash] 1->Column#12", + " └─HashJoin 32.00 batchCop[tiflash] inner join, equal:[eq(test.d1_t.d1_k, test.fact_t.d1_k)]", + " ├─ExchangeReceiver(Build) 4.00 batchCop[tiflash] ", + " │ └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.d1_t.d1_k", + " │ └─Selection 4.00 batchCop[tiflash] not(isnull(test.d1_t.d1_k))", + " │ └─TableFullScan 4.00 batchCop[tiflash] table:d1_t keep order:false", + " └─ExchangeReceiver(Probe) 16.00 batchCop[tiflash] ", + " └─ExchangeSender 16.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.fact_t.d1_k", + " └─Selection 16.00 batchCop[tiflash] not(isnull(test.fact_t.d1_k))", + " └─TableFullScan 16.00 batchCop[tiflash] table:fact_t keep order:false" ] }, { "SQL": "explain format = 'brief' select count(*) from fact_t, d1_t, d2_t, d3_t where fact_t.d1_k = d1_t.d1_k and fact_t.d2_k = d2_t.d2_k and fact_t.d3_k = d3_t.d3_k", "Plan": [ - "HashAgg 1.00 root funcs:count(Column#18)->Column#17", + "HashAgg 1.00 root funcs:count(Column#19)->Column#17", "└─TableReader 1.00 root data:ExchangeSender", " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#18", - " └─HashJoin 128.00 batchCop[tiflash] inner join, equal:[eq(test.fact_t.d3_k, test.d3_t.d3_k)]", - " ├─ExchangeReceiver(Build) 4.00 batchCop[tiflash] ", - " │ └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.d3_t.d3_k", - " │ └─Selection 4.00 batchCop[tiflash] not(isnull(test.d3_t.d3_k))", - " │ └─TableFullScan 4.00 batchCop[tiflash] table:d3_t keep order:false", - " └─ExchangeReceiver(Probe) 64.00 batchCop[tiflash] ", - " └─ExchangeSender 64.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.fact_t.d3_k", - " └─HashJoin 64.00 batchCop[tiflash] inner join, equal:[eq(test.fact_t.d2_k, test.d2_t.d2_k)]", - " ├─ExchangeReceiver(Build) 4.00 batchCop[tiflash] ", - " │ └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.d2_t.d2_k", - " │ └─Selection 4.00 batchCop[tiflash] not(isnull(test.d2_t.d2_k))", - " │ └─TableFullScan 4.00 batchCop[tiflash] table:d2_t keep order:false", - " └─ExchangeReceiver(Probe) 32.00 batchCop[tiflash] ", - " └─ExchangeSender 32.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.fact_t.d2_k", - " └─HashJoin 32.00 batchCop[tiflash] inner join, equal:[eq(test.d1_t.d1_k, test.fact_t.d1_k)]", - " ├─ExchangeReceiver(Build) 4.00 batchCop[tiflash] ", - " │ └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.d1_t.d1_k", - " │ └─Selection 4.00 batchCop[tiflash] not(isnull(test.d1_t.d1_k))", - " │ └─TableFullScan 4.00 batchCop[tiflash] table:d1_t keep order:false", - " └─ExchangeReceiver(Probe) 16.00 batchCop[tiflash] ", - " └─ExchangeSender 16.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.fact_t.d1_k", - " └─Selection 16.00 batchCop[tiflash] not(isnull(test.fact_t.d1_k)), not(isnull(test.fact_t.d2_k)), not(isnull(test.fact_t.d3_k))", - " └─TableFullScan 16.00 batchCop[tiflash] table:fact_t keep order:false" + " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#19", + " └─Projection 128.00 batchCop[tiflash] 1->Column#18", + " └─HashJoin 128.00 batchCop[tiflash] inner join, equal:[eq(test.fact_t.d3_k, test.d3_t.d3_k)]", + " ├─ExchangeReceiver(Build) 4.00 batchCop[tiflash] ", + " │ └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.d3_t.d3_k", + " │ └─Selection 4.00 batchCop[tiflash] not(isnull(test.d3_t.d3_k))", + " │ └─TableFullScan 4.00 batchCop[tiflash] table:d3_t keep order:false", + " └─ExchangeReceiver(Probe) 64.00 batchCop[tiflash] ", + " └─ExchangeSender 64.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.fact_t.d3_k", + " └─HashJoin 64.00 batchCop[tiflash] inner join, equal:[eq(test.fact_t.d2_k, test.d2_t.d2_k)]", + " ├─ExchangeReceiver(Build) 4.00 batchCop[tiflash] ", + " │ └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.d2_t.d2_k", + " │ └─Selection 4.00 batchCop[tiflash] not(isnull(test.d2_t.d2_k))", + " │ └─TableFullScan 4.00 batchCop[tiflash] table:d2_t keep order:false", + " └─ExchangeReceiver(Probe) 32.00 batchCop[tiflash] ", + " └─ExchangeSender 32.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.fact_t.d2_k", + " └─HashJoin 32.00 batchCop[tiflash] inner join, equal:[eq(test.d1_t.d1_k, test.fact_t.d1_k)]", + " ├─ExchangeReceiver(Build) 4.00 batchCop[tiflash] ", + " │ └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.d1_t.d1_k", + " │ └─Selection 4.00 batchCop[tiflash] not(isnull(test.d1_t.d1_k))", + " │ └─TableFullScan 4.00 batchCop[tiflash] table:d1_t keep order:false", + " └─ExchangeReceiver(Probe) 16.00 batchCop[tiflash] ", + " └─ExchangeSender 16.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.fact_t.d1_k", + " └─Selection 16.00 batchCop[tiflash] not(isnull(test.fact_t.d1_k)), not(isnull(test.fact_t.d2_k)), not(isnull(test.fact_t.d3_k))", + " └─TableFullScan 16.00 batchCop[tiflash] table:fact_t keep order:false" ] }, { "SQL": "explain format = 'brief' select count(*) from fact_t, d1_t where fact_t.d1_k = d1_t.d1_k", "Plan": [ - "HashAgg 1.00 root funcs:count(Column#12)->Column#11", + "HashAgg 1.00 root funcs:count(Column#13)->Column#11", "└─TableReader 1.00 root data:ExchangeSender", " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#12", - " └─HashJoin 32.00 batchCop[tiflash] inner join, equal:[eq(test.d1_t.d1_k, test.fact_t.d1_k)]", - " ├─ExchangeReceiver(Build) 4.00 batchCop[tiflash] ", - " │ └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.d1_t.d1_k", - " │ └─Selection 4.00 batchCop[tiflash] not(isnull(test.d1_t.d1_k))", - " │ └─TableFullScan 4.00 batchCop[tiflash] table:d1_t keep order:false", - " └─ExchangeReceiver(Probe) 16.00 batchCop[tiflash] ", - " └─ExchangeSender 16.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.fact_t.d1_k", - " └─Selection 16.00 batchCop[tiflash] not(isnull(test.fact_t.d1_k))", - " └─TableFullScan 16.00 batchCop[tiflash] table:fact_t keep order:false" + " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#13", + " └─Projection 32.00 batchCop[tiflash] 1->Column#12", + " └─HashJoin 32.00 batchCop[tiflash] inner join, equal:[eq(test.d1_t.d1_k, test.fact_t.d1_k)]", + " ├─ExchangeReceiver(Build) 4.00 batchCop[tiflash] ", + " │ └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.d1_t.d1_k", + " │ └─Selection 4.00 batchCop[tiflash] not(isnull(test.d1_t.d1_k))", + " │ └─TableFullScan 4.00 batchCop[tiflash] table:d1_t keep order:false", + " └─ExchangeReceiver(Probe) 16.00 batchCop[tiflash] ", + " └─ExchangeSender 16.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.fact_t.d1_k", + " └─Selection 16.00 batchCop[tiflash] not(isnull(test.fact_t.d1_k))", + " └─TableFullScan 16.00 batchCop[tiflash] table:fact_t keep order:false" ] }, { "SQL": "explain format = 'brief' select count(*) from fact_t, d1_t, d2_t, d3_t where fact_t.d1_k = d1_t.d1_k and fact_t.d1_k = d2_t.value and fact_t.d1_k = d3_t.value", "Plan": [ - "HashAgg 1.00 root funcs:count(Column#18)->Column#17", + "HashAgg 1.00 root funcs:count(Column#19)->Column#17", "└─TableReader 1.00 root data:ExchangeSender", " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#18", - " └─HashJoin 128.00 batchCop[tiflash] inner join, equal:[eq(test.fact_t.d1_k, test.d3_t.value)]", - " ├─ExchangeReceiver(Build) 4.00 batchCop[tiflash] ", - " │ └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.d3_t.value", - " │ └─Selection 4.00 batchCop[tiflash] not(isnull(test.d3_t.value))", - " │ └─TableFullScan 4.00 batchCop[tiflash] table:d3_t keep order:false", - " └─HashJoin(Probe) 64.00 batchCop[tiflash] inner join, equal:[eq(test.fact_t.d1_k, test.d2_t.value)]", + " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#19", + " └─Projection 128.00 batchCop[tiflash] 1->Column#18", + " └─HashJoin 128.00 batchCop[tiflash] inner join, equal:[eq(test.fact_t.d1_k, test.d3_t.value)]", " ├─ExchangeReceiver(Build) 4.00 batchCop[tiflash] ", - " │ └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.d2_t.value", - " │ └─Selection 4.00 batchCop[tiflash] not(isnull(test.d2_t.value))", - " │ └─TableFullScan 4.00 batchCop[tiflash] table:d2_t keep order:false", - " └─HashJoin(Probe) 32.00 batchCop[tiflash] inner join, equal:[eq(test.d1_t.d1_k, test.fact_t.d1_k)]", + " │ └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.d3_t.value", + " │ └─Selection 4.00 batchCop[tiflash] not(isnull(test.d3_t.value))", + " │ └─TableFullScan 4.00 batchCop[tiflash] table:d3_t keep order:false", + " └─HashJoin(Probe) 64.00 batchCop[tiflash] inner join, equal:[eq(test.fact_t.d1_k, test.d2_t.value)]", " ├─ExchangeReceiver(Build) 4.00 batchCop[tiflash] ", - " │ └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.d1_t.d1_k", - " │ └─Selection 4.00 batchCop[tiflash] not(isnull(test.d1_t.d1_k))", - " │ └─TableFullScan 4.00 batchCop[tiflash] table:d1_t keep order:false", - " └─ExchangeReceiver(Probe) 16.00 batchCop[tiflash] ", - " └─ExchangeSender 16.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.fact_t.d1_k", - " └─Selection 16.00 batchCop[tiflash] not(isnull(test.fact_t.d1_k))", - " └─TableFullScan 16.00 batchCop[tiflash] table:fact_t keep order:false" + " │ └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.d2_t.value", + " │ └─Selection 4.00 batchCop[tiflash] not(isnull(test.d2_t.value))", + " │ └─TableFullScan 4.00 batchCop[tiflash] table:d2_t keep order:false", + " └─HashJoin(Probe) 32.00 batchCop[tiflash] inner join, equal:[eq(test.d1_t.d1_k, test.fact_t.d1_k)]", + " ├─ExchangeReceiver(Build) 4.00 batchCop[tiflash] ", + " │ └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.d1_t.d1_k", + " │ └─Selection 4.00 batchCop[tiflash] not(isnull(test.d1_t.d1_k))", + " │ └─TableFullScan 4.00 batchCop[tiflash] table:d1_t keep order:false", + " └─ExchangeReceiver(Probe) 16.00 batchCop[tiflash] ", + " └─ExchangeSender 16.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.fact_t.d1_k", + " └─Selection 16.00 batchCop[tiflash] not(isnull(test.fact_t.d1_k))", + " └─TableFullScan 16.00 batchCop[tiflash] table:fact_t keep order:false" ] }, { @@ -597,19 +607,20 @@ { "SQL": "explain format = 'brief' select count(*) from fact_t join d1_t on fact_t.d1_k = d1_t.d1_k and fact_t.col1 > d1_t.value", "Plan": [ - "HashAgg 1.00 root funcs:count(Column#12)->Column#11", + "HashAgg 1.00 root funcs:count(Column#13)->Column#11", "└─TableReader 1.00 root data:ExchangeSender", " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#12", - " └─HashJoin 32.00 batchCop[tiflash] inner join, equal:[eq(test.d1_t.d1_k, test.fact_t.d1_k)], other cond:gt(test.fact_t.col1, test.d1_t.value)", - " ├─ExchangeReceiver(Build) 4.00 batchCop[tiflash] ", - " │ └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.d1_t.d1_k", - " │ └─Selection 4.00 batchCop[tiflash] not(isnull(test.d1_t.d1_k)), not(isnull(test.d1_t.value))", - " │ └─TableFullScan 4.00 batchCop[tiflash] table:d1_t keep order:false", - " └─ExchangeReceiver(Probe) 16.00 batchCop[tiflash] ", - " └─ExchangeSender 16.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.fact_t.d1_k", - " └─Selection 16.00 batchCop[tiflash] not(isnull(test.fact_t.col1)), not(isnull(test.fact_t.d1_k))", - " └─TableFullScan 16.00 batchCop[tiflash] table:fact_t keep order:false" + " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#13", + " └─Projection 32.00 batchCop[tiflash] 1->Column#12", + " └─HashJoin 32.00 batchCop[tiflash] inner join, equal:[eq(test.d1_t.d1_k, test.fact_t.d1_k)], other cond:gt(test.fact_t.col1, test.d1_t.value)", + " ├─ExchangeReceiver(Build) 4.00 batchCop[tiflash] ", + " │ └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.d1_t.d1_k", + " │ └─Selection 4.00 batchCop[tiflash] not(isnull(test.d1_t.d1_k)), not(isnull(test.d1_t.value))", + " │ └─TableFullScan 4.00 batchCop[tiflash] table:d1_t keep order:false", + " └─ExchangeReceiver(Probe) 16.00 batchCop[tiflash] ", + " └─ExchangeSender 16.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.fact_t.d1_k", + " └─Selection 16.00 batchCop[tiflash] not(isnull(test.fact_t.col1)), not(isnull(test.fact_t.d1_k))", + " └─TableFullScan 16.00 batchCop[tiflash] table:fact_t keep order:false" ] }, { @@ -632,26 +643,27 @@ { "SQL": "explain format = 'brief' select count(*) from (select case when t1.col1 is null then t2.col1 + 5 else 10 end as col1, t2.d1_k as d1_k from fact_t t1 right join fact_t t2 on t1.d1_k = t2.d1_k) fact_t join d1_t on fact_t.d1_k = d1_t.d1_k and fact_t.col1 > 5", "Plan": [ - "HashAgg 1.00 root funcs:count(Column#22)->Column#19", + "HashAgg 1.00 root funcs:count(Column#23)->Column#19", "└─TableReader 1.00 root data:ExchangeSender", " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#22", - " └─HashJoin 204.80 batchCop[tiflash] inner join, equal:[eq(test.d1_t.d1_k, test.fact_t.d1_k)]", - " ├─ExchangeReceiver(Build) 4.00 batchCop[tiflash] ", - " │ └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.d1_t.d1_k", - " │ └─Selection 4.00 batchCop[tiflash] not(isnull(test.d1_t.d1_k))", - " │ └─TableFullScan 4.00 batchCop[tiflash] table:d1_t keep order:false", - " └─Projection(Probe) 102.40 batchCop[tiflash] test.fact_t.d1_k", - " └─Selection 102.40 batchCop[tiflash] gt(case(isnull(test.fact_t.col1), plus(test.fact_t.col1, 5), 10), 5)", - " └─HashJoin 128.00 batchCop[tiflash] right outer join, equal:[eq(test.fact_t.d1_k, test.fact_t.d1_k)]", - " ├─ExchangeReceiver(Build) 16.00 batchCop[tiflash] ", - " │ └─ExchangeSender 16.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.fact_t.d1_k", - " │ └─Selection 16.00 batchCop[tiflash] not(isnull(test.fact_t.d1_k))", - " │ └─TableFullScan 16.00 batchCop[tiflash] table:t1 keep order:false", - " └─ExchangeReceiver(Probe) 16.00 batchCop[tiflash] ", - " └─ExchangeSender 16.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.fact_t.d1_k", - " └─Selection 16.00 batchCop[tiflash] not(isnull(test.fact_t.d1_k))", - " └─TableFullScan 16.00 batchCop[tiflash] table:t2 keep order:false" + " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#23", + " └─Projection 204.80 batchCop[tiflash] 1->Column#22", + " └─HashJoin 204.80 batchCop[tiflash] inner join, equal:[eq(test.d1_t.d1_k, test.fact_t.d1_k)]", + " ├─ExchangeReceiver(Build) 4.00 batchCop[tiflash] ", + " │ └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.d1_t.d1_k", + " │ └─Selection 4.00 batchCop[tiflash] not(isnull(test.d1_t.d1_k))", + " │ └─TableFullScan 4.00 batchCop[tiflash] table:d1_t keep order:false", + " └─Projection(Probe) 102.40 batchCop[tiflash] test.fact_t.d1_k", + " └─Selection 102.40 batchCop[tiflash] gt(case(isnull(test.fact_t.col1), plus(test.fact_t.col1, 5), 10), 5)", + " └─HashJoin 128.00 batchCop[tiflash] right outer join, equal:[eq(test.fact_t.d1_k, test.fact_t.d1_k)]", + " ├─ExchangeReceiver(Build) 16.00 batchCop[tiflash] ", + " │ └─ExchangeSender 16.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.fact_t.d1_k", + " │ └─Selection 16.00 batchCop[tiflash] not(isnull(test.fact_t.d1_k))", + " │ └─TableFullScan 16.00 batchCop[tiflash] table:t1 keep order:false", + " └─ExchangeReceiver(Probe) 16.00 batchCop[tiflash] ", + " └─ExchangeSender 16.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.fact_t.d1_k", + " └─Selection 16.00 batchCop[tiflash] not(isnull(test.fact_t.d1_k))", + " └─TableFullScan 16.00 batchCop[tiflash] table:t2 keep order:false" ] }, { @@ -1498,9 +1510,9 @@ { "SQL": "desc format = 'brief' select /*+ hash_agg()*/ count(*) from (select id + 1 as b from t)A", "Plan": [ - "HashAgg 1.00 root funcs:count(Column#6)->Column#5", + "HashAgg 1.00 root funcs:count(Column#8)->Column#5", "└─TableReader 1.00 root data:HashAgg", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#6", + " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#8", " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" ] }, @@ -1527,9 +1539,9 @@ { "SQL": "desc format = 'brief' select /*+ stream_agg()*/ count(*) from (select id + 1 as b from t)A", "Plan": [ - "StreamAgg 1.00 root funcs:count(Column#6)->Column#5", + "StreamAgg 1.00 root funcs:count(Column#8)->Column#5", "└─TableReader 1.00 root data:StreamAgg", - " └─StreamAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#6", + " └─StreamAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#8", " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" ] }, @@ -1643,10 +1655,10 @@ { "SQL": "desc format = 'brief' select /*+ hash_agg()*/ count(*) from (select id + 1 as b from t)A", "Plan": [ - "HashAgg 1.00 root funcs:count(Column#7)->Column#5", + "HashAgg 1.00 root funcs:count(Column#9)->Column#5", "└─TableReader 1.00 root data:ExchangeSender", " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#7", + " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#9", " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" ] }, @@ -1674,9 +1686,9 @@ { "SQL": "desc format = 'brief' select /*+ stream_agg()*/ count(*) from (select id + 1 as b from t)A", "Plan": [ - "StreamAgg 1.00 root funcs:count(Column#6)->Column#5", + "StreamAgg 1.00 root funcs:count(Column#8)->Column#5", "└─TableReader 1.00 root data:StreamAgg", - " └─StreamAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#6", + " └─StreamAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#8", " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" ] }, @@ -1964,10 +1976,10 @@ { "SQL": "desc format = 'brief' select /*+ hash_agg()*/ count(*) from (select id+1 from t)A", "Plan": [ - "HashAgg 1.00 root funcs:count(Column#7)->Column#5", + "HashAgg 1.00 root funcs:count(Column#9)->Column#5", "└─TableReader 1.00 root data:ExchangeSender", " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#7", + " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#9", " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" ] }, @@ -2272,7 +2284,7 @@ ] }, { - "Name": "TestMppAggWithJoin", + "Name": "TestMppAggTopNWithJoin", "Cases": [ { "SQL": "desc format = 'brief' select * from t join ( select count(*), id from t group by id) as A on A.id = t.id", @@ -2423,6 +2435,106 @@ " └─Selection(Probe) 9990.00 batchCop[tiflash] not(isnull(test.t.id))", " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" ] + }, + { + "SQL": "desc format = 'brief' select * from t join t t1 on t.id = t1.id order by t.value limit 1", + "Plan": [ + "TopN 1.00 root test.t.value, offset:0, count:1", + "└─TableReader 1.00 root data:ExchangeSender", + " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─TopN 1.00 batchCop[tiflash] test.t.value, offset:0, count:1", + " └─HashJoin 12487.50 batchCop[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", + " ├─ExchangeReceiver(Build) 9990.00 batchCop[tiflash] ", + " │ └─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: Broadcast", + " │ └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", + " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 batchCop[tiflash] not(isnull(test.t.id))", + " └─TableFullScan 10000.00 batchCop[tiflash] table:t1 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select * from t join t t1 on t.id = t1.id order by t.value % 100 limit 1", + "Plan": [ + "Projection 1.00 root test.t.id, test.t.value, test.t.id, test.t.value", + "└─TopN 1.00 root Column#7, offset:0, count:1", + " └─Projection 12487.50 root test.t.id, test.t.value, test.t.id, test.t.value, mod(test.t.value, 100)->Column#7", + " └─TableReader 12487.50 root data:ExchangeSender", + " └─ExchangeSender 12487.50 cop[tiflash] ExchangeType: PassThrough", + " └─HashJoin 12487.50 cop[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", + " ├─ExchangeReceiver(Build) 9990.00 cop[tiflash] ", + " │ └─ExchangeSender 9990.00 cop[tiflash] ExchangeType: Broadcast", + " │ └─Selection 9990.00 cop[tiflash] not(isnull(test.t.id))", + " │ └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 cop[tiflash] not(isnull(test.t.id))", + " └─TableFullScan 10000.00 cop[tiflash] table:t1 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select count(*) from (select t.id, t.value v1 from t join t t1 on t.id = t1.id order by t.value limit 20) v group by v.v1", + "Plan": [ + "HashAgg 20.00 root group by:test.t.value, funcs:count(1)->Column#7", + "└─TopN 20.00 root test.t.value, offset:0, count:20", + " └─TableReader 20.00 root data:ExchangeSender", + " └─ExchangeSender 20.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─TopN 20.00 batchCop[tiflash] test.t.value, offset:0, count:20", + " └─HashJoin 12487.50 batchCop[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", + " ├─ExchangeReceiver(Build) 9990.00 batchCop[tiflash] ", + " │ └─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: Broadcast", + " │ └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", + " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 batchCop[tiflash] not(isnull(test.t.id))", + " └─TableFullScan 10000.00 batchCop[tiflash] table:t1 keep order:false, stats:pseudo" + ] + } + ] + }, + { + "Name": "TestIndexMerge", + "Cases": [ + { + "SQL": "desc format='brief' select /*+ use_index_merge(t) */ * from t where a =1 or (b=1 and b+2>1)", + "Plan": [ + "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": [ + "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": [ + "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": [ + "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 } ] }, @@ -2452,5 +2564,31 @@ ] } ] + }, + { + "Name": "TestIssue23887", + "Cases": [ + { + "SQL": "select (2) in (select b from t) from (select t.a < (select t.a from t t1 limit 1) from t) t", + "Plan": [ + "HashJoin 10000.00 root CARTESIAN left outer semi join, other cond:eq(2, test.t.b)", + "├─TableReader(Build) 10000.00 root data:TableFullScan", + "│ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + "└─Projection(Probe) 10000.00 root 1->Column#25", + " └─Apply 10000.00 root CARTESIAN left outer join", + " ├─TableReader(Build) 10000.00 root data:TableFullScan", + " │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + " └─Projection(Probe) 1.00 root 1->Column#26", + " └─Limit 1.00 root offset:0, count:1", + " └─TableReader 1.00 root data:Limit", + " └─Limit 1.00 cop[tikv] offset:0, count:1", + " └─TableFullScan 1.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Res": [ + "1", + "1" + ] + } + ] } ] 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 5549f2b934d3e..06382a91e38b1 100644 --- a/planner/core/testdata/integration_suite_out.json +++ b/planner/core/testdata/integration_suite_out.json @@ -901,8 +901,8 @@ { "SQL": "select * from t1", "Plan": [ - "TableReader_5 3.00 root data:TableFullScan_4", - "└─TableFullScan_4 3.00 cop[tikv] table:t1 keep order:false" + "TableReader 3.00 root data:TableFullScan", + "└─TableFullScan 3.00 cop[tikv] table:t1 keep order:false" ], "Res": [ "1 111 1.1000000000 11", @@ -913,8 +913,8 @@ { "SQL": "select * from t1 where t1.a >= 1 and t1.a < 4", "Plan": [ - "TableReader_6 3.00 root data:TableRangeScan_5", - "└─TableRangeScan_5 3.00 cop[tikv] table:t1 range:[1,4), keep order:false" + "TableReader 3.00 root data:TableRangeScan", + "└─TableRangeScan 3.00 cop[tikv] table:t1 range:[1,4), keep order:false" ], "Res": [ "1 111 1.1000000000 11", @@ -925,8 +925,8 @@ { "SQL": "select * from t1 where t1.a = 1 and t1.b < \"333\"", "Plan": [ - "TableReader_6 0.67 root data:TableRangeScan_5", - "└─TableRangeScan_5 0.67 cop[tikv] table:t1 range:[1 -inf,1 \"333\"), keep order:false" + "TableReader 0.67 root data:TableRangeScan", + "└─TableRangeScan 0.67 cop[tikv] table:t1 range:[1 -inf,1 \"333\"), keep order:false" ], "Res": [ "1 111 1.1000000000 11" @@ -935,8 +935,8 @@ { "SQL": "select t1.a, t1.b, t1.c from t1 where t1.c = 3.3", "Plan": [ - "IndexReader_6 1.00 root index:IndexRangeScan_5", - "└─IndexRangeScan_5 1.00 cop[tikv] table:t1, index:c(c) range:[3.3000000000,3.3000000000], keep order:false" + "IndexReader 1.00 root index:IndexRangeScan", + "└─IndexRangeScan 1.00 cop[tikv] table:t1, index:c(c) range:[3.3000000000,3.3000000000], keep order:false" ], "Res": [ "3 333 3.3000000000" @@ -945,8 +945,8 @@ { "SQL": "select t1.b, t1.c from t1 where t1.c = 2.2", "Plan": [ - "IndexReader_6 1.00 root index:IndexRangeScan_5", - "└─IndexRangeScan_5 1.00 cop[tikv] table:t1, index:c(c) range:[2.2000000000,2.2000000000], keep order:false" + "IndexReader 1.00 root index:IndexRangeScan", + "└─IndexRangeScan 1.00 cop[tikv] table:t1, index:c(c) range:[2.2000000000,2.2000000000], keep order:false" ], "Res": [ "222 2.2000000000" @@ -955,9 +955,9 @@ { "SQL": "select /*+ use_index(t1, c) */ * from t1", "Plan": [ - "IndexLookUp_6 3.00 root ", - "├─IndexFullScan_4(Build) 3.00 cop[tikv] table:t1, index:c(c) keep order:false", - "└─TableRowIDScan_5(Probe) 3.00 cop[tikv] table:t1 keep order:false" + "IndexLookUp 3.00 root ", + "├─IndexFullScan(Build) 3.00 cop[tikv] table:t1, index:c(c) keep order:false", + "└─TableRowIDScan(Probe) 3.00 cop[tikv] table:t1 keep order:false" ], "Res": [ "1 111 1.1000000000 11", @@ -968,9 +968,9 @@ { "SQL": "select * from t1 use index(c) where t1.c in (2.2, 3.3)", "Plan": [ - "IndexLookUp_7 2.00 root ", - "├─IndexRangeScan_5(Build) 2.00 cop[tikv] table:t1, index:c(c) range:[2.2000000000,2.2000000000], [3.3000000000,3.3000000000], keep order:false", - "└─TableRowIDScan_6(Probe) 2.00 cop[tikv] table:t1 keep order:false" + "IndexLookUp 2.00 root ", + "├─IndexRangeScan(Build) 2.00 cop[tikv] table:t1, index:c(c) range:[2.2000000000,2.2000000000], [3.3000000000,3.3000000000], keep order:false", + "└─TableRowIDScan(Probe) 2.00 cop[tikv] table:t1 keep order:false" ], "Res": [ "2 222 2.2000000000 12", @@ -980,8 +980,8 @@ { "SQL": "select * from t1 where t1.a = 1 order by b", "Plan": [ - "TableReader_12 1.00 root data:TableRangeScan_11", - "└─TableRangeScan_11 1.00 cop[tikv] table:t1 range:[1,1], keep order:true" + "TableReader 1.00 root data:TableRangeScan", + "└─TableRangeScan 1.00 cop[tikv] table:t1 range:[1,1], keep order:true" ], "Res": [ "1 111 1.1000000000 11" @@ -990,10 +990,10 @@ { "SQL": "select * from t1 order by a, b limit 1", "Plan": [ - "Limit_10 1.00 root offset:0, count:1", - "└─TableReader_20 1.00 root data:Limit_19", - " └─Limit_19 1.00 cop[tikv] offset:0, count:1", - " └─TableFullScan_18 1.00 cop[tikv] table:t1 keep order:true" + "Limit 1.00 root offset:0, count:1", + "└─TableReader 1.00 root data:Limit", + " └─Limit 1.00 cop[tikv] offset:0, count:1", + " └─TableFullScan 1.00 cop[tikv] table:t1 keep order:true" ], "Res": [ "1 111 1.1000000000 11" @@ -1002,10 +1002,10 @@ { "SQL": "select /*+ use_index_merge(t1 primary, c) */ * from t1 where t1.a >= 1 or t1.c = 2.2", "Plan": [ - "IndexMerge_8 3.00 root ", - "├─TableRangeScan_5(Build) 3.00 cop[tikv] table:t1 range:[1,+inf], keep order:false", - "├─IndexRangeScan_6(Build) 1.00 cop[tikv] table:t1, index:c(c) range:[2.2000000000,2.2000000000], keep order:false", - "└─TableRowIDScan_7(Probe) 3.00 cop[tikv] table:t1 keep order:false" + "IndexMerge 3.00 root ", + "├─TableRangeScan(Build) 3.00 cop[tikv] table:t1 range:[1,+inf], keep order:false", + "├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, index:c(c) range:[2.2000000000,2.2000000000], keep order:false", + "└─TableRowIDScan(Probe) 3.00 cop[tikv] table:t1 keep order:false" ], "Res": [ "1 111 1.1000000000 11", @@ -1016,10 +1016,10 @@ { "SQL": "select /*+ use_index_merge(t1 primary, c) */ * from t1 where t1.a = 1 and t1.b = '111' or t1.c = 3.3", "Plan": [ - "IndexMerge_8 2.11 root ", - "├─TableRangeScan_5(Build) 1.00 cop[tikv] table:t1 range:[1 \"111\",1 \"111\"], keep order:false", - "├─IndexRangeScan_6(Build) 1.00 cop[tikv] table:t1, index:c(c) range:[3.3000000000,3.3000000000], keep order:false", - "└─TableRowIDScan_7(Probe) 2.11 cop[tikv] table:t1 keep order:false" + "IndexMerge 1.67 root ", + "├─TableRangeScan(Build) 1.00 cop[tikv] table:t1 range:[1 \"111\",1 \"111\"], keep order:false", + "├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, index:c(c) range:[3.3000000000,3.3000000000], keep order:false", + "└─TableRowIDScan(Probe) 1.67 cop[tikv] table:t1 keep order:false" ], "Res": [ "1 111 1.1000000000 11", @@ -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/planner/core/testdata/plan_suite_unexported_out.json b/planner/core/testdata/plan_suite_unexported_out.json index e835d0b3f66a6..d51424a97abe9 100644 --- a/planner/core/testdata/plan_suite_unexported_out.json +++ b/planner/core/testdata/plan_suite_unexported_out.json @@ -220,13 +220,13 @@ "[planner:3586]Window '': frame start or end is negative, NULL or of non-integral type", "[planner:3588]Window '' with RANGE frame has ORDER BY expression of datetime type. Only INTERVAL bound value allowed.", "TableReader(Table(t))->Window(sum(cast(test.t.a, decimal(65,0) BINARY))->Column#14 over(order by test.t.a range between 1.0 preceding and 1 following))->Projection", - "IndexReader(Index(t.f)[[NULL,+inf]])->Window(row_number()->Column#14 over())->Projection", + "IndexReader(Index(t.f)[[NULL,+inf]])->Projection->Window(row_number()->Column#14 over())->Projection", "TableReader(Table(t))->HashAgg->Window(max(Column#13)->Column#15 over(rows between 1 preceding and 1 following))->Projection", "[planner:1210]Incorrect arguments to nth_value", "[planner:1210]Incorrect arguments to nth_value", "[planner:1210]Incorrect arguments to nth_value", "[planner:1210]Incorrect arguments to ntile", - "IndexReader(Index(t.f)[[NULL,+inf]])->Window(ntile()->Column#14 over())->Projection", + "IndexReader(Index(t.f)[[NULL,+inf]])->Projection->Window(ntile()->Column#14 over())->Projection", "TableReader(Table(t))->Sort->Window(avg(cast(test.t.a, decimal(15,4) BINARY))->Column#14 over(partition by test.t.b))->Projection", "TableReader(Table(t))->Window(nth_value(test.t.i_date, 1)->Column#14 over())->Projection", "TableReader(Table(t))->Window(sum(cast(test.t.b, decimal(65,0) BINARY))->Column#15, sum(cast(test.t.c, decimal(65,0) BINARY))->Column#16 over(order by test.t.a range between unbounded preceding and current row))->Projection", @@ -293,13 +293,13 @@ "[planner:3586]Window '': frame start or end is negative, NULL or of non-integral type", "[planner:3588]Window '' with RANGE frame has ORDER BY expression of datetime type. Only INTERVAL bound value allowed.", "TableReader(Table(t))->Window(sum(cast(test.t.a, decimal(65,0) BINARY))->Column#14 over(order by test.t.a range between 1.0 preceding and 1 following))->Projection", - "IndexReader(Index(t.f)[[NULL,+inf]])->Window(row_number()->Column#14 over())->Projection", + "IndexReader(Index(t.f)[[NULL,+inf]])->Projection->Window(row_number()->Column#14 over())->Projection", "TableReader(Table(t))->HashAgg->Window(max(Column#13)->Column#15 over(rows between 1 preceding and 1 following))->Projection", "[planner:1210]Incorrect arguments to nth_value", "[planner:1210]Incorrect arguments to nth_value", "[planner:1210]Incorrect arguments to nth_value", "[planner:1210]Incorrect arguments to ntile", - "IndexReader(Index(t.f)[[NULL,+inf]])->Window(ntile()->Column#14 over())->Projection", + "IndexReader(Index(t.f)[[NULL,+inf]])->Projection->Window(ntile()->Column#14 over())->Projection", "TableReader(Table(t))->Sort->Window(avg(cast(test.t.a, decimal(15,4) BINARY))->Column#14 over(partition by test.t.b))->Partition(execution info: concurrency:4, data sources:[TableReader_10])->Projection", "TableReader(Table(t))->Window(nth_value(test.t.i_date, 1)->Column#14 over())->Projection", "TableReader(Table(t))->Window(sum(cast(test.t.b, decimal(65,0) BINARY))->Column#15, sum(cast(test.t.c, decimal(65,0) BINARY))->Column#16 over(order by test.t.a range between unbounded preceding and current row))->Projection", diff --git a/server/rpc_server.go b/server/rpc_server.go index 1bb777e171ffd..bf425ed4c86e7 100644 --- a/server/rpc_server.go +++ b/server/rpc_server.go @@ -28,9 +28,9 @@ import ( "github.com/pingcap/tidb/privilege" "github.com/pingcap/tidb/privilege/privileges" "github.com/pingcap/tidb/session" - "github.com/pingcap/tidb/store/mockstore/mocktikv" "github.com/pingcap/tidb/store/mockstore/unistore" "github.com/pingcap/tidb/store/tikv" + "github.com/pingcap/tidb/store/tikv/mockstore/mocktikv" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/memory" 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 d083f8968f41d..3bf75fcaca45f 100644 --- a/session/session.go +++ b/session/session.go @@ -509,6 +509,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 +556,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 +572,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 @@ -2093,7 +2105,9 @@ func CreateSessionWithOpt(store kv.Storage, opt *Opt) (Session, error) { // which periodically updates stats using the collected data. if do.StatsHandle() != nil && do.StatsUpdating() { s.statsCollector = do.StatsHandle().NewSessionStatsCollector() - s.idxUsageCollector = do.StatsHandle().NewSessionIndexUsageCollector() + if GetIndexUsageSyncLease() > 0 { + s.idxUsageCollector = do.StatsHandle().NewSessionIndexUsageCollector() + } } return s, nil @@ -2548,6 +2562,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_test.go b/session/session_test.go index 6ba37b08db05b..1799733db035d 100644 --- a/session/session_test.go +++ b/session/session_test.go @@ -48,13 +48,14 @@ import ( "github.com/pingcap/tidb/store/copr" "github.com/pingcap/tidb/store/driver" "github.com/pingcap/tidb/store/mockstore" - "github.com/pingcap/tidb/store/mockstore/mocktikv" + "github.com/pingcap/tidb/store/mockstore/mockcopr" "github.com/pingcap/tidb/store/tikv" tikvstore "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/mockstore/cluster" "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" @@ -258,7 +259,7 @@ func (m mockPumpPullBinlogsClient) Recv() (*binlog.PullBinlogResp, error) { } func (p *mockBinlogPump) PullBinlogs(ctx context.Context, in *binlog.PullBinlogReq, opts ...grpc.CallOption) (binlog.Pump_PullBinlogsClient, error) { - return mockPumpPullBinlogsClient{mocktikv.MockGRPCClientStream()}, nil + return mockPumpPullBinlogsClient{mockcopr.MockGRPCClientStream()}, nil } func (s *testSessionSuite) TestForCoverage(c *C) { @@ -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") @@ -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;") @@ -3314,7 +3317,7 @@ PARTITION BY RANGE (c) ( GroupID: groupID, Role: placement.Leader, Count: 1, - LabelConstraints: []placement.LabelConstraint{ + LabelConstraints: []placement.Constraint{ { Key: placement.DCLabelKey, Op: placement.In, @@ -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/session/tidb.go b/session/tidb.go index 8dc13ed580d65..85732b457f7a6 100644 --- a/session/tidb.go +++ b/session/tidb.go @@ -65,7 +65,7 @@ func (dm *domainMap) Get(store kv.Storage) (d *domain.Domain, err error) { ddlLease := time.Duration(atomic.LoadInt64(&schemaLease)) statisticLease := time.Duration(atomic.LoadInt64(&statsLease)) - idxUsageSyncLease := time.Duration(atomic.LoadInt64(&indexUsageSyncLease)) + idxUsageSyncLease := GetIndexUsageSyncLease() err = util.RunWithRetry(util.DefaultMaxRetries, util.RetryInterval, func() (retry bool, err1 error) { logutil.BgLogger().Info("new domain", zap.String("store", store.UUID()), @@ -161,6 +161,11 @@ func SetIndexUsageSyncLease(lease time.Duration) { atomic.StoreInt64(&indexUsageSyncLease, int64(lease)) } +// GetIndexUsageSyncLease returns the index usage sync lease time. +func GetIndexUsageSyncLease() time.Duration { + return time.Duration(atomic.LoadInt64(&indexUsageSyncLease)) +} + // DisableStats4Test disables the stats for tests. func DisableStats4Test() { SetStatsLease(-1) diff --git a/session/tidb_test.go b/session/tidb_test.go index 02e9571c18482..615f388ff1e2e 100644 --- a/session/tidb_test.go +++ b/session/tidb_test.go @@ -215,3 +215,28 @@ func (s *testMainSuite) TestKeysNeedLock(c *C) { c.Assert(flag.HasPresumeKeyNotExists(), IsTrue) c.Assert(keyNeedToLock(indexKey, deleteVal, flag), IsTrue) } + +func (s *testMainSuite) TestIndexUsageSyncLease(c *C) { + store, err := mockstore.NewMockStore() + c.Assert(err, IsNil) + do, err := BootstrapSession(store) + c.Assert(err, IsNil) + do.SetStatsUpdating(true) + st, err := CreateSessionWithOpt(store, nil) + c.Assert(err, IsNil) + se, ok := st.(*session) + c.Assert(ok, IsTrue) + c.Assert(se.idxUsageCollector, IsNil) + + SetIndexUsageSyncLease(1) + defer SetIndexUsageSyncLease(0) + st, err = CreateSessionWithOpt(store, nil) + c.Assert(err, IsNil) + se, ok = st.(*session) + c.Assert(ok, IsTrue) + c.Assert(se.idxUsageCollector, NotNil) + + do.Close() + err = store.Close() + c.Assert(err, IsNil) +} 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/session.go b/sessionctx/variable/session.go index d4d83d456f0b3..77f679c3089a3 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -826,6 +826,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 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/sysvar.go b/sessionctx/variable/sysvar.go index ec693444018eb..851516caa6588 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -1231,6 +1231,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 @@ -1640,6 +1644,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..dedc4d8aaba5b 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. diff --git a/sessionctx/variable/varsutil.go b/sessionctx/variable/varsutil.go index 9af28d6b90d04..70aa047443e87 100644 --- a/sessionctx/variable/varsutil.go +++ b/sessionctx/variable/varsutil.go @@ -404,6 +404,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/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 7f967031a61c7..f60d57a984b3b 100644 --- a/statistics/handle/handle_test.go +++ b/statistics/handle/handle_test.go @@ -2058,6 +2058,8 @@ type statsSerialSuite struct { func (s *statsSerialSuite) TestIndexUsageInformation(c *C) { defer cleanEnv(c, s.store, s.do) + session.SetIndexUsageSyncLease(1) + defer session.SetIndexUsageSyncLease(0) tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("create table t_idx(a int, b int)") @@ -2097,6 +2099,8 @@ func (s *statsSerialSuite) TestIndexUsageInformation(c *C) { func (s *statsSerialSuite) TestGCIndexUsageInformation(c *C) { defer cleanEnv(c, s.store, s.do) + session.SetIndexUsageSyncLease(1) + defer session.SetIndexUsageSyncLease(0) tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("create table t_idx(a int, b int)") @@ -2730,3 +2734,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/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/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 8dd4501d89e9c..aa2e21e89a0c6 100644 --- a/store/copr/batch_coprocessor.go +++ b/store/copr/batch_coprocessor.go @@ -323,7 +323,7 @@ func (b *batchCopIterator) retryBatchCopTask(ctx context.Context, bo *tikv.Backo func (b *batchCopIterator) handleTaskOnce(ctx context.Context, bo *tikv.Backoffer, task *batchCopTask) ([]*batchCopTask, error) { sender := NewRegionBatchRequestSender(b.store.GetRegionCache(), b.store.GetTiKVClient()) - var regionInfos []*coprocessor.RegionInfo + var regionInfos = make([]*coprocessor.RegionInfo, 0, len(task.copTasks)) for _, task := range task.copTasks { regionInfos = append(regionInfos, &coprocessor.RegionInfo{ RegionId: task.task.region.GetID(), diff --git a/store/copr/coprocessor.go b/store/copr/coprocessor.go index 484f6b078e3c8..481b6fd363293 100644 --- a/store/copr/coprocessor.go +++ b/store/copr/coprocessor.go @@ -912,8 +912,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_test.go b/store/copr/coprocessor_test.go index b7769096f973e..fbd029f72f016 100644 --- a/store/copr/coprocessor_test.go +++ b/store/copr/coprocessor_test.go @@ -18,9 +18,9 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/store/mockstore/mocktikv" "github.com/pingcap/tidb/store/tikv" tikvstore "github.com/pingcap/tidb/store/tikv/kv" + "github.com/pingcap/tidb/store/tikv/mockstore/mocktikv" ) type testCoprocessorSuite struct { diff --git a/store/driver/sql_fail_test.go b/store/driver/sql_fail_test.go index 22306c69cbed2..903dcedcb1878 100644 --- a/store/driver/sql_fail_test.go +++ b/store/driver/sql_fail_test.go @@ -75,11 +75,11 @@ func (s *testSQLSerialSuite) TestFailBusyServerCop(c *C) { var wg sync.WaitGroup wg.Add(2) - c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/mockstore/mocktikv/rpcServerBusy", `return(true)`), IsNil) + c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/mockstore/mocktikv/rpcServerBusy", `return(true)`), IsNil) go func() { defer wg.Done() time.Sleep(time.Millisecond * 100) - c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/mockstore/mocktikv/rpcServerBusy"), IsNil) + c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/mockstore/mocktikv/rpcServerBusy"), IsNil) }() go func() { diff --git a/store/gcworker/gc_worker_test.go b/store/gcworker/gc_worker_test.go index bb5f27dd35a7a..3bfd616929aec 100644 --- a/store/gcworker/gc_worker_test.go +++ b/store/gcworker/gc_worker_test.go @@ -38,9 +38,9 @@ import ( "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/store/mockstore" - "github.com/pingcap/tidb/store/mockstore/mocktikv" "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/store/tikv/mockstore/cluster" + "github.com/pingcap/tidb/store/tikv/mockstore/mocktikv" "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/store/tikv/oracle/oracles" "github.com/pingcap/tidb/store/tikv/tikvrpc" diff --git a/store/mockstore/mocktikv/cluster_test.go b/store/mockstore/cluster_test.go similarity index 91% rename from store/mockstore/mocktikv/cluster_test.go rename to store/mockstore/cluster_test.go index 0e85fb5b6ca33..2e34edad55266 100644 --- a/store/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" @@ -24,8 +24,8 @@ import ( "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/sessionctx/stmtctx" - "github.com/pingcap/tidb/store/mockstore/mocktikv" "github.com/pingcap/tidb/store/tikv" + "github.com/pingcap/tidb/store/tikv/mockstore/mocktikv" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/codec" @@ -39,7 +39,7 @@ type testClusterSuite struct { } func (s *testClusterSuite) TestClusterSplit(c *C) { - rpcClient, cluster, pdClient, err := mocktikv.NewTiKVAndPDClient("") + rpcClient, cluster, pdClient, err := mocktikv.NewTiKVAndPDClient("", nil) c.Assert(err, IsNil) mocktikv.BootstrapWithSingleStore(cluster) mvccStore := rpcClient.MvccStore @@ -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/mocktikv/aggregate.go b/store/mockstore/mockcopr/aggregate.go similarity index 99% rename from store/mockstore/mocktikv/aggregate.go rename to store/mockstore/mockcopr/aggregate.go index ff8afbb963c6f..25f7c84611cbb 100644 --- a/store/mockstore/mocktikv/aggregate.go +++ b/store/mockstore/mockcopr/aggregate.go @@ -11,7 +11,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package mocktikv +package mockcopr import ( "context" diff --git a/store/mockstore/mocktikv/analyze.go b/store/mockstore/mockcopr/analyze.go similarity index 97% rename from store/mockstore/mocktikv/analyze.go rename to store/mockstore/mockcopr/analyze.go index 24afa5197e3fb..b016096314de4 100644 --- a/store/mockstore/mocktikv/analyze.go +++ b/store/mockstore/mockcopr/analyze.go @@ -11,7 +11,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package mocktikv +package mockcopr import ( "context" @@ -75,8 +75,8 @@ func (h coprHandler) handleAnalyzeIndexReq(req *coprocessor.Request, analyzeReq colsLen: int(analyzeReq.IdxReq.NumColumns), kvRanges: ranges, startTS: startTS, - isolationLevel: h.isolationLevel, - mvccStore: h.mvccStore, + isolationLevel: h.GetIsolationLevel(), + mvccStore: h.GetMVCCStore(), IndexScan: &tipb.IndexScan{Desc: false}, execDetail: new(execDetail), hdStatus: tablecodec.HandleNotNeeded, @@ -170,8 +170,8 @@ func (h coprHandler) handleAnalyzeColumnsReq(req *coprocessor.Request, analyzeRe kvRanges: ranges, colIDs: evalCtx.colIDs, startTS: startTS, - isolationLevel: h.isolationLevel, - mvccStore: h.mvccStore, + isolationLevel: h.GetIsolationLevel(), + mvccStore: h.GetMVCCStore(), execDetail: new(execDetail), rd: rd, }, diff --git a/store/mockstore/mocktikv/checksum.go b/store/mockstore/mockcopr/checksum.go similarity index 98% rename from store/mockstore/mocktikv/checksum.go rename to store/mockstore/mockcopr/checksum.go index 5c99a55ee70bf..1ba4e44b400c2 100644 --- a/store/mockstore/mocktikv/checksum.go +++ b/store/mockstore/mockcopr/checksum.go @@ -11,7 +11,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package mocktikv +package mockcopr import ( "fmt" diff --git a/store/mockstore/mocktikv/cop_handler_dag.go b/store/mockstore/mockcopr/cop_handler_dag.go similarity index 99% rename from store/mockstore/mocktikv/cop_handler_dag.go rename to store/mockstore/mockcopr/cop_handler_dag.go index a42a3a1974521..577c7c4e4e853 100644 --- a/store/mockstore/mocktikv/cop_handler_dag.go +++ b/store/mockstore/mockcopr/cop_handler_dag.go @@ -11,7 +11,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package mocktikv +package mockcopr import ( "bytes" @@ -32,6 +32,7 @@ import ( "github.com/pingcap/tidb/expression/aggregation" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/sessionctx/stmtctx" + "github.com/pingcap/tidb/store/tikv/mockstore/mocktikv" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" @@ -591,7 +592,7 @@ func (mock *mockCopStreamClient) Recv() (*coprocessor.Response, error) { chunk, finish, ran, counts, warnings, err := mock.readBlockFromExecutor() resp.Range = ran if err != nil { - if locked, ok := errors.Cause(err).(*ErrLocked); ok { + if locked, ok := errors.Cause(err).(*mocktikv.ErrLocked); ok { resp.Locked = &kvrpcpb.LockInfo{ Key: locked.Key, PrimaryLock: locked.Primary, @@ -783,7 +784,7 @@ func buildResp(selResp *tipb.SelectResponse, execDetails []*execDetail, err erro } // Select errors have been contained in `SelectResponse.Error` - if locked, ok := errors.Cause(err).(*ErrLocked); ok { + if locked, ok := errors.Cause(err).(*mocktikv.ErrLocked); ok { resp.Locked = &kvrpcpb.LockInfo{ Key: locked.Key, PrimaryLock: locked.Primary, diff --git a/store/mockstore/mocktikv/cop_handler_dag_test.go b/store/mockstore/mockcopr/cop_handler_dag_test.go similarity index 99% rename from store/mockstore/mocktikv/cop_handler_dag_test.go rename to store/mockstore/mockcopr/cop_handler_dag_test.go index 87c2947af6ca2..201980f91e1e4 100644 --- a/store/mockstore/mocktikv/cop_handler_dag_test.go +++ b/store/mockstore/mockcopr/cop_handler_dag_test.go @@ -11,7 +11,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package mocktikv +package mockcopr import ( "time" diff --git a/store/mockstore/mocktikv/copr_handler.go b/store/mockstore/mockcopr/copr_handler.go similarity index 94% rename from store/mockstore/mocktikv/copr_handler.go rename to store/mockstore/mockcopr/copr_handler.go index ea9b16acd88e2..c7be9616fded2 100644 --- a/store/mockstore/mocktikv/copr_handler.go +++ b/store/mockstore/mockcopr/copr_handler.go @@ -11,7 +11,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package mocktikv +package mockcopr import ( "context" @@ -19,11 +19,12 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/coprocessor" "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/store/tikv/mockstore/mocktikv" "github.com/pingcap/tipb/go-tipb" ) type coprHandler struct { - *Session + *mocktikv.Session } func (h coprHandler) handleBatchCopRequest(ctx context.Context, req *coprocessor.BatchRequest) (*mockBatchCopDataClient, error) { diff --git a/store/mockstore/mocktikv/executor.go b/store/mockstore/mockcopr/executor.go similarity index 98% rename from store/mockstore/mocktikv/executor.go rename to store/mockstore/mockcopr/executor.go index 7a90b1f15af81..b5d54b74f6ca1 100644 --- a/store/mockstore/mocktikv/executor.go +++ b/store/mockstore/mockcopr/executor.go @@ -11,7 +11,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package mocktikv +package mockcopr import ( "bytes" @@ -26,6 +26,7 @@ import ( "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/sessionctx/stmtctx" + "github.com/pingcap/tidb/store/tikv/mockstore/mocktikv" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" @@ -76,7 +77,7 @@ type tableScanExec struct { startTS uint64 isolationLevel kvrpcpb.IsolationLevel resolvedLocks []uint64 - mvccStore MVCCStore + mvccStore mocktikv.MVCCStore cursor int seekKey []byte start int @@ -208,8 +209,8 @@ func (e *tableScanExec) getRowFromRange(ran kv.KeyRange) ([][]byte, error) { e.seekKey = ran.StartKey } } - var pairs []Pair - var pair Pair + var pairs []mocktikv.Pair + var pair mocktikv.Pair if e.Desc { pairs = e.mvccStore.ReverseScan(ran.StartKey, e.seekKey, 1, e.startTS, e.isolationLevel, e.resolvedLocks) } else { @@ -255,7 +256,7 @@ type indexScanExec struct { startTS uint64 isolationLevel kvrpcpb.IsolationLevel resolvedLocks []uint64 - mvccStore MVCCStore + mvccStore mocktikv.MVCCStore cursor int seekKey []byte hdStatus tablecodec.HandleStatus @@ -382,8 +383,8 @@ func (e *indexScanExec) getRowFromRange(ran kv.KeyRange) ([][]byte, error) { e.seekKey = ran.StartKey } } - var pairs []Pair - var pair Pair + var pairs []mocktikv.Pair + var pair mocktikv.Pair if e.Desc { pairs = e.mvccStore.ReverseScan(ran.StartKey, e.seekKey, 1, e.startTS, e.isolationLevel, e.resolvedLocks) } else { diff --git a/store/mockstore/mocktikv/executor_test.go b/store/mockstore/mockcopr/executor_test.go similarity index 96% rename from store/mockstore/mocktikv/executor_test.go rename to store/mockstore/mockcopr/executor_test.go index a7f17ca8cc932..c442b0cbac767 100644 --- a/store/mockstore/mocktikv/executor_test.go +++ b/store/mockstore/mockcopr/executor_test.go @@ -11,7 +11,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package mocktikv_test +package mockcopr_test import ( "context" @@ -23,9 +23,10 @@ import ( "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/session" + "github.com/pingcap/tidb/store/mockstore/mockcopr" "github.com/pingcap/tidb/store/mockstore/mockstorage" - "github.com/pingcap/tidb/store/mockstore/mocktikv" "github.com/pingcap/tidb/store/tikv" + "github.com/pingcap/tidb/store/tikv/mockstore/mocktikv" "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/util/testkit" @@ -41,7 +42,7 @@ type testExecutorSuite struct { } func (s *testExecutorSuite) SetUpSuite(c *C) { - rpcClient, cluster, pdClient, err := mocktikv.NewTiKVAndPDClient("") + rpcClient, cluster, pdClient, err := mocktikv.NewTiKVAndPDClient("", mockcopr.NewCoprRPCHandler()) c.Assert(err, IsNil) mocktikv.BootstrapWithSingleStore(cluster) s.cluster = cluster diff --git a/store/mockstore/mocktikv/rpc_copr.go b/store/mockstore/mockcopr/rpc_copr.go similarity index 85% rename from store/mockstore/mocktikv/rpc_copr.go rename to store/mockstore/mockcopr/rpc_copr.go index 50a42f13d7ca9..c85a518348c83 100644 --- a/store/mockstore/mocktikv/rpc_copr.go +++ b/store/mockstore/mockcopr/rpc_copr.go @@ -11,7 +11,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package mocktikv +package mockcopr import ( "context" @@ -22,6 +22,7 @@ import ( "github.com/pingcap/kvproto/pkg/coprocessor" "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/store/tikv/mockstore/mocktikv" "github.com/pingcap/tidb/store/tikv/tikvrpc" ) @@ -30,7 +31,8 @@ type coprRPCHandler struct { done chan struct{} } -func newCoprRPCHandler() *coprRPCHandler { +// NewCoprRPCHandler creates a handler to process coprocessor requests. +func NewCoprRPCHandler() mocktikv.CoprRPCHandler { ch := make(chan *tikvrpc.Lease, 1024) done := make(chan struct{}) go tikvrpc.CheckStreamTimeoutLoop(ch, done) @@ -40,7 +42,7 @@ func newCoprRPCHandler() *coprRPCHandler { } } -func (mc *coprRPCHandler) HandleCmdCop(reqCtx *kvrpcpb.Context, session *Session, r *coprocessor.Request) *coprocessor.Response { +func (mc *coprRPCHandler) HandleCmdCop(reqCtx *kvrpcpb.Context, session *mocktikv.Session, r *coprocessor.Request) *coprocessor.Response { if err := session.CheckRequestContext(reqCtx); err != nil { return &coprocessor.Response{RegionError: err} } @@ -58,7 +60,7 @@ func (mc *coprRPCHandler) HandleCmdCop(reqCtx *kvrpcpb.Context, session *Session return res } -func (mc *coprRPCHandler) HandleBatchCop(ctx context.Context, reqCtx *kvrpcpb.Context, session *Session, r *coprocessor.BatchRequest, timeout time.Duration) (*tikvrpc.BatchCopStreamResponse, error) { +func (mc *coprRPCHandler) HandleBatchCop(ctx context.Context, reqCtx *kvrpcpb.Context, session *mocktikv.Session, r *coprocessor.BatchRequest, timeout time.Duration) (*tikvrpc.BatchCopStreamResponse, error) { if err := session.CheckRequestContext(reqCtx); err != nil { return &tikvrpc.BatchCopStreamResponse{ Tikv_BatchCoprocessorClient: &mockBathCopErrClient{Error: err}, @@ -86,7 +88,7 @@ func (mc *coprRPCHandler) HandleBatchCop(ctx context.Context, reqCtx *kvrpcpb.Co return batchResp, nil } -func (mc *coprRPCHandler) HandleCopStream(ctx context.Context, reqCtx *kvrpcpb.Context, session *Session, r *coprocessor.Request, timeout time.Duration) (*tikvrpc.CopStreamResponse, error) { +func (mc *coprRPCHandler) HandleCopStream(ctx context.Context, reqCtx *kvrpcpb.Context, session *mocktikv.Session, r *coprocessor.Request, timeout time.Duration) (*tikvrpc.CopStreamResponse, error) { if err := session.CheckRequestContext(reqCtx); err != nil { return &tikvrpc.CopStreamResponse{ Tikv_CoprocessorStreamClient: &mockCopStreamErrClient{Error: err}, diff --git a/store/mockstore/mocktikv/topn.go b/store/mockstore/mockcopr/topn.go similarity index 99% rename from store/mockstore/mocktikv/topn.go rename to store/mockstore/mockcopr/topn.go index 3b2a5d952c48a..7b1d79948e329 100644 --- a/store/mockstore/mocktikv/topn.go +++ b/store/mockstore/mockcopr/topn.go @@ -11,7 +11,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package mocktikv +package mockcopr import ( "container/heap" diff --git a/store/mockstore/mockstore.go b/store/mockstore/mockstore.go index 77fb061ae56bf..b7802c26c8fa7 100644 --- a/store/mockstore/mockstore.go +++ b/store/mockstore/mockstore.go @@ -20,10 +20,10 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/store/mockstore/mocktikv" "github.com/pingcap/tidb/store/mockstore/unistore" "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/store/tikv/mockstore/cluster" + "github.com/pingcap/tidb/store/tikv/mockstore/mocktikv" pd "github.com/tikv/pd/client" ) diff --git a/store/mockstore/tikv.go b/store/mockstore/tikv.go index b6289d639e381..10378a51e34b2 100644 --- a/store/mockstore/tikv.go +++ b/store/mockstore/tikv.go @@ -16,15 +16,16 @@ package mockstore import ( "github.com/pingcap/errors" "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/store/mockstore/mockcopr" "github.com/pingcap/tidb/store/mockstore/mockstorage" - "github.com/pingcap/tidb/store/mockstore/mocktikv" "github.com/pingcap/tidb/store/tikv" + "github.com/pingcap/tidb/store/tikv/mockstore/mocktikv" ) // newMockTikvStore creates a mocked tikv store, the path is the file path to store the data. // If path is an empty string, a memory storage will be created. func newMockTikvStore(opt *mockOptions) (kv.Storage, error) { - client, cluster, pdClient, err := mocktikv.NewTiKVAndPDClient(opt.path) + client, cluster, pdClient, err := mocktikv.NewTiKVAndPDClient(opt.path, mockcopr.NewCoprRPCHandler()) if err != nil { return nil, errors.Trace(err) } 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/tikv/mvcc_test.go b/store/mockstore/unistore/tikv/mvcc_test.go index 858848430eb51..323e50aea9f89 100644 --- a/store/mockstore/unistore/tikv/mvcc_test.go +++ b/store/mockstore/unistore/tikv/mvcc_test.go @@ -1501,8 +1501,8 @@ func (s *testMvccSuite) TestResolveCommit(c *C) { } func MustLoad(startTS, commitTS uint64, store *TestStore, pairs ...string) { - var keys [][]byte - var vals [][]byte + var keys = make([][]byte, 0, len(pairs)) + var vals = make([][]byte, 0, len(pairs)) for _, pair := range pairs { strs := strings.Split(pair, ":") keys = append(keys, []byte(strs[0])) 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/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/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/util/deadlock/deadlock.go b/store/tikv/mockstore/deadlock/deadlock.go similarity index 100% rename from util/deadlock/deadlock.go rename to store/tikv/mockstore/deadlock/deadlock.go diff --git a/util/deadlock/deadlock_test.go b/store/tikv/mockstore/deadlock/deadlock_test.go similarity index 100% rename from util/deadlock/deadlock_test.go rename to store/tikv/mockstore/deadlock/deadlock_test.go diff --git a/store/mockstore/mocktikv/cluster.go b/store/tikv/mockstore/mocktikv/cluster.go similarity index 96% rename from store/mockstore/mocktikv/cluster.go rename to store/tikv/mockstore/mocktikv/cluster.go index abeade418cd69..5d962564568f3 100644 --- a/store/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/mockstore/mocktikv/cluster_manipulate.go b/store/tikv/mockstore/mocktikv/cluster_manipulate.go similarity index 100% rename from store/mockstore/mocktikv/cluster_manipulate.go rename to store/tikv/mockstore/mocktikv/cluster_manipulate.go diff --git a/store/mockstore/mocktikv/errors.go b/store/tikv/mockstore/mocktikv/errors.go similarity index 100% rename from store/mockstore/mocktikv/errors.go rename to store/tikv/mockstore/mocktikv/errors.go diff --git a/store/mockstore/mocktikv/mock.go b/store/tikv/mockstore/mocktikv/mock.go similarity index 80% rename from store/mockstore/mocktikv/mock.go rename to store/tikv/mockstore/mocktikv/mock.go index be124b735a1fb..59502ba42e00c 100644 --- a/store/mockstore/mocktikv/mock.go +++ b/store/tikv/mockstore/mocktikv/mock.go @@ -19,12 +19,12 @@ import ( ) // NewTiKVAndPDClient creates a TiKV client and PD client from options. -func NewTiKVAndPDClient(path string) (*RPCClient, *Cluster, pd.Client, error) { +func NewTiKVAndPDClient(path string, coprHandler CoprRPCHandler) (*RPCClient, *Cluster, pd.Client, error) { mvccStore, err := NewMVCCLevelDB(path) if err != nil { return nil, nil, nil, errors.Trace(err) } cluster := NewCluster(mvccStore) - return NewRPCClient(cluster, mvccStore), cluster, NewPDClient(cluster), nil + return NewRPCClient(cluster, mvccStore, coprHandler), cluster, NewPDClient(cluster), nil } diff --git a/store/mockstore/mocktikv/mock_tikv_test.go b/store/tikv/mockstore/mocktikv/mock_tikv_test.go similarity index 100% rename from store/mockstore/mocktikv/mock_tikv_test.go rename to store/tikv/mockstore/mocktikv/mock_tikv_test.go diff --git a/store/mockstore/mocktikv/mvcc.go b/store/tikv/mockstore/mocktikv/mvcc.go similarity index 99% rename from store/mockstore/mocktikv/mvcc.go rename to store/tikv/mockstore/mocktikv/mvcc.go index 98cbbb6ce0ce3..8d0883dd043a5 100644 --- a/store/mockstore/mocktikv/mvcc.go +++ b/store/tikv/mockstore/mocktikv/mvcc.go @@ -22,7 +22,7 @@ import ( "github.com/google/btree" "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/kvrpcpb" - "github.com/pingcap/tidb/util/codec" + "github.com/pingcap/tidb/store/tikv/util/codec" ) type mvccValueType int diff --git a/store/mockstore/mocktikv/mvcc_leveldb.go b/store/tikv/mockstore/mocktikv/mvcc_leveldb.go similarity index 98% rename from store/mockstore/mocktikv/mvcc_leveldb.go rename to store/tikv/mockstore/mocktikv/mvcc_leveldb.go index 5b101d1081572..ab9b7dc5987a7 100644 --- a/store/mockstore/mocktikv/mvcc_leveldb.go +++ b/store/tikv/mockstore/mocktikv/mvcc_leveldb.go @@ -27,14 +27,21 @@ import ( "github.com/pingcap/goleveldb/leveldb/util" "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/parser/terror" - "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/store/tikv/logutil" + "github.com/pingcap/tidb/store/tikv/mockstore/deadlock" "github.com/pingcap/tidb/store/tikv/oracle" - "github.com/pingcap/tidb/util/codec" - "github.com/pingcap/tidb/util/deadlock" - "github.com/pingcap/tidb/util/logutil" + "github.com/pingcap/tidb/store/tikv/util/codec" "go.uber.org/zap" ) +// Used for pessimistic lock wait time +// these two constants are special for lock protocol with tikv +// 0 means always wait, -1 means nowait, others meaning lock wait in milliseconds +var ( + LockAlwaysWait = int64(0) + LockNoWait = int64(-1) +) + // MVCCLevelDB implements the MVCCStore interface. type MVCCLevelDB struct { // Key layout: @@ -502,14 +509,14 @@ func (mvcc *MVCCLevelDB) PessimisticLock(req *kvrpcpb.PessimisticLockRequest) *k if err != nil { anyError = true } - if lockWaitTime == kv.LockNoWait { + if lockWaitTime == LockNoWait { if _, ok := err.(*ErrLocked); ok { break } } } if anyError { - if lockWaitTime != kv.LockNoWait { + if lockWaitTime != LockNoWait { // TODO: remove this when implement sever side wait. simulateServerSideWaitLock(errs) } diff --git a/store/mockstore/mocktikv/mvcc_test.go b/store/tikv/mockstore/mocktikv/mvcc_test.go similarity index 100% rename from store/mockstore/mocktikv/mvcc_test.go rename to store/tikv/mockstore/mocktikv/mvcc_test.go diff --git a/store/mockstore/mocktikv/pd.go b/store/tikv/mockstore/mocktikv/pd.go similarity index 100% rename from store/mockstore/mocktikv/pd.go rename to store/tikv/mockstore/mocktikv/pd.go diff --git a/store/mockstore/mocktikv/rpc.go b/store/tikv/mockstore/mocktikv/rpc.go similarity index 99% rename from store/mockstore/mocktikv/rpc.go rename to store/tikv/mockstore/mocktikv/rpc.go index 5c2ef3893d313..85852b7506dc8 100644 --- a/store/mockstore/mocktikv/rpc.go +++ b/store/tikv/mockstore/mocktikv/rpc.go @@ -582,11 +582,11 @@ type RPCClient struct { // NewRPCClient creates an RPCClient. // Note that close the RPCClient may close the underlying MvccStore. -func NewRPCClient(cluster *Cluster, mvccStore MVCCStore) *RPCClient { +func NewRPCClient(cluster *Cluster, mvccStore MVCCStore, coprHandler CoprRPCHandler) *RPCClient { return &RPCClient{ Cluster: cluster, MvccStore: mvccStore, - coprHandler: newCoprRPCHandler(), + coprHandler: coprHandler, } } diff --git a/store/mockstore/mocktikv/session.go b/store/tikv/mockstore/mocktikv/session.go similarity index 97% rename from store/mockstore/mocktikv/session.go rename to store/tikv/mockstore/mocktikv/session.go index 9e9d352b7b1e3..7997c6c9f4ca7 100644 --- a/store/mockstore/mocktikv/session.go +++ b/store/tikv/mockstore/mocktikv/session.go @@ -18,7 +18,6 @@ import ( "github.com/pingcap/kvproto/pkg/errorpb" "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/kvproto/pkg/metapb" - "github.com/pingcap/tidb/ddl/placement" ) // Session stores session scope rpc data. @@ -164,7 +163,7 @@ func (s *Session) checkKeyInRegion(key []byte) bool { func isTiFlashStore(store *metapb.Store) bool { for _, l := range store.GetLabels() { - if l.GetKey() == placement.EngineLabelKey && l.GetValue() == placement.EngineLabelTiFlash { + if l.GetKey() == "engine" && l.GetValue() == "tiflash" { return true } } diff --git a/store/mockstore/mocktikv/utils.go b/store/tikv/mockstore/mocktikv/utils.go similarity index 100% rename from store/mockstore/mocktikv/utils.go rename to store/tikv/mockstore/mocktikv/utils.go 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/region_cache_test.go b/store/tikv/region_cache_test.go index cfbf043774b06..592541eb5b9d2 100644 --- a/store/tikv/region_cache_test.go +++ b/store/tikv/region_cache_test.go @@ -26,8 +26,8 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/kvproto/pkg/errorpb" "github.com/pingcap/kvproto/pkg/metapb" - "github.com/pingcap/tidb/store/mockstore/mocktikv" "github.com/pingcap/tidb/store/tikv/kv" + "github.com/pingcap/tidb/store/tikv/mockstore/mocktikv" pd "github.com/tikv/pd/client" ) @@ -922,7 +922,7 @@ func (s *testRegionCacheSuite) TestUpdateStoreAddr(c *C) { client := &RawKVClient{ clusterID: 0, regionCache: NewRegionCache(mocktikv.NewPDClient(s.cluster)), - rpcClient: mocktikv.NewRPCClient(s.cluster, mvccStore), + rpcClient: mocktikv.NewRPCClient(s.cluster, mvccStore, nil), } defer client.Close() testKey := []byte("test_key") @@ -947,7 +947,7 @@ func (s *testRegionCacheSuite) TestReplaceAddrWithNewStore(c *C) { client := &RawKVClient{ clusterID: 0, regionCache: NewRegionCache(mocktikv.NewPDClient(s.cluster)), - rpcClient: mocktikv.NewRPCClient(s.cluster, mvccStore), + rpcClient: mocktikv.NewRPCClient(s.cluster, mvccStore, nil), } defer client.Close() testKey := []byte("test_key") @@ -976,7 +976,7 @@ func (s *testRegionCacheSuite) TestReplaceNewAddrAndOldOfflineImmediately(c *C) client := &RawKVClient{ clusterID: 0, regionCache: NewRegionCache(mocktikv.NewPDClient(s.cluster)), - rpcClient: mocktikv.NewRPCClient(s.cluster, mvccStore), + rpcClient: mocktikv.NewRPCClient(s.cluster, mvccStore, nil), } defer client.Close() testKey := []byte("test_key") @@ -1012,7 +1012,7 @@ func (s *testRegionCacheSuite) TestReplaceStore(c *C) { client := &RawKVClient{ clusterID: 0, regionCache: NewRegionCache(mocktikv.NewPDClient(s.cluster)), - rpcClient: mocktikv.NewRPCClient(s.cluster, mvccStore), + rpcClient: mocktikv.NewRPCClient(s.cluster, mvccStore, nil), } defer client.Close() testKey := []byte("test_key") diff --git a/store/tikv/region_request_test.go b/store/tikv/region_request_test.go index 92c171972093f..2f3056f3683f4 100644 --- a/store/tikv/region_request_test.go +++ b/store/tikv/region_request_test.go @@ -32,8 +32,8 @@ import ( "github.com/pingcap/kvproto/pkg/tikvpb" "github.com/pingcap/tidb/store/tikv/kv" - "github.com/pingcap/tidb/store/mockstore/mocktikv" "github.com/pingcap/tidb/store/tikv/config" + "github.com/pingcap/tidb/store/tikv/mockstore/mocktikv" "github.com/pingcap/tidb/store/tikv/tikvrpc" "google.golang.org/grpc" ) @@ -72,7 +72,7 @@ func (s *testRegionRequestToSingleStoreSuite) SetUpTest(c *C) { s.cache = NewRegionCache(pdCli) s.bo = NewNoopBackoff(context.Background()) s.mvccStore = mocktikv.MustNewMVCCStore() - client := mocktikv.NewRPCClient(s.cluster, s.mvccStore) + client := mocktikv.NewRPCClient(s.cluster, s.mvccStore, nil) s.regionRequestSender = NewRegionRequestSender(s.cache, client) } @@ -83,7 +83,7 @@ func (s *testRegionRequestToThreeStoresSuite) SetUpTest(c *C) { s.cache = NewRegionCache(pdCli) s.bo = NewNoopBackoff(context.Background()) s.mvccStore = mocktikv.MustNewMVCCStore() - client := mocktikv.NewRPCClient(s.cluster, s.mvccStore) + client := mocktikv.NewRPCClient(s.cluster, s.mvccStore, nil) s.regionRequestSender = NewRegionRequestSender(s.cache, client) } diff --git a/store/tikv/snapshot.go b/store/tikv/snapshot.go index 904b8653093ac..39f2d3e1e66b8 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" ) @@ -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/2pc_fail_test.go b/store/tikv/tests/2pc_fail_test.go index 0c8365e7820a8..ed8820a302fe0 100644 --- a/store/tikv/tests/2pc_fail_test.go +++ b/store/tikv/tests/2pc_fail_test.go @@ -26,9 +26,9 @@ import ( // TestFailCommitPrimaryRpcErrors tests rpc errors are handled properly when // committing primary region task. func (s *testCommitterSuite) TestFailCommitPrimaryRpcErrors(c *C) { - c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/mockstore/mocktikv/rpcCommitResult", `return("timeout")`), IsNil) + c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/mockstore/mocktikv/rpcCommitResult", `return("timeout")`), IsNil) defer func() { - c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/mockstore/mocktikv/rpcCommitResult"), IsNil) + c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/mockstore/mocktikv/rpcCommitResult"), IsNil) }() // The rpc error will be wrapped to ErrResultUndetermined. t1 := s.begin(c) @@ -46,9 +46,9 @@ func (s *testCommitterSuite) TestFailCommitPrimaryRpcErrors(c *C) { // TestFailCommitPrimaryRegionError tests RegionError is handled properly when // committing primary region task. func (s *testCommitterSuite) TestFailCommitPrimaryRegionError(c *C) { - c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/mockstore/mocktikv/rpcCommitResult", `return("notLeader")`), IsNil) + c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/mockstore/mocktikv/rpcCommitResult", `return("notLeader")`), IsNil) defer func() { - c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/mockstore/mocktikv/rpcCommitResult"), IsNil) + c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/mockstore/mocktikv/rpcCommitResult"), IsNil) }() // Ensure it returns the original error without wrapped to ErrResultUndetermined // if it exceeds max retry timeout on RegionError. @@ -63,9 +63,9 @@ func (s *testCommitterSuite) TestFailCommitPrimaryRegionError(c *C) { // TestFailCommitPrimaryRPCErrorThenRegionError tests the case when commit first // receive a rpc timeout, then region errors afterwrards. func (s *testCommitterSuite) TestFailCommitPrimaryRPCErrorThenRegionError(c *C) { - c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/mockstore/mocktikv/rpcCommitResult", `1*return("timeout")->return("notLeader")`), IsNil) + c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/mockstore/mocktikv/rpcCommitResult", `1*return("timeout")->return("notLeader")`), IsNil) defer func() { - c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/mockstore/mocktikv/rpcCommitResult"), IsNil) + c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/mockstore/mocktikv/rpcCommitResult"), IsNil) }() // The region error will be wrapped to ErrResultUndetermined. t1 := s.begin(c) @@ -79,9 +79,9 @@ func (s *testCommitterSuite) TestFailCommitPrimaryRPCErrorThenRegionError(c *C) // TestFailCommitPrimaryKeyError tests KeyError is handled properly when // committing primary region task. func (s *testCommitterSuite) TestFailCommitPrimaryKeyError(c *C) { - c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/mockstore/mocktikv/rpcCommitResult", `return("keyError")`), IsNil) + c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/mockstore/mocktikv/rpcCommitResult", `return("keyError")`), IsNil) defer func() { - c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/mockstore/mocktikv/rpcCommitResult"), IsNil) + c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/mockstore/mocktikv/rpcCommitResult"), IsNil) }() // Ensure it returns the original error without wrapped to ErrResultUndetermined // if it meets KeyError. @@ -94,9 +94,9 @@ func (s *testCommitterSuite) TestFailCommitPrimaryKeyError(c *C) { } func (s *testCommitterSuite) TestFailCommitTimeout(c *C) { - c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/mockstore/mocktikv/rpcCommitTimeout", `return(true)`), IsNil) + c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/mockstore/mocktikv/rpcCommitTimeout", `return(true)`), IsNil) defer func() { - c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/mockstore/mocktikv/rpcCommitTimeout"), IsNil) + c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/mockstore/mocktikv/rpcCommitTimeout"), IsNil) }() txn := s.begin(c) err := txn.Set([]byte("a"), []byte("a1")) diff --git a/store/tikv/tests/2pc_test.go b/store/tikv/tests/2pc_test.go index 95a8e9ac077ea..89857f0f6c8a1 100644 --- a/store/tikv/tests/2pc_test.go +++ b/store/tikv/tests/2pc_test.go @@ -31,11 +31,11 @@ import ( pb "github.com/pingcap/kvproto/pkg/kvrpcpb" tidbkv "github.com/pingcap/tidb/kv" drivertxn "github.com/pingcap/tidb/store/driver/txn" - "github.com/pingcap/tidb/store/mockstore/mocktikv" "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/store/tikv/config" "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/mockstore/cluster" + "github.com/pingcap/tidb/store/tikv/mockstore/mocktikv" "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/store/tikv/tikvrpc" "github.com/pingcap/tidb/tablecodec" @@ -66,7 +66,7 @@ func (s *testCommitterSuite) SetUpTest(c *C) { cluster := mocktikv.NewCluster(mvccStore) mocktikv.BootstrapWithMultiRegions(cluster, []byte("a"), []byte("b"), []byte("c")) s.cluster = cluster - client := mocktikv.NewRPCClient(cluster, mvccStore) + client := mocktikv.NewRPCClient(cluster, mvccStore, nil) pdCli := &tikv.CodecPDClient{Client: mocktikv.NewPDClient(cluster)} spkv := tikv.NewMockSafePointKV() store, err := tikv.NewKVStore("mocktikv-store", pdCli, spkv, client) diff --git a/store/tikv/tests/delete_range_test.go b/store/tikv/tests/delete_range_test.go index 22b2389e982c9..86c02f0987234 100644 --- a/store/tikv/tests/delete_range_test.go +++ b/store/tikv/tests/delete_range_test.go @@ -20,9 +20,10 @@ import ( "sort" . "github.com/pingcap/check" - "github.com/pingcap/tidb/store/mockstore/mocktikv" + "github.com/pingcap/tidb/store/mockstore/mockcopr" "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/store/tikv/mockstore/cluster" + "github.com/pingcap/tidb/store/tikv/mockstore/mocktikv" ) type testDeleteRangeSuite struct { @@ -34,7 +35,7 @@ type testDeleteRangeSuite struct { var _ = Suite(&testDeleteRangeSuite{}) func (s *testDeleteRangeSuite) SetUpTest(c *C) { - client, cluster, pdClient, err := mocktikv.NewTiKVAndPDClient("") + client, cluster, pdClient, err := mocktikv.NewTiKVAndPDClient("", mockcopr.NewCoprRPCHandler()) c.Assert(err, IsNil) mocktikv.BootstrapWithMultiRegions(cluster, []byte("b"), []byte("c"), []byte("d")) s.cluster = cluster diff --git a/store/tikv/tests/range_task_test.go b/store/tikv/tests/range_task_test.go index a62d0e895186e..fa4c83297e20e 100644 --- a/store/tikv/tests/range_task_test.go +++ b/store/tikv/tests/range_task_test.go @@ -20,10 +20,11 @@ import ( "sort" . "github.com/pingcap/check" - "github.com/pingcap/tidb/store/mockstore/mocktikv" + "github.com/pingcap/tidb/store/mockstore/mockcopr" "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/mockstore/cluster" + "github.com/pingcap/tidb/store/tikv/mockstore/mocktikv" ) type testRangeTaskSuite struct { @@ -61,7 +62,7 @@ func (s *testRangeTaskSuite) SetUpTest(c *C) { } allRegionRanges = append(allRegionRanges, makeRange("z", "")) - client, cluster, pdClient, err := mocktikv.NewTiKVAndPDClient("") + client, cluster, pdClient, err := mocktikv.NewTiKVAndPDClient("", mockcopr.NewCoprRPCHandler()) c.Assert(err, IsNil) mocktikv.BootstrapWithMultiRegions(cluster, splitKeys...) s.cluster = cluster 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 31fb141e6b505..43ebd9a21be77 100644 --- a/store/tikv/tests/split_test.go +++ b/store/tikv/tests/split_test.go @@ -21,9 +21,10 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/pingcap/tidb/store/mockstore/mocktikv" + "github.com/pingcap/tidb/store/mockstore/mockcopr" "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/store/tikv/mockstore/cluster" + "github.com/pingcap/tidb/store/tikv/mockstore/mocktikv" pd "github.com/tikv/pd/client" ) @@ -37,7 +38,7 @@ type testSplitSuite struct { var _ = Suite(&testSplitSuite{}) func (s *testSplitSuite) SetUpTest(c *C) { - client, cluster, pdClient, err := mocktikv.NewTiKVAndPDClient("") + client, cluster, pdClient, err := mocktikv.NewTiKVAndPDClient("", mockcopr.NewCoprRPCHandler()) c.Assert(err, IsNil) mocktikv.BootstrapWithSingleStore(cluster) s.cluster = cluster diff --git a/store/tikv/tests/store_fail_test.go b/store/tikv/tests/store_fail_test.go index edba81d04e85f..8d7a7ff7cbc2c 100644 --- a/store/tikv/tests/store_fail_test.go +++ b/store/tikv/tests/store_fail_test.go @@ -33,11 +33,11 @@ func (s *testStoreSerialSuite) TestFailBusyServerKV(c *C) { var wg sync.WaitGroup wg.Add(2) - c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/mockstore/mocktikv/rpcServerBusy", `return(true)`), IsNil) + c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/mockstore/mocktikv/rpcServerBusy", `return(true)`), IsNil) go func() { defer wg.Done() time.Sleep(time.Millisecond * 100) - c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/mockstore/mocktikv/rpcServerBusy"), IsNil) + c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/mockstore/mocktikv/rpcServerBusy"), IsNil) }() go func() { diff --git a/store/tikv/txn.go b/store/tikv/txn.go index fb3e4d17b2ef9..9f3698abf1f6b 100644 --- a/store/tikv/txn.go +++ b/store/tikv/txn.go @@ -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/codec/number.go b/store/tikv/util/codec/number.go new file mode 100644 index 0000000000000..47912e813b2fc --- /dev/null +++ b/store/tikv/util/codec/number.go @@ -0,0 +1,284 @@ +// Copyright 2015 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 codec + +import ( + "encoding/binary" + "math" + + "github.com/pingcap/errors" +) + +const signMask uint64 = 0x8000000000000000 + +// EncodeIntToCmpUint make int v to comparable uint type +func EncodeIntToCmpUint(v int64) uint64 { + return uint64(v) ^ signMask +} + +// DecodeCmpUintToInt decodes the u that encoded by EncodeIntToCmpUint +func DecodeCmpUintToInt(u uint64) int64 { + return int64(u ^ signMask) +} + +// EncodeInt appends the encoded value to slice b and returns the appended slice. +// EncodeInt guarantees that the encoded value is in ascending order for comparison. +func EncodeInt(b []byte, v int64) []byte { + var data [8]byte + u := EncodeIntToCmpUint(v) + binary.BigEndian.PutUint64(data[:], u) + return append(b, data[:]...) +} + +// EncodeIntDesc appends the encoded value to slice b and returns the appended slice. +// EncodeIntDesc guarantees that the encoded value is in descending order for comparison. +func EncodeIntDesc(b []byte, v int64) []byte { + var data [8]byte + u := EncodeIntToCmpUint(v) + binary.BigEndian.PutUint64(data[:], ^u) + return append(b, data[:]...) +} + +// DecodeInt decodes value encoded by EncodeInt before. +// It returns the leftover un-decoded slice, decoded value if no error. +func DecodeInt(b []byte) ([]byte, int64, error) { + if len(b) < 8 { + return nil, 0, errors.New("insufficient bytes to decode value") + } + + u := binary.BigEndian.Uint64(b[:8]) + v := DecodeCmpUintToInt(u) + b = b[8:] + return b, v, nil +} + +// DecodeIntDesc decodes value encoded by EncodeInt before. +// It returns the leftover un-decoded slice, decoded value if no error. +func DecodeIntDesc(b []byte) ([]byte, int64, error) { + if len(b) < 8 { + return nil, 0, errors.New("insufficient bytes to decode value") + } + + u := binary.BigEndian.Uint64(b[:8]) + v := DecodeCmpUintToInt(^u) + b = b[8:] + return b, v, nil +} + +// EncodeUint appends the encoded value to slice b and returns the appended slice. +// EncodeUint guarantees that the encoded value is in ascending order for comparison. +func EncodeUint(b []byte, v uint64) []byte { + var data [8]byte + binary.BigEndian.PutUint64(data[:], v) + return append(b, data[:]...) +} + +// EncodeUintDesc appends the encoded value to slice b and returns the appended slice. +// EncodeUintDesc guarantees that the encoded value is in descending order for comparison. +func EncodeUintDesc(b []byte, v uint64) []byte { + var data [8]byte + binary.BigEndian.PutUint64(data[:], ^v) + return append(b, data[:]...) +} + +// DecodeUint decodes value encoded by EncodeUint before. +// It returns the leftover un-decoded slice, decoded value if no error. +func DecodeUint(b []byte) ([]byte, uint64, error) { + if len(b) < 8 { + return nil, 0, errors.New("insufficient bytes to decode value") + } + + v := binary.BigEndian.Uint64(b[:8]) + b = b[8:] + return b, v, nil +} + +// DecodeUintDesc decodes value encoded by EncodeInt before. +// It returns the leftover un-decoded slice, decoded value if no error. +func DecodeUintDesc(b []byte) ([]byte, uint64, error) { + if len(b) < 8 { + return nil, 0, errors.New("insufficient bytes to decode value") + } + + data := b[:8] + v := binary.BigEndian.Uint64(data) + b = b[8:] + return b, ^v, nil +} + +// EncodeVarint appends the encoded value to slice b and returns the appended slice. +// Note that the encoded result is not memcomparable. +func EncodeVarint(b []byte, v int64) []byte { + var data [binary.MaxVarintLen64]byte + n := binary.PutVarint(data[:], v) + return append(b, data[:n]...) +} + +// DecodeVarint decodes value encoded by EncodeVarint before. +// It returns the leftover un-decoded slice, decoded value if no error. +func DecodeVarint(b []byte) ([]byte, int64, error) { + v, n := binary.Varint(b) + if n > 0 { + return b[n:], v, nil + } + if n < 0 { + return nil, 0, errors.New("value larger than 64 bits") + } + return nil, 0, errors.New("insufficient bytes to decode value") +} + +// EncodeUvarint appends the encoded value to slice b and returns the appended slice. +// Note that the encoded result is not memcomparable. +func EncodeUvarint(b []byte, v uint64) []byte { + var data [binary.MaxVarintLen64]byte + n := binary.PutUvarint(data[:], v) + return append(b, data[:n]...) +} + +// DecodeUvarint decodes value encoded by EncodeUvarint before. +// It returns the leftover un-decoded slice, decoded value if no error. +func DecodeUvarint(b []byte) ([]byte, uint64, error) { + v, n := binary.Uvarint(b) + if n > 0 { + return b[n:], v, nil + } + if n < 0 { + return nil, 0, errors.New("value larger than 64 bits") + } + return nil, 0, errors.New("insufficient bytes to decode value") +} + +const ( + negativeTagEnd = 8 // negative tag is (negativeTagEnd - length). + positiveTagStart = 0xff - 8 // Positive tag is (positiveTagStart + length). +) + +// EncodeComparableVarint encodes an int64 to a mem-comparable bytes. +func EncodeComparableVarint(b []byte, v int64) []byte { + if v < 0 { + // All negative value has a tag byte prefix (negativeTagEnd - length). + // Smaller negative value encodes to more bytes, has smaller tag. + if v >= -0xff { + return append(b, negativeTagEnd-1, byte(v)) + } else if v >= -0xffff { + return append(b, negativeTagEnd-2, byte(v>>8), byte(v)) + } else if v >= -0xffffff { + return append(b, negativeTagEnd-3, byte(v>>16), byte(v>>8), byte(v)) + } else if v >= -0xffffffff { + return append(b, negativeTagEnd-4, byte(v>>24), byte(v>>16), byte(v>>8), byte(v)) + } else if v >= -0xffffffffff { + return append(b, negativeTagEnd-5, byte(v>>32), byte(v>>24), byte(v>>16), byte(v>>8), byte(v)) + } else if v >= -0xffffffffffff { + return append(b, negativeTagEnd-6, byte(v>>40), byte(v>>32), byte(v>>24), byte(v>>16), byte(v>>8), + byte(v)) + } else if v >= -0xffffffffffffff { + return append(b, negativeTagEnd-7, byte(v>>48), byte(v>>40), byte(v>>32), byte(v>>24), byte(v>>16), + byte(v>>8), byte(v)) + } + return append(b, negativeTagEnd-8, byte(v>>56), byte(v>>48), byte(v>>40), byte(v>>32), byte(v>>24), + byte(v>>16), byte(v>>8), byte(v)) + } + return EncodeComparableUvarint(b, uint64(v)) +} + +// EncodeComparableUvarint encodes uint64 into mem-comparable bytes. +func EncodeComparableUvarint(b []byte, v uint64) []byte { + // The first byte has 256 values, [0, 7] is reserved for negative tags, + // [248, 255] is reserved for larger positive tags, + // So we can store value [0, 239] in a single byte. + // Values cannot be stored in single byte has a tag byte prefix (positiveTagStart+length). + // Larger value encodes to more bytes, has larger tag. + if v <= positiveTagStart-negativeTagEnd { + return append(b, byte(v)+negativeTagEnd) + } else if v <= 0xff { + return append(b, positiveTagStart+1, byte(v)) + } else if v <= 0xffff { + return append(b, positiveTagStart+2, byte(v>>8), byte(v)) + } else if v <= 0xffffff { + return append(b, positiveTagStart+3, byte(v>>16), byte(v>>8), byte(v)) + } else if v <= 0xffffffff { + return append(b, positiveTagStart+4, byte(v>>24), byte(v>>16), byte(v>>8), byte(v)) + } else if v <= 0xffffffffff { + return append(b, positiveTagStart+5, byte(v>>32), byte(v>>24), byte(v>>16), byte(v>>8), byte(v)) + } else if v <= 0xffffffffffff { + return append(b, positiveTagStart+6, byte(v>>40), byte(v>>32), byte(v>>24), byte(v>>16), byte(v>>8), + byte(v)) + } else if v <= 0xffffffffffffff { + return append(b, positiveTagStart+7, byte(v>>48), byte(v>>40), byte(v>>32), byte(v>>24), byte(v>>16), + byte(v>>8), byte(v)) + } + return append(b, positiveTagStart+8, byte(v>>56), byte(v>>48), byte(v>>40), byte(v>>32), byte(v>>24), + byte(v>>16), byte(v>>8), byte(v)) +} + +var ( + errDecodeInsufficient = errors.New("insufficient bytes to decode value") + errDecodeInvalid = errors.New("invalid bytes to decode value") +) + +// DecodeComparableUvarint decodes mem-comparable uvarint. +func DecodeComparableUvarint(b []byte) ([]byte, uint64, error) { + if len(b) == 0 { + return nil, 0, errDecodeInsufficient + } + first := b[0] + b = b[1:] + if first < negativeTagEnd { + return nil, 0, errors.Trace(errDecodeInvalid) + } + if first <= positiveTagStart { + return b, uint64(first) - negativeTagEnd, nil + } + length := int(first) - positiveTagStart + if len(b) < length { + return nil, 0, errors.Trace(errDecodeInsufficient) + } + var v uint64 + for _, c := range b[:length] { + v = (v << 8) | uint64(c) + } + return b[length:], v, nil +} + +// DecodeComparableVarint decodes mem-comparable varint. +func DecodeComparableVarint(b []byte) ([]byte, int64, error) { + if len(b) == 0 { + return nil, 0, errors.Trace(errDecodeInsufficient) + } + first := b[0] + if first >= negativeTagEnd && first <= positiveTagStart { + return b, int64(first) - negativeTagEnd, nil + } + b = b[1:] + var length int + var v uint64 + if first < negativeTagEnd { + length = negativeTagEnd - int(first) + v = math.MaxUint64 // negative value has all bits on by default. + } else { + length = int(first) - positiveTagStart + } + if len(b) < length { + return nil, 0, errors.Trace(errDecodeInsufficient) + } + for _, c := range b[:length] { + v = (v << 8) | uint64(c) + } + if first > positiveTagStart && v > math.MaxInt64 { + return nil, 0, errors.Trace(errDecodeInvalid) + } else if first < negativeTagEnd && v <= math.MaxInt64 { + return nil, 0, errors.Trace(errDecodeInvalid) + } + return b[length:], int64(v), nil +} diff --git a/store/tikv/util/execdetails.go b/store/tikv/util/execdetails.go index ee84a88fc6172..4647d8caa86db 100644 --- a/store/tikv/util/execdetails.go +++ b/store/tikv/util/execdetails.go @@ -14,10 +14,16 @@ package util import ( + "bytes" "fmt" "math" + "strconv" "sync" + "sync/atomic" "time" + + "github.com/pingcap/kvproto/pkg/kvrpcpb" + "github.com/pingcap/tidb/util/memory" ) type commitDetailCtxKeyType struct{} @@ -179,3 +185,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(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 + } +} + +// 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/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/tables.go b/table/tables/tables.go index d0c920ec92685..7a03d17061510 100644 --- a/table/tables/tables.go +++ b/table/tables/tables.go @@ -544,11 +544,11 @@ func NewCommonAddRecordCtx(size int) *CommonAddRecordCtx { // TryGetCommonPkColumnIds get the IDs of primary key column if the table has common handle. func TryGetCommonPkColumnIds(tbl *model.TableInfo) []int64 { - var pkColIds []int64 if !tbl.IsCommonHandle { return nil } pkIdx := FindPrimaryIndex(tbl) + pkColIds := make([]int64, 0, len(pkIdx.Columns)) for _, idxCol := range pkIdx.Columns { pkColIds = append(pkColIds, tbl.Columns[idxCol.Offset].ID) } @@ -572,12 +572,12 @@ func PrimaryPrefixColumnIDs(tbl *model.TableInfo) (prefixCols []int64) { // TryGetCommonPkColumns get the primary key columns if the table has common handle. func TryGetCommonPkColumns(tbl table.Table) []*table.Column { - var pkCols []*table.Column if !tbl.Meta().IsCommonHandle { return nil } pkIdx := FindPrimaryIndex(tbl.Meta()) cols := tbl.Cols() + pkCols := make([]*table.Column, 0, len(pkIdx.Columns)) for _, idxCol := range pkIdx.Columns { pkCols = append(pkCols, cols[idxCol.Offset]) } 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/tests/globalkilltest/global_kill_test.go b/tests/globalkilltest/global_kill_test.go index 495dcfe27eb14..f424a4b52d167 100644 --- a/tests/globalkilltest/global_kill_test.go +++ b/tests/globalkilltest/global_kill_test.go @@ -28,7 +28,7 @@ import ( _ "github.com/go-sql-driver/mysql" "github.com/juju/errors" . "github.com/pingcap/check" - "github.com/pingcap/log" + "github.com/pingcap/log" "github.com/pingcap/tidb/util/logutil" "go.etcd.io/etcd/clientv3" "go.uber.org/zap" 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/rowcodec/rowcodec_test.go b/util/rowcodec/rowcodec_test.go index 0b24f68a243c1..22d2b50fe2f8e 100644 --- a/util/rowcodec/rowcodec_test.go +++ b/util/rowcodec/rowcodec_test.go @@ -740,7 +740,7 @@ func (s *testSuite) TestCodecUtil(c *C) { c.Assert(rowcodec.IsNewFormat(oldRow), IsFalse) // test stringer for decoder. - var cols []rowcodec.ColInfo + var cols = make([]rowcodec.ColInfo, 0, len(tps)) for i, ft := range tps { cols = append(cols, rowcodec.ColInfo{ ID: colIDs[i], 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, },