Skip to content

Commit

Permalink
cherry pick pingcap#30588 to release-5.1
Browse files Browse the repository at this point in the history
Signed-off-by: ti-srebot <[email protected]>
  • Loading branch information
bestwoody authored and ti-srebot committed Dec 17, 2021
1 parent 0c172eb commit 3956f40
Show file tree
Hide file tree
Showing 5 changed files with 200 additions and 0 deletions.
182 changes: 182 additions & 0 deletions executor/executor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -8813,3 +8813,185 @@ 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()))
}
>>>>>>> 8e11e0367... *: fix the flen type datetime for union/case-when/control-funcs (#30588)
3 changes: 3 additions & 0 deletions expression/builtin_control.go
Original file line number Diff line number Diff line change
Expand Up @@ -145,6 +145,8 @@ func InferType4ControlFuncs(lexp, rexp Expression) *types.FieldType {
if resultFieldType.Tp == mysql.TypeEnum || resultFieldType.Tp == mysql.TypeSet {
resultFieldType.Tp = mysql.TypeVarchar
}
} else if resultFieldType.Tp == mysql.TypeDatetime {
types.TryToFixFlenOfDatetime(resultFieldType)
}
return resultFieldType
}
Expand Down Expand Up @@ -194,6 +196,7 @@ func (c *caseWhenFunctionClass) getFunction(ctx sessionctx.Context, args []Expre
decimal = 0
}
fieldTp.Decimal, fieldTp.Flen = decimal, flen
types.TryToFixFlenOfDatetime(fieldTp)
if fieldTp.EvalType().IsStringKind() && !isBinaryStr {
fieldTp.Charset, fieldTp.Collate = DeriveCollationFromExprs(ctx, args...)
if fieldTp.Charset == charset.CharsetBin && fieldTp.Collate == charset.CollationBin {
Expand Down
4 changes: 4 additions & 0 deletions expression/typeinfer_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -802,6 +802,8 @@ func (s *testInferTypeSuite) createTestCase4ControlFuncs() []typeInferTestCase {
{"ifnull(null, null)", mysql.TypeNull, charset.CharsetBin, mysql.BinaryFlag, 0, 0},
{"ifnull(c_double_d, c_timestamp_d)", mysql.TypeVarchar, charset.CharsetUTF8MB4, 0, 22, types.UnspecifiedLength},
{"ifnull(c_json, c_decimal)", mysql.TypeLongBlob, charset.CharsetUTF8MB4, 0, math.MaxUint32, types.UnspecifiedLength},
{"ifnull(c_time, c_date)", mysql.TypeDatetime, charset.CharsetUTF8MB4, 0, mysql.MaxDatetimeWidthNoFsp + 3 + 1, 3},
{"ifnull(c_time_d, c_date)", mysql.TypeDatetime, charset.CharsetUTF8MB4, 0, mysql.MaxDatetimeWidthNoFsp, 0},
{"if(c_int_d, c_decimal, c_int_d)", mysql.TypeNewDecimal, charset.CharsetBin, mysql.BinaryFlag, 14, 3},
{"if(c_int_d, c_char, c_int_d)", mysql.TypeString, charset.CharsetUTF8MB4, mysql.BinaryFlag, 20, types.UnspecifiedLength},
{"if(c_int_d, c_binary, c_int_d)", mysql.TypeString, charset.CharsetBin, mysql.BinaryFlag, 20, types.UnspecifiedLength},
Expand All @@ -816,6 +818,8 @@ func (s *testInferTypeSuite) createTestCase4ControlFuncs() []typeInferTestCase {
{"case when c_int_d > 1 then c_double_d else c_bchar end", mysql.TypeString, charset.CharsetUTF8MB4, mysql.BinaryFlag, 22, types.UnspecifiedLength},
{"case when c_int_d > 2 then c_double_d when c_int_d < 1 then c_decimal else c_double_d end", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, 22, 3},
{"case when c_double_d > 2 then c_decimal else 1 end", mysql.TypeNewDecimal, charset.CharsetBin, mysql.BinaryFlag, 6, 3},
{"case when c_time is not null then c_time else c_date end", mysql.TypeDatetime, charset.CharsetUTF8MB4, mysql.BinaryFlag, mysql.MaxDatetimeWidthNoFsp + 3 + 1, 3},
{"case when c_time_d is not null then c_time_d else c_date end", mysql.TypeDatetime, charset.CharsetUTF8MB4, mysql.BinaryFlag, mysql.MaxDatetimeWidthNoFsp, 0},
{"case when null then null else null end", mysql.TypeNull, charset.CharsetBin, mysql.BinaryFlag, 0, types.UnspecifiedLength},
}
}
Expand Down
1 change: 1 addition & 0 deletions planner/core/logical_plan_builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -1340,6 +1340,7 @@ func unionJoinFieldType(a, b *types.FieldType) *types.FieldType {
resultTp.Decimal = mathutil.Max(a.Decimal, b.Decimal)
// `Flen - Decimal` is the fraction before '.'
resultTp.Flen = mathutil.Max(a.Flen-a.Decimal, b.Flen-b.Decimal) + resultTp.Decimal
types.TryToFixFlenOfDatetime(resultTp)
if resultTp.EvalType() != types.ETInt && (a.EvalType() == types.ETInt || b.EvalType() == types.ETInt) && resultTp.Flen < mysql.MaxIntWidth {
resultTp.Flen = mysql.MaxIntWidth
}
Expand Down
10 changes: 10 additions & 0 deletions types/field_type.go
Original file line number Diff line number Diff line change
Expand Up @@ -105,6 +105,16 @@ func AggFieldType(tps []*FieldType) *FieldType {
return &currType
}

// TryToFixFlenOfDatetime try to fix flen of Datetime for specific func or other field merge cases
func TryToFixFlenOfDatetime(resultTp *FieldType) {
if resultTp.Tp == mysql.TypeDatetime {
resultTp.Flen = mysql.MaxDatetimeWidthNoFsp
if resultTp.Decimal > 0 {
resultTp.Flen += resultTp.Decimal + 1
}
}
}

// AggregateEvalType aggregates arguments' EvalType of a multi-argument function.
func AggregateEvalType(fts []*FieldType, flag *uint) EvalType {
var (
Expand Down

0 comments on commit 3956f40

Please sign in to comment.