Skip to content

Commit

Permalink
Merge branch 'master' into remove_random_test
Browse files Browse the repository at this point in the history
  • Loading branch information
rebelice authored Apr 22, 2021
2 parents 66fe8cf + 734084f commit 38dffd4
Show file tree
Hide file tree
Showing 158 changed files with 3,424 additions and 1,246 deletions.
18 changes: 9 additions & 9 deletions cmd/explaintest/r/explain_easy.result
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -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
Expand Down
4 changes: 2 additions & 2 deletions cmd/explaintest/r/explain_indexmerge.result
Original file line number Diff line number Diff line change
Expand Up @@ -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
15 changes: 8 additions & 7 deletions cmd/explaintest/r/explain_join_stats.result
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
3 changes: 2 additions & 1 deletion config/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -208,6 +208,7 @@ func (c *Config) getTiKVConfig() *tikvcfg.Config {
OpenTracingEnable: c.OpenTracing.Enable,
Path: c.Path,
EnableForwarding: c.EnableForwarding,
TxnScope: c.Labels["zone"],
}
}

Expand Down Expand Up @@ -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"`
Expand Down
9 changes: 9 additions & 0 deletions config/config_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ import (
"os"
"os/user"
"path/filepath"
"reflect"
"runtime"
"testing"

Expand Down Expand Up @@ -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) {
Expand Down
19 changes: 2 additions & 17 deletions config/config_util.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,7 @@ import (

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

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

const (
globalTxnScope = "global"
)

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

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

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

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

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

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

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

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

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

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

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

labelConstraints, err := placement.CheckLabelConstraints(strings.Split(strings.TrimSpace(labels), ","))
labelConstraints, err := placement.NewConstraints(strings.Split(strings.TrimSpace(labels), ","))
if err != nil {
return rules, err
}
Expand Down Expand Up @@ -6062,11 +6063,13 @@ func (d *ddl) AlterTableAlterPartition(ctx sessionctx.Context, ident ast.Ident,
// refer to tidb#22065.
// add -engine=tiflash to every rule to avoid schedules to tiflash instances.
// placement rules in SQL is not compatible with `set tiflash replica` yet
rule.LabelConstraints = append(rule.LabelConstraints, placement.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
Expand All @@ -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},
Expand Down
7 changes: 5 additions & 2 deletions ddl/failtest/fail_db_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,7 @@ import (
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/store/mockstore"
"github.com/pingcap/tidb/store/tikv/mockstore/cluster"
"github.com/pingcap/tidb/tablecodec"
"github.com/pingcap/tidb/util/logutil"
"github.com/pingcap/tidb/util/testkit"
"github.com/pingcap/tidb/util/testleak"
Expand Down Expand Up @@ -237,7 +238,8 @@ func (s *testFailDBSuite) TestAddIndexFailed(c *C) {
tblID := tbl.Meta().ID

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

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

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

err = ddlutil.LoadDDLReorgVars(tk.Se)
c.Assert(err, IsNil)
Expand Down
Loading

0 comments on commit 38dffd4

Please sign in to comment.