Skip to content

Commit

Permalink
Merge remote-tracking branch 'upstream/release-4.0' into release-4.0-…
Browse files Browse the repository at this point in the history
…ae65cae8d04c
  • Loading branch information
wshwsh12 committed Nov 23, 2021
2 parents 061bc31 + 1e156bc commit 2adaf3c
Show file tree
Hide file tree
Showing 8 changed files with 112 additions and 38 deletions.
33 changes: 4 additions & 29 deletions expression/builtin_compare.go
Original file line number Diff line number Diff line change
Expand Up @@ -634,11 +634,7 @@ func (b *builtinGreatestTimeSig) Clone() builtinFunc {

// evalString evals a builtinGreatestTimeSig.
// See http://dev.mysql.com/doc/refman/5.7/en/comparison-operators.html#function_greatest
func (b *builtinGreatestTimeSig) evalString(row chunk.Row) (res string, isNull bool, err error) {
var (
strRes string
timeRes types.Time
)
func (b *builtinGreatestTimeSig) evalString(row chunk.Row) (strRes string, isNull bool, err error) {
sc := b.ctx.GetSessionVars().StmtCtx
for i := 0; i < len(b.args); i++ {
v, isNull, err := b.args[i].EvalString(b.ctx, row)
Expand All @@ -657,16 +653,8 @@ func (b *builtinGreatestTimeSig) evalString(row chunk.Row) (res string, isNull b
if i == 0 || strings.Compare(v, strRes) > 0 {
strRes = v
}
if i == 0 || t.Compare(timeRes) > 0 {
timeRes = t
}
}
if timeRes.IsZero() {
res = strRes
} else {
res = timeRes.String()
}
return res, false, nil
return strRes, false, nil
}

type leastFunctionClass struct {
Expand Down Expand Up @@ -852,12 +840,7 @@ func (b *builtinLeastTimeSig) Clone() builtinFunc {

// evalString evals a builtinLeastTimeSig.
// See http://dev.mysql.com/doc/refman/5.7/en/comparison-operators.html#functionleast
func (b *builtinLeastTimeSig) evalString(row chunk.Row) (res string, isNull bool, err error) {
var (
// timeRes will be converted to a strRes only when the arguments is a valid datetime value.
strRes string // Record the strRes of each arguments.
timeRes types.Time // Record the time representation of a valid arguments.
)
func (b *builtinLeastTimeSig) evalString(row chunk.Row) (strRes string, isNull bool, err error) {
sc := b.ctx.GetSessionVars().StmtCtx
for i := 0; i < len(b.args); i++ {
v, isNull, err := b.args[i].EvalString(b.ctx, row)
Expand All @@ -875,17 +858,9 @@ func (b *builtinLeastTimeSig) evalString(row chunk.Row) (res string, isNull bool
if i == 0 || strings.Compare(v, strRes) < 0 {
strRes = v
}
if i == 0 || t.Compare(timeRes) < 0 {
timeRes = t
}
}

if timeRes.IsZero() {
res = strRes
} else {
res = timeRes.String()
}
return res, false, nil
return strRes, false, nil
}

type intervalFunctionClass struct {
Expand Down
4 changes: 2 additions & 2 deletions expression/builtin_compare_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -295,11 +295,11 @@ func (s *testEvaluatorSuite) TestGreatestLeastFunc(c *C) {
},
{
[]interface{}{tm, "invalid_time_1", "invalid_time_2", tmWithFsp},
curTimeWithFspString, curTimeString, false, false,
"invalid_time_2", curTimeString, false, false,
},
{
[]interface{}{tm, "invalid_time_2", "invalid_time_1", tmWithFsp},
curTimeWithFspString, curTimeString, false, false,
"invalid_time_2", curTimeString, false, false,
},
{
[]interface{}{tm, "invalid_time", nil, tmWithFsp},
Expand Down
3 changes: 1 addition & 2 deletions expression/builtin_time_vec.go
Original file line number Diff line number Diff line change
Expand Up @@ -1836,12 +1836,11 @@ func (b *builtinHourSig) vecEvalInt(input *chunk.Chunk, result *chunk.Column) er
result.ResizeInt64(n, false)
result.MergeNulls(buf)
i64s := result.Int64s()
ds := buf.GoDurations()
for i := 0; i < n; i++ {
if result.IsNull(i) {
continue
}
i64s[i] = int64(ds[i].Hours())
i64s[i] = int64(buf.GetDuration(i, int(types.UnspecifiedFsp)).Hour())
}
return nil
}
Expand Down
12 changes: 12 additions & 0 deletions expression/column.go
Original file line number Diff line number Diff line change
Expand Up @@ -196,6 +196,13 @@ type Column struct {
OrigName string
IsHidden bool

// IsPrefix indicates whether this column is a prefix column in index.
//
// for example:
// pk(col1, col2), index(col1(10)), key: col1(10)_col1_col2 => index's col1 will be true
// pk(col1(10), col2), index(col1), key: col1_col1(10)_col2 => pk's col1 will be true
IsPrefix bool

// InOperand indicates whether this column is the inner operand of column equal condition converted
// from `[not] in (subq)`.
InOperand bool
Expand Down Expand Up @@ -504,6 +511,11 @@ func ColInfo2Col(cols []*Column, col *model.ColumnInfo) *Column {
func indexCol2Col(colInfos []*model.ColumnInfo, cols []*Column, col *model.IndexColumn) *Column {
for i, info := range colInfos {
if info.Name.L == col.Name.L {
if col.Length > 0 && info.FieldType.Flen > col.Length {
c := *cols[i]
c.IsPrefix = true
return &c
}
return cols[i]
}
}
Expand Down
34 changes: 32 additions & 2 deletions expression/integration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -3698,8 +3698,7 @@ func (s *testIntegrationSuite) TestCompareBuiltin(c *C) {
result.Check(testkit.Rows("3 c 1.3 2"))
tk.MustQuery("show warnings").Check(testkit.Rows())
result = tk.MustQuery(`select greatest(cast("2017-01-01" as datetime), "123", "234", cast("2018-01-01" as date)), greatest(cast("2017-01-01" as date), "123", null)`)
// todo: MySQL returns "2018-01-01 <nil>"
result.Check(testkit.Rows("2018-01-01 00:00:00 <nil>"))
result.Check(testkit.Rows("234 <nil>"))
tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Warning|1292|Incorrect time value: '123'", "Warning|1292|Incorrect time value: '234'", "Warning|1292|Incorrect time value: '123'"))
// for least
result = tk.MustQuery(`select least(1, 2, 3), least("a", "b", "c"), least(1.1, 1.2, 1.3), least("123a", 1, 2)`)
Expand Down Expand Up @@ -8359,6 +8358,24 @@ func (s *testIntegrationSuite) TestControlFunctionWithEnumOrSet(c *C) {
tk.MustQuery("SELECT '1' = (case when 0 <=> 1 then a else a end) from t;").Check(testkit.Rows("1"))
}

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

tk.MustExec("drop table if exists t1;")
tk.MustExec("create table t1(c1 datetime);")
tk.MustExec("insert into t1 values('2021-12-12 10:10:10.000');")
tk.MustExec("set tidb_enable_vectorized_expression = on;")
tk.MustQuery("select greatest(c1, '99999999999999') from t1;").Check(testkit.Rows("99999999999999"))
tk.MustExec("set tidb_enable_vectorized_expression = off;")
tk.MustQuery("select greatest(c1, '99999999999999') from t1;").Check(testkit.Rows("99999999999999"))

tk.MustExec("set tidb_enable_vectorized_expression = on;")
tk.MustQuery("select least(c1, '99999999999999') from t1;").Check(testkit.Rows("2021-12-12 10:10:10"))
tk.MustExec("set tidb_enable_vectorized_expression = off;")
tk.MustQuery("select least(c1, '99999999999999') from t1;").Check(testkit.Rows("2021-12-12 10:10:10"))
}

func (s *testIntegrationSuite) TestConstPropNullFunctions(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
Expand All @@ -8377,6 +8394,19 @@ func (s *testIntegrationSuite) TestConstPropNullFunctions(c *C) {
tk.MustQuery("select * from t2 where t2.i2=((select count(1) from t1 where t1.i1=t2.i2))").Check(testkit.Rows("1 <nil> 0.1"))
}

func (s *testIntegrationSuite) TestIssue28643(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
tk.MustExec("drop table if exists t")
tk.MustExec("create table t(a time(4));")
tk.MustExec("insert into t values(\"-838:59:59.000000\");")
tk.MustExec("insert into t values(\"838:59:59.000000\");")
tk.MustExec("set tidb_enable_vectorized_expression = on;")
tk.MustQuery("select hour(a) from t;").Check(testkit.Rows("838", "838"))
tk.MustExec("set tidb_enable_vectorized_expression = off;")
tk.MustQuery("select hour(a) from t;").Check(testkit.Rows("838", "838"))
}

func (s *testIntegrationSuite) TestIssue29244(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
Expand Down
36 changes: 36 additions & 0 deletions planner/core/integration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -2125,3 +2125,39 @@ func (s *testIntegrationSuite) TestIssue27797(c *C) {
result = tk.MustQuery("select col2 from IDT_HP24172 where col1 = 8388607 and col1 in (select col1 from IDT_HP24172);")
result.Check(testkit.Rows("<nil>"))
}

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

tk.MustExec("drop table if exists tbl_29711")
tk.MustExec("CREATE TABLE `tbl_29711` (" +
"`col_250` text COLLATE utf8_unicode_ci NOT NULL," +
"`col_251` varchar(10) COLLATE utf8_unicode_ci NOT NULL," +
"PRIMARY KEY (`col_251`,`col_250`(1)));")
tk.MustQuery("explain " +
"select /*+ LIMIT_TO_COP() */ col_250,col_251 from tbl_29711 use index (primary) where col_251 between 'Bob' and 'David' order by col_250,col_251 limit 6;").
Check(testkit.Rows(
"TopN_9 6.00 root test.tbl_29711.col_250:asc, test.tbl_29711.col_251:asc, offset:0, count:6",
"└─IndexLookUp_15 6.00 root ",
" ├─IndexRangeScan_12(Build) 250.00 cop[tikv] table:tbl_29711, index:PRIMARY(col_251, col_250) range:[\"Bob\",\"David\"], keep order:false, stats:pseudo",
" └─TopN_14(Probe) 6.00 cop[tikv] test.tbl_29711.col_250:asc, test.tbl_29711.col_251:asc, offset:0, count:6",
" └─TableRowIDScan_13 250.00 cop[tikv] table:tbl_29711 keep order:false, stats:pseudo",
))

tk.MustExec("drop table if exists t29711")
tk.MustExec("CREATE TABLE `t29711` (" +
"`a` varchar(10) DEFAULT NULL," +
"`b` int(11) DEFAULT NULL," +
"`c` int(11) DEFAULT NULL," +
"KEY `ia` (`a`(2)))")
tk.MustQuery("explain select /*+ LIMIT_TO_COP() */ * from t29711 use index (ia) order by a limit 10;").
Check(testkit.Rows(
"TopN_8 10.00 root test.t29711.a:asc, offset:0, count:10",
"└─IndexLookUp_14 10.00 root ",
" ├─IndexFullScan_11(Build) 10000.00 cop[tikv] table:t29711, index:ia(a) keep order:false, stats:pseudo",
" └─TopN_13(Probe) 10.00 cop[tikv] test.t29711.a:asc, offset:0, count:10",
" └─TableRowIDScan_12 10000.00 cop[tikv] table:t29711 keep order:false, stats:pseudo",
))

}
26 changes: 24 additions & 2 deletions planner/core/task.go
Original file line number Diff line number Diff line change
Expand Up @@ -955,14 +955,36 @@ func (p *PhysicalTopN) getPushedDownTopN(childPlan PhysicalPlan) *PhysicalTopN {
return topN
}

// canPushToIndexPlan checks if this TopN can be pushed to the index side of copTask.
// It can be pushed to the index side when all columns used by ByItems are available from the index side and
// there's no prefix index column.
func (p *PhysicalTopN) canPushToIndexPlan(indexPlan PhysicalPlan, byItemCols []*expression.Column) bool {
schema := indexPlan.Schema()
for _, col := range byItemCols {
pos := schema.ColumnIndex(col)
if pos == -1 {
return false
}
if schema.Columns[pos].IsPrefix {
return false
}
}
return true
}

func (p *PhysicalTopN) attach2Task(tasks ...task) task {
t := tasks[0].copy()
inputCount := t.count()
if copTask, ok := t.(*copTask); ok && p.canPushDown(copTask) && len(copTask.rootTaskConds) == 0 {
cols := make([]*expression.Column, 0, len(p.ByItems))
for _, item := range p.ByItems {
cols = append(cols, expression.ExtractColumns(item.Expr)...)
}
needPushDown := len(cols) > 0
if copTask, ok := t.(*copTask); ok && needPushDown && p.canPushDown(copTask) && len(copTask.rootTaskConds) == 0 {
// If all columns in topN are from index plan, we push it to index plan, otherwise we finish the index plan and
// push it to table plan.
var pushedDownTopN *PhysicalTopN
if !copTask.indexPlanFinished && p.allColsFromSchema(copTask.indexPlan.Schema()) {
if !copTask.indexPlanFinished && p.canPushToIndexPlan(copTask.indexPlan, cols) {
pushedDownTopN = p.getPushedDownTopN(copTask.indexPlan)
copTask.indexPlan = pushedDownTopN
} else {
Expand Down
2 changes: 1 addition & 1 deletion table/tables/tables_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -355,7 +355,7 @@ func (ts *testSuite) TestTableFromMeta(c *C) {
c.Assert(ts.se.NewTxn(context.Background()), IsNil)
tb, err := ts.dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("meta"))
c.Assert(err, IsNil)
tbInfo := tb.Meta()
tbInfo := tb.Meta().Clone()

// For test coverage
tbInfo.Columns[0].GeneratedExprString = "a"
Expand Down

0 comments on commit 2adaf3c

Please sign in to comment.