Skip to content

Commit

Permalink
cherry pick pingcap#32308 to release-5.3
Browse files Browse the repository at this point in the history
Signed-off-by: ti-srebot <[email protected]>
  • Loading branch information
ekexium authored and ti-srebot committed Feb 15, 2022
1 parent 83b273a commit f4c093e
Show file tree
Hide file tree
Showing 2 changed files with 347 additions and 2 deletions.
345 changes: 345 additions & 0 deletions executor/executor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -9384,3 +9384,348 @@ func (s *testSerialSuite) TestIssue28650(c *C) {
}()
}
}
<<<<<<< HEAD
=======

func (s *testSerialSuite) TestIssue30289(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
fpName := "github.com/pingcap/tidb/executor/issue30289"
c.Assert(failpoint.Enable(fpName, `return(true)`), IsNil)
defer func() {
c.Assert(failpoint.Disable(fpName), IsNil)
}()
tk.MustExec("drop table if exists t")
tk.MustExec("create table t(a int)")
err := tk.QueryToErr("select /*+ hash_join(t1) */ * from t t1 join t t2 on t1.a=t2.a")
c.Assert(err.Error(), Matches, "issue30289 build return error")
}

func (s *testSerialSuite) TestIssue29498(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
tk.MustExec("DROP TABLE IF EXISTS t1;")
tk.MustExec("CREATE TABLE t1 (t3 TIME(3), d DATE, t TIME);")
tk.MustExec("INSERT INTO t1 VALUES ('00:00:00.567', '2002-01-01', '00:00:02');")

res := tk.MustQuery("SELECT CONCAT(IFNULL(t3, d)) AS col1 FROM t1;")
row := res.Rows()[0][0].(string)
c.Assert(len(row), Equals, mysql.MaxDatetimeWidthNoFsp+3+1)
c.Assert(row[len(row)-12:], Equals, "00:00:00.567")

res = tk.MustQuery("SELECT IFNULL(t3, d) AS col1 FROM t1;")
row = res.Rows()[0][0].(string)
c.Assert(len(row), Equals, mysql.MaxDatetimeWidthNoFsp+3+1)
c.Assert(row[len(row)-12:], Equals, "00:00:00.567")

res = tk.MustQuery("SELECT CONCAT(IFNULL(t, d)) AS col1 FROM t1;")
row = res.Rows()[0][0].(string)
c.Assert(len(row), Equals, mysql.MaxDatetimeWidthNoFsp)
c.Assert(row[len(row)-8:], Equals, "00:00:02")

res = tk.MustQuery("SELECT IFNULL(t, d) AS col1 FROM t1;")
row = res.Rows()[0][0].(string)
c.Assert(len(row), Equals, mysql.MaxDatetimeWidthNoFsp)
c.Assert(row[len(row)-8:], Equals, "00:00:02")

res = tk.MustQuery("SELECT CONCAT(xx) FROM (SELECT t3 AS xx FROM t1 UNION SELECT d FROM t1) x ORDER BY -xx LIMIT 1;")
row = res.Rows()[0][0].(string)
c.Assert(len(row), Equals, mysql.MaxDatetimeWidthNoFsp+3+1)
c.Assert(row[len(row)-12:], Equals, "00:00:00.567")

res = tk.MustQuery("SELECT CONCAT(CASE WHEN d IS NOT NULL THEN t3 ELSE d END) AS col1 FROM t1;")
row = res.Rows()[0][0].(string)
c.Assert(len(row), Equals, mysql.MaxDatetimeWidthNoFsp+3+1)
c.Assert(row[len(row)-12:], Equals, "00:00:00.567")
}

// Test invoke Close without invoking Open before for each operators.
func (s *testSerialSuite) TestUnreasonablyClose(c *C) {
defer testleak.AfterTest(c)()

is := infoschema.MockInfoSchema([]*model.TableInfo{plannercore.MockSignedTable(), plannercore.MockUnsignedTable()})
se, err := session.CreateSession4Test(s.store)
c.Assert(err, IsNil)
_, err = se.Execute(context.Background(), "use test")
c.Assert(err, IsNil)
// To enable the shuffleExec operator.
_, err = se.Execute(context.Background(), "set @@tidb_merge_join_concurrency=4")
c.Assert(err, IsNil)

var opsNeedsCovered = []plannercore.PhysicalPlan{
&plannercore.PhysicalHashJoin{},
&plannercore.PhysicalMergeJoin{},
&plannercore.PhysicalIndexJoin{},
&plannercore.PhysicalIndexHashJoin{},
&plannercore.PhysicalTableReader{},
&plannercore.PhysicalIndexReader{},
&plannercore.PhysicalIndexLookUpReader{},
&plannercore.PhysicalIndexMergeReader{},
&plannercore.PhysicalApply{},
&plannercore.PhysicalHashAgg{},
&plannercore.PhysicalStreamAgg{},
&plannercore.PhysicalLimit{},
&plannercore.PhysicalSort{},
&plannercore.PhysicalTopN{},
&plannercore.PhysicalCTE{},
&plannercore.PhysicalCTETable{},
&plannercore.PhysicalMaxOneRow{},
&plannercore.PhysicalProjection{},
&plannercore.PhysicalSelection{},
&plannercore.PhysicalTableDual{},
&plannercore.PhysicalWindow{},
&plannercore.PhysicalShuffle{},
&plannercore.PhysicalUnionAll{},
}
executorBuilder := executor.NewMockExecutorBuilderForTest(se, is, nil, math.MaxUint64, false, "global")

var opsNeedsCoveredMask uint64 = 1<<len(opsNeedsCovered) - 1
opsAlreadyCoveredMask := uint64(0)
for i, tc := range []string{
"select /*+ hash_join(t1)*/ * from t t1 join t t2 on t1.a = t2.a",
"select /*+ merge_join(t1)*/ * from t t1 join t t2 on t1.f = t2.f",
"select t.f from t use index(f)",
"select /*+ inl_join(t1) */ * from t t1 join t t2 on t1.f=t2.f",
"select /*+ inl_hash_join(t1) */ * from t t1 join t t2 on t1.f=t2.f",
"SELECT count(1) FROM (SELECT (SELECT min(a) FROM t as t2 WHERE t2.a > t1.a) AS a from t as t1) t",
"select /*+ hash_agg() */ count(f) from t group by a",
"select /*+ stream_agg() */ count(f) from t group by a",
"select * from t order by a, f",
"select * from t order by a, f limit 1",
"select * from t limit 1",
"select (select t1.a from t t1 where t1.a > t2.a) as a from t t2;",
"select a + 1 from t",
"select count(*) a from t having a > 1",
"select * from t where a = 1.1",
"with recursive cte1(c1) as (select 1 union select c1 + 1 from cte1 limit 5 offset 0) select * from cte1",
"select /*+use_index_merge(t, c_d_e, f)*/ * from t where c < 1 or f > 2",
"select sum(f) over (partition by f) from t",
"select /*+ merge_join(t1)*/ * from t t1 join t t2 on t1.d = t2.d",
"select a from t union all select a from t",
} {
comment := Commentf("case:%v sql:%s", i, tc)
c.Assert(err, IsNil, comment)
stmt, err := s.ParseOneStmt(tc, "", "")
c.Assert(err, IsNil, comment)

err = se.NewTxn(context.Background())
c.Assert(err, IsNil, comment)
p, _, err := planner.Optimize(context.TODO(), se, stmt, is)
c.Assert(err, IsNil, comment)
// This for loop level traverses the plan tree to get which operators are covered.
for child := []plannercore.PhysicalPlan{p.(plannercore.PhysicalPlan)}; len(child) != 0; {
newChild := make([]plannercore.PhysicalPlan, 0, len(child))
for _, ch := range child {
found := false
for k, t := range opsNeedsCovered {
if reflect.TypeOf(t) == reflect.TypeOf(ch) {
opsAlreadyCoveredMask |= 1 << k
found = true
break
}
}
c.Assert(found, IsTrue, Commentf("case: %v sql: %s operator %v is not registered in opsNeedsCoveredMask", i, tc, reflect.TypeOf(ch)))
switch x := ch.(type) {
case *plannercore.PhysicalCTE:
newChild = append(newChild, x.RecurPlan)
newChild = append(newChild, x.SeedPlan)
continue
case *plannercore.PhysicalShuffle:
newChild = append(newChild, x.DataSources...)
newChild = append(newChild, x.Tails...)
continue
}
newChild = append(newChild, ch.Children()...)
}
child = newChild
}

e := executorBuilder.Build(p)

func() {
defer func() {
r := recover()
buf := make([]byte, 4096)
stackSize := runtime.Stack(buf, false)
buf = buf[:stackSize]
c.Assert(r, IsNil, Commentf("case: %v\n sql: %s\n error stack: %v", i, tc, string(buf)))
}()
c.Assert(e.Close(), IsNil, comment)
}()
}
// The following code is used to make sure all the operators registered
// in opsNeedsCoveredMask are covered.
commentBuf := strings.Builder{}
if opsAlreadyCoveredMask != opsNeedsCoveredMask {
for i := range opsNeedsCovered {
if opsAlreadyCoveredMask&(1<<i) != 1<<i {
commentBuf.WriteString(fmt.Sprintf(" %v", reflect.TypeOf(opsNeedsCovered[i])))
}
}
}
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 *testSerialSuite) TestEncodingSet(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
tk.MustExec("CREATE TABLE `enum-set` (`set` SET(" +
"'x00','x01','x02','x03','x04','x05','x06','x07','x08','x09','x10','x11','x12','x13','x14','x15'," +
"'x16','x17','x18','x19','x20','x21','x22','x23','x24','x25','x26','x27','x28','x29','x30','x31'," +
"'x32','x33','x34','x35','x36','x37','x38','x39','x40','x41','x42','x43','x44','x45','x46','x47'," +
"'x48','x49','x50','x51','x52','x53','x54','x55','x56','x57','x58','x59','x60','x61','x62','x63'" +
")NOT NULL PRIMARY KEY)")
tk.MustExec("INSERT INTO `enum-set` VALUES\n(\"x00,x59\");")
tk.MustQuery("select `set` from `enum-set` use index(PRIMARY)").Check(testkit.Rows("x00,x59"))
tk.MustExec("admin check table `enum-set`")
}
>>>>>>> 886650b58... codec: Don't convert set or enum datum to float64 when encoding them (#32308)
4 changes: 2 additions & 2 deletions util/codec/codec.go
Original file line number Diff line number Diff line change
Expand Up @@ -114,9 +114,9 @@ func encode(sc *stmtctx.StatementContext, b []byte, vals []types.Datum, comparab
err = sc.HandleOverflow(err, err)
}
case types.KindMysqlEnum:
b = encodeUnsignedInt(b, uint64(vals[i].GetMysqlEnum().ToNumber()), comparable)
b = encodeUnsignedInt(b, vals[i].GetMysqlEnum().Value, comparable)
case types.KindMysqlSet:
b = encodeUnsignedInt(b, uint64(vals[i].GetMysqlSet().ToNumber()), comparable)
b = encodeUnsignedInt(b, vals[i].GetMysqlSet().Value, comparable)
case types.KindMysqlBit, types.KindBinaryLiteral:
// We don't need to handle errors here since the literal is ensured to be able to store in uint64 in convertToMysqlBit.
var val uint64
Expand Down

0 comments on commit f4c093e

Please sign in to comment.