Skip to content

Commit

Permalink
executor: migrate test-infra to testify for issue test of executor_te…
Browse files Browse the repository at this point in the history
…st (#32861)

ref #28577
  • Loading branch information
hawkingrei authored Mar 7, 2022
1 parent 6dbf2de commit 164bee7
Show file tree
Hide file tree
Showing 2 changed files with 193 additions and 180 deletions.
193 changes: 193 additions & 0 deletions executor/executor_issue_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ import (
"github.com/pingcap/errors"
"github.com/pingcap/failpoint"
"github.com/pingcap/tidb/config"
"github.com/pingcap/tidb/parser/auth"
"github.com/pingcap/tidb/parser/mysql"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/testkit"
Expand Down Expand Up @@ -382,3 +383,195 @@ func TestIssue29498(t *testing.T) {
require.Equal(t, mysql.MaxDatetimeWidthNoFsp+3+1, len(row))
require.Equal(t, "00:00:00.567", row[len(row)-12:])
}

func TestIssue30971(t *testing.T) {
store, clean := testkit.CreateMockStore(t)
defer clean()
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
tk.MustExec("drop table if exists t1, t2")
tk.MustExec("create table t1 (id int);")
tk.MustExec("create table t2 (id int, c int);")

testCases := []struct {
sql string
fields int
}{
// Fix a bug that the column length field returned to client is incorrect using MySQL prepare protocol.
{"select * from t1 union select 1 from t1", 1},
{"select c from t2 union select * from t1", 1},
{"select * from t1", 1},
{"select * from t2 where c in (select * from t1)", 2},
{"insert into t1 values (?)", 0},
{"update t1 set id = ?", 0},
}
for _, test := range testCases {
_, _, fields, err := tk.Session().PrepareStmt(test.sql)
require.NoError(t, err)
require.Len(t, fields, test.fields)
}
}

func TestIndexJoin31494(t *testing.T) {
store, dom, clean := testkit.CreateMockStoreAndDomain(t)
defer clean()
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")

tk.MustExec("drop table if exists t1, t2;")
tk.MustExec("create table t1(a int(11) default null, b int(11) default null, key(b));")
insertStr := "insert into t1 values(1, 1)"
for i := 1; i < 32768; i++ {
insertStr += fmt.Sprintf(", (%d, %d)", i, i)
}
tk.MustExec(insertStr)
tk.MustExec("create table t2(a int(11) default null, b int(11) default null, c int(11) default null)")
insertStr = "insert into t2 values(1, 1, 1)"
for i := 1; i < 32768; i++ {
insertStr += fmt.Sprintf(", (%d, %d, %d)", i, i, i)
}
tk.MustExec(insertStr)
sm := &mockSessionManager1{
PS: make([]*util.ProcessInfo, 0),
}
tk.Session().SetSessionManager(sm)
dom.ExpensiveQueryHandle().SetSessionManager(sm)
defer config.RestoreFunc()()
config.UpdateGlobal(func(conf *config.Config) {
conf.OOMAction = config.OOMActionCancel
})
require.True(t, tk.Session().Auth(&auth.UserIdentity{Username: "root", Hostname: "%"}, nil, nil))
tk.MustExec("set @@tidb_mem_quota_query=2097152;")
// This bug will be reproduced in 10 times.
for i := 0; i < 10; i++ {
err := tk.QueryToErr("select /*+ inl_join(t1) */ * from t1 right join t2 on t1.b=t2.b;")
require.Error(t, err)
require.Regexp(t, "Out Of Memory Quota!.*", err.Error())
err = tk.QueryToErr("select /*+ inl_hash_join(t1) */ * from t1 right join t2 on t1.b=t2.b;")
require.Error(t, err)
require.Regexp(t, "Out Of Memory Quota!.*", err.Error())
}
}

// Details at https://github.com/pingcap/tidb/issues/31038
func TestFix31038(t *testing.T) {
defer config.RestoreFunc()()
config.UpdateGlobal(func(conf *config.Config) {
conf.EnableCollectExecutionInfo = false
})
store, clean := testkit.CreateMockStore(t)
defer clean()
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
tk.MustExec("drop table if exists t123")
tk.MustExec("create table t123 (id int);")
require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/store/copr/disable-collect-execution", `return(true)`))
tk.MustQuery("select * from t123;")
require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/store/copr/disable-collect-execution"))
}

func TestFix31530(t *testing.T) {
store, clean := testkit.CreateMockStore(t)
defer clean()
tk := testkit.NewTestKit(t, store)
tk2 := testkit.NewTestKit(t, store)
tk.MustExec("use test")
tk2.MustExec("use test")
defer func() {
tk.MustExec("drop table if exists t1")
}()
tk.MustExec("drop table if exists t1")
tk.MustExec("create table t1 (id int primary key, v int)")
tk.MustExec("insert into t1 values(1, 10)")
tk.MustExec("begin pessimistic")
tk.MustQuery("select * from t1").Check(testkit.Rows("1 10"))

// update t1 before session1 transaction not finished
tk2.MustExec("update t1 set v=11 where id=1")

tk.MustQuery("(select 'a' as c, id, v from t1 for update) union all (select 'b', id, v from t1) order by c").Check(testkit.Rows("a 1 11", "b 1 10"))
tk.MustQuery("(select 'a' as c, id, v from t1) union all (select 'b', id, v from t1 for update) order by c").Check(testkit.Rows("a 1 10", "b 1 11"))
tk.MustQuery("(select 'a' as c, id, v from t1 where id=1 for update) union all (select 'b', id, v from t1 where id=1) order by c").Check(testkit.Rows("a 1 11", "b 1 10"))
tk.MustQuery("(select 'a' as c, id, v from t1 where id=1) union all (select 'b', id, v from t1 where id=1 for update) order by c").Check(testkit.Rows("a 1 10", "b 1 11"))
tk.MustExec("rollback")
}

func TestFix31537(t *testing.T) {
store, clean := testkit.CreateMockStore(t)
defer clean()
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
tk.MustExec(`CREATE TABLE trade (
t_id bigint(16) NOT NULL AUTO_INCREMENT,
t_dts datetime NOT NULL,
t_st_id char(4) NOT NULL,
t_tt_id char(3) NOT NULL,
t_is_cash tinyint(1) NOT NULL,
t_s_symb char(15) NOT NULL,
t_qty mediumint(7) NOT NULL,
t_bid_price decimal(8,2) NOT NULL,
t_ca_id bigint(12) NOT NULL,
t_exec_name varchar(49) NOT NULL,
t_trade_price decimal(8,2) DEFAULT NULL,
t_chrg decimal(10,2) NOT NULL,
t_comm decimal(10,2) NOT NULL,
t_tax decimal(10,2) NOT NULL,
t_lifo tinyint(1) NOT NULL,
PRIMARY KEY (t_id) /*T![clustered_index] CLUSTERED */,
KEY i_t_ca_id_dts (t_ca_id,t_dts),
KEY i_t_s_symb_dts (t_s_symb,t_dts),
CONSTRAINT fk_trade_st FOREIGN KEY (t_st_id) REFERENCES status_type (st_id),
CONSTRAINT fk_trade_tt FOREIGN KEY (t_tt_id) REFERENCES trade_type (tt_id),
CONSTRAINT fk_trade_s FOREIGN KEY (t_s_symb) REFERENCES security (s_symb),
CONSTRAINT fk_trade_ca FOREIGN KEY (t_ca_id) REFERENCES customer_account (ca_id)
) ;`)
tk.MustExec(`CREATE TABLE trade_history (
th_t_id bigint(16) NOT NULL,
th_dts datetime NOT NULL,
th_st_id char(4) NOT NULL,
PRIMARY KEY (th_t_id,th_st_id) /*T![clustered_index] NONCLUSTERED */,
KEY i_th_t_id_dts (th_t_id,th_dts),
CONSTRAINT fk_trade_history_t FOREIGN KEY (th_t_id) REFERENCES trade (t_id),
CONSTRAINT fk_trade_history_st FOREIGN KEY (th_st_id) REFERENCES status_type (st_id)
);
`)
tk.MustExec(`CREATE TABLE status_type (
st_id char(4) NOT NULL,
st_name char(10) NOT NULL,
PRIMARY KEY (st_id) /*T![clustered_index] NONCLUSTERED */
);`)
tk.MustQuery(`trace plan SELECT T_ID, T_S_SYMB, T_QTY, ST_NAME, TH_DTS FROM ( SELECT T_ID AS ID FROM TRADE WHERE T_CA_ID = 43000014236 ORDER BY T_DTS DESC LIMIT 10 ) T, TRADE, TRADE_HISTORY, STATUS_TYPE WHERE TRADE.T_ID = ID AND TRADE_HISTORY.TH_T_ID = TRADE.T_ID AND STATUS_TYPE.ST_ID = TRADE_HISTORY.TH_ST_ID ORDER BY TH_DTS DESC LIMIT 30;`)
}

func TestIssue30382(t *testing.T) {
store, clean := testkit.CreateMockStore(t)
defer clean()
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
tk.MustExec("set @@session.tidb_enable_list_partition = ON;")
tk.MustExec("drop table if exists t1, t2;")
tk.MustExec("create table t1 (c_int int, c_str varchar(40), c_decimal decimal(12, 6), primary key (c_int) , key(c_str(2)) , key(c_decimal) ) partition by list (c_int) ( partition p0 values IN (1, 5, 9, 13, 17, 21, 25, 29, 33, 37), partition p1 values IN (2, 6, 10, 14, 18, 22, 26, 30, 34, 38), partition p2 values IN (3, 7, 11, 15, 19, 23, 27, 31, 35, 39), partition p3 values IN (4, 8, 12, 16, 20, 24, 28, 32, 36, 40)) ;")
tk.MustExec("create table t2 (c_int int, c_str varchar(40), c_decimal decimal(12, 6), primary key (c_int) , key(c_str) , key(c_decimal) ) partition by hash (c_int) partitions 4;")
tk.MustExec("insert into t1 values (6, 'musing mayer', 1.280), (7, 'wizardly heisenberg', 6.589), (8, 'optimistic swirles', 9.633), (9, 'hungry haslett', 2.659), (10, 'stupefied wiles', 2.336);")
tk.MustExec("insert into t2 select * from t1 ;")
tk.MustExec("begin;")
tk.MustQuery("select * from t1 where c_str <> any (select c_str from t2 where c_decimal < 5) for update;").Sort().Check(testkit.Rows(
"10 stupefied wiles 2.336000",
"6 musing mayer 1.280000",
"7 wizardly heisenberg 6.589000",
"8 optimistic swirles 9.633000",
"9 hungry haslett 2.659000"))
tk.MustQuery("explain format = 'brief' select * from t1 where c_str <> any (select c_str from t2 where c_decimal < 5) for update;").Check(testkit.Rows(
"SelectLock 6400.00 root for update 0",
"└─HashJoin 6400.00 root CARTESIAN inner join, other cond:or(gt(Column#8, 1), or(ne(test.t1.c_str, Column#7), if(ne(Column#9, 0), NULL, 0)))",
" ├─Selection(Build) 0.80 root ne(Column#10, 0)",
" │ └─StreamAgg 1.00 root funcs:max(Column#17)->Column#7, funcs:count(distinct Column#18)->Column#8, funcs:sum(Column#19)->Column#9, funcs:count(1)->Column#10",
" │ └─Projection 3323.33 root test.t2.c_str, test.t2.c_str, cast(isnull(test.t2.c_str), decimal(20,0) BINARY)->Column#19",
" │ └─TableReader 3323.33 root partition:all data:Selection",
" │ └─Selection 3323.33 cop[tikv] lt(test.t2.c_decimal, 5)",
" │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo",
" └─TableReader(Probe) 8000.00 root partition:all data:Selection",
" └─Selection 8000.00 cop[tikv] if(isnull(test.t1.c_str), NULL, 1)",
" └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo"))
tk.MustExec("commit")
}
180 changes: 0 additions & 180 deletions executor/executor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -9097,186 +9097,6 @@ func (s *testSerialSuite) TestUnreasonablyClose(c *C) {
c.Assert(opsAlreadyCoveredMask, Equals, opsNeedsCoveredMask, Commentf("these operators are not covered %s", commentBuf.String()))
}

func (s *testSerialSuite) TestIssue30971(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
tk.MustExec("drop table if exists t1, t2")
tk.MustExec("create table t1 (id int);")
tk.MustExec("create table t2 (id int, c int);")

testCases := []struct {
sql string
fields int
}{
// Fix a bug that the column length field returned to client is incorrect using MySQL prepare protocol.
{"select * from t1 union select 1 from t1", 1},
{"select c from t2 union select * from t1", 1},
{"select * from t1", 1},
{"select * from t2 where c in (select * from t1)", 2},
{"insert into t1 values (?)", 0},
{"update t1 set id = ?", 0},
}
for _, test := range testCases {
_, _, fields, err := tk.Se.PrepareStmt(test.sql)
c.Assert(err, IsNil)
c.Assert(fields, HasLen, test.fields)
}
}

func (s *testSerialSuite) TestIndexJoin31494(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")

tk.MustExec("drop table if exists t1, t2;")
tk.MustExec("create table t1(a int(11) default null, b int(11) default null, key(b));")
insertStr := "insert into t1 values(1, 1)"
for i := 1; i < 32768; i++ {
insertStr += fmt.Sprintf(", (%d, %d)", i, i)
}
tk.MustExec(insertStr)
tk.MustExec("create table t2(a int(11) default null, b int(11) default null, c int(11) default null)")
insertStr = "insert into t2 values(1, 1, 1)"
for i := 1; i < 32768; i++ {
insertStr += fmt.Sprintf(", (%d, %d, %d)", i, i, i)
}
tk.MustExec(insertStr)
sm := &mockSessionManager1{
PS: make([]*util.ProcessInfo, 0),
}
tk.Se.SetSessionManager(sm)
s.domain.ExpensiveQueryHandle().SetSessionManager(sm)
defer config.RestoreFunc()()
config.UpdateGlobal(func(conf *config.Config) {
conf.OOMAction = config.OOMActionCancel
})
c.Assert(tk.Se.Auth(&auth.UserIdentity{Username: "root", Hostname: "%"}, nil, nil), IsTrue)
tk.MustExec("set @@tidb_mem_quota_query=2097152;")
// This bug will be reproduced in 10 times.
for i := 0; i < 10; i++ {
err := tk.QueryToErr("select /*+ inl_join(t1) */ * from t1 right join t2 on t1.b=t2.b;")
c.Assert(err, NotNil)
c.Assert(err.Error(), Matches, "Out Of Memory Quota!.*")
err = tk.QueryToErr("select /*+ inl_hash_join(t1) */ * from t1 right join t2 on t1.b=t2.b;")
c.Assert(err, NotNil)
c.Assert(err.Error(), Matches, "Out Of Memory Quota!.*")
}
}

// Details at https://github.com/pingcap/tidb/issues/31038
func (s *testSerialSuite) TestFix31038(c *C) {
defer config.RestoreFunc()()
config.UpdateGlobal(func(conf *config.Config) {
conf.EnableCollectExecutionInfo = false
})
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
tk.MustExec("drop table if exists t123")
tk.MustExec("create table t123 (id int);")
failpoint.Enable("github.com/pingcap/tidb/store/copr/disable-collect-execution", `return(true)`)
tk.MustQuery("select * from t123;")
failpoint.Disable("github.com/pingcap/tidb/store/copr/disable-collect-execution")
}

func (s *testSerialSuite) TestFix31530(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk2 := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
tk2.MustExec("use test")
defer func() {
tk.MustExec("drop table if exists t1")
}()
tk.MustExec("drop table if exists t1")
tk.MustExec("create table t1 (id int primary key, v int)")
tk.MustExec("insert into t1 values(1, 10)")
tk.MustExec("begin pessimistic")
tk.MustQuery("select * from t1").Check(testkit.Rows("1 10"))

// update t1 before session1 transaction not finished
tk2.MustExec("update t1 set v=11 where id=1")

tk.MustQuery("(select 'a' as c, id, v from t1 for update) union all (select 'b', id, v from t1) order by c").Check(testkit.Rows("a 1 11", "b 1 10"))
tk.MustQuery("(select 'a' as c, id, v from t1) union all (select 'b', id, v from t1 for update) order by c").Check(testkit.Rows("a 1 10", "b 1 11"))
tk.MustQuery("(select 'a' as c, id, v from t1 where id=1 for update) union all (select 'b', id, v from t1 where id=1) order by c").Check(testkit.Rows("a 1 11", "b 1 10"))
tk.MustQuery("(select 'a' as c, id, v from t1 where id=1) union all (select 'b', id, v from t1 where id=1 for update) order by c").Check(testkit.Rows("a 1 10", "b 1 11"))
tk.MustExec("rollback")
}

func (s *testSerialSuite) TestFix31537(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
tk.MustExec(`CREATE TABLE trade (
t_id bigint(16) NOT NULL AUTO_INCREMENT,
t_dts datetime NOT NULL,
t_st_id char(4) NOT NULL,
t_tt_id char(3) NOT NULL,
t_is_cash tinyint(1) NOT NULL,
t_s_symb char(15) NOT NULL,
t_qty mediumint(7) NOT NULL,
t_bid_price decimal(8,2) NOT NULL,
t_ca_id bigint(12) NOT NULL,
t_exec_name varchar(49) NOT NULL,
t_trade_price decimal(8,2) DEFAULT NULL,
t_chrg decimal(10,2) NOT NULL,
t_comm decimal(10,2) NOT NULL,
t_tax decimal(10,2) NOT NULL,
t_lifo tinyint(1) NOT NULL,
PRIMARY KEY (t_id) /*T![clustered_index] CLUSTERED */,
KEY i_t_ca_id_dts (t_ca_id,t_dts),
KEY i_t_s_symb_dts (t_s_symb,t_dts),
CONSTRAINT fk_trade_st FOREIGN KEY (t_st_id) REFERENCES status_type (st_id),
CONSTRAINT fk_trade_tt FOREIGN KEY (t_tt_id) REFERENCES trade_type (tt_id),
CONSTRAINT fk_trade_s FOREIGN KEY (t_s_symb) REFERENCES security (s_symb),
CONSTRAINT fk_trade_ca FOREIGN KEY (t_ca_id) REFERENCES customer_account (ca_id)
) ;`)
tk.MustExec(`CREATE TABLE trade_history (
th_t_id bigint(16) NOT NULL,
th_dts datetime NOT NULL,
th_st_id char(4) NOT NULL,
PRIMARY KEY (th_t_id,th_st_id) /*T![clustered_index] NONCLUSTERED */,
KEY i_th_t_id_dts (th_t_id,th_dts),
CONSTRAINT fk_trade_history_t FOREIGN KEY (th_t_id) REFERENCES trade (t_id),
CONSTRAINT fk_trade_history_st FOREIGN KEY (th_st_id) REFERENCES status_type (st_id)
);
`)
tk.MustExec(`CREATE TABLE status_type (
st_id char(4) NOT NULL,
st_name char(10) NOT NULL,
PRIMARY KEY (st_id) /*T![clustered_index] NONCLUSTERED */
);`)
tk.MustQuery(`trace plan SELECT T_ID, T_S_SYMB, T_QTY, ST_NAME, TH_DTS FROM ( SELECT T_ID AS ID FROM TRADE WHERE T_CA_ID = 43000014236 ORDER BY T_DTS DESC LIMIT 10 ) T, TRADE, TRADE_HISTORY, STATUS_TYPE WHERE TRADE.T_ID = ID AND TRADE_HISTORY.TH_T_ID = TRADE.T_ID AND STATUS_TYPE.ST_ID = TRADE_HISTORY.TH_ST_ID ORDER BY TH_DTS DESC LIMIT 30;`)
}

func (s *testSuiteP1) TestIssue30382(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
tk.MustExec("set @@session.tidb_enable_list_partition = ON;")
tk.MustExec("drop table if exists t1, t2;")
tk.MustExec("create table t1 (c_int int, c_str varchar(40), c_decimal decimal(12, 6), primary key (c_int) , key(c_str(2)) , key(c_decimal) ) partition by list (c_int) ( partition p0 values IN (1, 5, 9, 13, 17, 21, 25, 29, 33, 37), partition p1 values IN (2, 6, 10, 14, 18, 22, 26, 30, 34, 38), partition p2 values IN (3, 7, 11, 15, 19, 23, 27, 31, 35, 39), partition p3 values IN (4, 8, 12, 16, 20, 24, 28, 32, 36, 40)) ;")
tk.MustExec("create table t2 (c_int int, c_str varchar(40), c_decimal decimal(12, 6), primary key (c_int) , key(c_str) , key(c_decimal) ) partition by hash (c_int) partitions 4;")
tk.MustExec("insert into t1 values (6, 'musing mayer', 1.280), (7, 'wizardly heisenberg', 6.589), (8, 'optimistic swirles', 9.633), (9, 'hungry haslett', 2.659), (10, 'stupefied wiles', 2.336);")
tk.MustExec("insert into t2 select * from t1 ;")
tk.MustExec("begin;")
tk.MustQuery("select * from t1 where c_str <> any (select c_str from t2 where c_decimal < 5) for update;").Sort().Check(testkit.Rows(
"10 stupefied wiles 2.336000",
"6 musing mayer 1.280000",
"7 wizardly heisenberg 6.589000",
"8 optimistic swirles 9.633000",
"9 hungry haslett 2.659000"))
tk.MustQuery("explain format = 'brief' select * from t1 where c_str <> any (select c_str from t2 where c_decimal < 5) for update;").Check(testkit.Rows(
"SelectLock 6400.00 root for update 0",
"└─HashJoin 6400.00 root CARTESIAN inner join, other cond:or(gt(Column#8, 1), or(ne(test.t1.c_str, Column#7), if(ne(Column#9, 0), NULL, 0)))",
" ├─Selection(Build) 0.80 root ne(Column#10, 0)",
" │ └─StreamAgg 1.00 root funcs:max(Column#17)->Column#7, funcs:count(distinct Column#18)->Column#8, funcs:sum(Column#19)->Column#9, funcs:count(1)->Column#10",
" │ └─Projection 3323.33 root test.t2.c_str, test.t2.c_str, cast(isnull(test.t2.c_str), decimal(20,0) BINARY)->Column#19",
" │ └─TableReader 3323.33 root partition:all data:Selection",
" │ └─Selection 3323.33 cop[tikv] lt(test.t2.c_decimal, 5)",
" │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo",
" └─TableReader(Probe) 8000.00 root partition:all data:Selection",
" └─Selection 8000.00 cop[tikv] if(isnull(test.t1.c_str), NULL, 1)",
" └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo"))
tk.MustExec("commit")
}

func (s *testSerialSuite) TestEncodingSet(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
Expand Down

0 comments on commit 164bee7

Please sign in to comment.