From 3956f405ce114e8a858a3a6e1577d6a2155d5b1a Mon Sep 17 00:00:00 2001 From: bestwoody <89765764+bestwoody@users.noreply.github.com> Date: Fri, 17 Dec 2021 13:14:35 +0800 Subject: [PATCH] cherry pick #30588 to release-5.1 Signed-off-by: ti-srebot --- executor/executor_test.go | 182 +++++++++++++++++++++++++++ expression/builtin_control.go | 3 + expression/typeinfer_test.go | 4 + planner/core/logical_plan_builder.go | 1 + types/field_type.go | 10 ++ 5 files changed, 200 insertions(+) diff --git a/executor/executor_test.go b/executor/executor_test.go index f33554fec287f..ad193e8a722bb 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -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< 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<>>>>>> 8e11e0367... *: fix the flen type datetime for union/case-when/control-funcs (#30588) diff --git a/expression/builtin_control.go b/expression/builtin_control.go index e40311eb614a5..f6f8ebe0d18dc 100644 --- a/expression/builtin_control.go +++ b/expression/builtin_control.go @@ -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 } @@ -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 { diff --git a/expression/typeinfer_test.go b/expression/typeinfer_test.go index e0324123bbaae..9dc34c4b8a400 100644 --- a/expression/typeinfer_test.go +++ b/expression/typeinfer_test.go @@ -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}, @@ -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}, } } diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index 03caaaa14756d..b68d8c5736d3b 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -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 } diff --git a/types/field_type.go b/types/field_type.go index 706d7cf595a7e..9c4efef4121a3 100644 --- a/types/field_type.go +++ b/types/field_type.go @@ -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 (