Skip to content

Commit

Permalink
Merge pull request #7 from hanfei1991/release-4.0-29178dfa8dbe
Browse files Browse the repository at this point in the history
Release 4.0 29178df
  • Loading branch information
hanfei1991 authored Sep 1, 2020
2 parents 24b7687 + b9e1616 commit 516918e
Show file tree
Hide file tree
Showing 11 changed files with 184 additions and 19 deletions.
67 changes: 67 additions & 0 deletions executor/partition_table_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,67 @@
// Copyright 2020 PingCAP, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// See the License for the specific language governing permissions and
// limitations under the License.

package executor_test

import (
. "github.com/pingcap/check"
"github.com/pingcap/tidb/util/testkit"
)

func (s *testSuite9) TestPartitionReaderUnderApply(c *C) {
tk := testkit.NewTestKitWithInit(c, s.store)
tk.MustExec("use test")

// For issue 19458.
tk.MustExec("drop table if exists t")
tk.MustExec("create table t(c_int int)")
tk.MustExec("insert into t values(1), (2), (3), (4), (5), (6), (7), (8), (9)")
tk.MustExec("DROP TABLE IF EXISTS `t1`")
tk.MustExec(`CREATE TABLE t1 (
c_int int NOT NULL,
c_str varchar(40) NOT NULL,
c_datetime datetime NOT NULL,
c_timestamp timestamp NULL DEFAULT NULL,
c_double double DEFAULT NULL,
c_decimal decimal(12,6) DEFAULT NULL,
PRIMARY KEY (c_int,c_str,c_datetime)
) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_0900_ai_ci
PARTITION BY RANGE (c_int)
(PARTITION p0 VALUES LESS THAN (2) ENGINE = InnoDB,
PARTITION p1 VALUES LESS THAN (4) ENGINE = InnoDB,
PARTITION p2 VALUES LESS THAN (6) ENGINE = InnoDB,
PARTITION p3 VALUES LESS THAN (8) ENGINE = InnoDB,
PARTITION p4 VALUES LESS THAN (10) ENGINE = InnoDB,
PARTITION p5 VALUES LESS THAN (20) ENGINE = InnoDB,
PARTITION p6 VALUES LESS THAN (50) ENGINE = InnoDB,
PARTITION p7 VALUES LESS THAN (1000000000) ENGINE = InnoDB)`)
tk.MustExec("INSERT INTO `t1` VALUES (19,'nifty feistel','2020-02-28 04:01:28','2020-02-04 06:11:57',32.430079,1.284000),(20,'objective snyder','2020-04-15 17:55:04','2020-05-30 22:04:13',37.690874,9.372000)")
tk.MustExec("begin")
tk.MustExec("insert into t1 values (22, 'wizardly saha', '2020-05-03 16:35:22', '2020-05-03 02:18:42', 96.534810, 0.088)")
tk.MustQuery("select c_int from t where (select min(t1.c_int) from t1 where t1.c_int > t.c_int) > (select count(*) from t1 where t1.c_int > t.c_int) order by c_int").Check(testkit.Rows(
"1", "2", "3", "4", "5", "6", "7", "8", "9"))
tk.MustExec("rollback")

// For issue 19450.
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))")
tk.MustExec("create table t2 (c_int int, c_str varchar(40), c_decimal decimal(12, 6), primary key (c_int)) partition by hash (c_int) partitions 4")
tk.MustExec("insert into t1 values (1, 'romantic robinson', 4.436), (2, 'stoic chaplygin', 9.826), (3, 'vibrant shamir', 6.300), (4, 'hungry wilson', 4.900), (5, 'naughty swartz', 9.524)")
tk.MustExec("insert into t2 select * from t1")
tk.MustQuery("select * from t1 where c_decimal in (select c_decimal from t2 where t1.c_int = t2.c_int or t1.c_int = t2.c_int and t1.c_str > t2.c_str)").Check(testkit.Rows(
"1 romantic robinson 4.436000",
"2 stoic chaplygin 9.826000",
"3 vibrant shamir 6.300000",
"4 hungry wilson 4.900000",
"5 naughty swartz 9.524000"))
}
14 changes: 14 additions & 0 deletions expression/builtin_math.go
Original file line number Diff line number Diff line change
Expand Up @@ -1965,6 +1965,9 @@ func (b *builtinTruncateIntSig) evalInt(row chunk.Row) (int64, bool, error) {
if isNull || err != nil {
return 0, isNull, err
}
if mysql.HasUnsignedFlag(b.args[1].GetType().Flag) {
return x, false, nil
}

d, isNull, err := b.args[1].EvalInt(b.ctx, row)
if isNull || err != nil {
Expand All @@ -1974,6 +1977,10 @@ func (b *builtinTruncateIntSig) evalInt(row chunk.Row) (int64, bool, error) {
if d >= 0 {
return x, false, nil
}
// -MinInt = MinInt, special case
if d == mathutil.MinInt {
return 0, false, nil
}
shift := int64(math.Pow10(int(-d)))
return x / shift * shift, false, nil
}
Expand All @@ -1995,6 +2002,9 @@ func (b *builtinTruncateUintSig) evalInt(row chunk.Row) (int64, bool, error) {
if isNull || err != nil {
return 0, isNull, err
}
if mysql.HasUnsignedFlag(b.args[1].GetType().Flag) {
return x, false, nil
}
uintx := uint64(x)

d, isNull, err := b.args[1].EvalInt(b.ctx, row)
Expand All @@ -2004,6 +2014,10 @@ func (b *builtinTruncateUintSig) evalInt(row chunk.Row) (int64, bool, error) {
if d >= 0 {
return x, false, nil
}
// -MinInt = MinInt, special case
if d == mathutil.MinInt {
return 0, false, nil
}
shift := uint64(math.Pow10(int(-d)))
return int64(uintx / shift * shift), false, nil
}
27 changes: 23 additions & 4 deletions expression/builtin_math_vec.go
Original file line number Diff line number Diff line change
Expand Up @@ -803,13 +803,22 @@ func (b *builtinTruncateIntSig) vecEvalInt(input *chunk.Chunk, result *chunk.Col
i64s := result.Int64s()
buf64s := buf.Int64s()

if mysql.HasUnsignedFlag(b.args[1].GetType().Flag) {
return nil
}

for i := 0; i < len(i64s); i++ {
if result.IsNull(i) {
continue
}
if buf64s[i] < 0 {
shift := int64(math.Pow10(int(-buf64s[i])))
i64s[i] = i64s[i] / shift * shift
// -MinInt = MinInt, special case
if buf64s[i] == mathutil.MinInt {
i64s[i] = 0
} else {
shift := int64(math.Pow10(int(-buf64s[i])))
i64s[i] = i64s[i] / shift * shift
}
}
}
return nil
Expand Down Expand Up @@ -838,13 +847,23 @@ func (b *builtinTruncateUintSig) vecEvalInt(input *chunk.Chunk, result *chunk.Co
i64s := result.Int64s()
buf64s := buf.Int64s()

if mysql.HasUnsignedFlag(b.args[1].GetType().Flag) {
return nil
}

for i := 0; i < n; i++ {
if result.IsNull(i) {
continue
}

if buf64s[i] < 0 {
shift := uint64(math.Pow10(int(-buf64s[i])))
i64s[i] = int64(uint64(i64s[i]) / shift * shift)
// -MinInt = MinInt, special case
if buf64s[i] == mathutil.MinInt {
i64s[i] = 0
} else {
shift := uint64(math.Pow10(int(-buf64s[i])))
i64s[i] = int64(uint64(i64s[i]) / shift * shift)
}
}
}
return nil
Expand Down
36 changes: 36 additions & 0 deletions expression/integration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -508,6 +508,21 @@ func (s *testIntegrationSuite2) TestMathBuiltin(c *C) {
result.Check(testkit.Rows("100 123.45 123.4 123.456 1.230000000000000000000000000000 123.45"))
result = tk.MustQuery("SELECT truncate(9223372036854775807, -7), truncate(9223372036854775808, -10), truncate(cast(-1 as unsigned), -10);")
result.Check(testkit.Rows("9223372036850000000 9223372030000000000 18446744070000000000"))
// issue 17181,19390
tk.MustQuery("select truncate(42, -9223372036854775808);").Check(testkit.Rows("0"))
tk.MustQuery("select truncate(42, 9223372036854775808);").Check(testkit.Rows("42"))
tk.MustQuery("select truncate(42, -2147483648);").Check(testkit.Rows("0"))
tk.MustQuery("select truncate(42, 2147483648);").Check(testkit.Rows("42"))
tk.MustQuery("select truncate(42, 18446744073709551615);").Check(testkit.Rows("42"))
tk.MustQuery("select truncate(42, 4294967295);").Check(testkit.Rows("42"))
tk.MustQuery("select truncate(42, -0);").Check(testkit.Rows("42"))
tk.MustQuery("select truncate(42, -307);").Check(testkit.Rows("0"))
tk.MustQuery("select truncate(42, -308);").Check(testkit.Rows("0"))
tk.MustQuery("select truncate(42, -309);").Check(testkit.Rows("0"))
tk.MustExec(`drop table if exists t;`)
tk.MustExec("create table t (a bigint unsigned);")
tk.MustExec("insert into t values (18446744073709551615), (4294967295), (9223372036854775808), (2147483648);")
tk.MustQuery("select truncate(42, a) from t;").Check(testkit.Rows("42", "42", "42", "42"))

tk.MustExec(`drop table if exists t;`)
tk.MustExec(`create table t(a date, b datetime, c timestamp, d varchar(20));`)
Expand Down Expand Up @@ -6558,6 +6573,14 @@ func (s *testIntegrationSuite) TestIssue17727(c *C) {
tk.MustQuery("select @@last_plan_from_cache;").Check(testkit.Rows("1"))
}

func (s *testIntegrationSuite) TestIssue18515(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 int, b json, c int AS (JSON_EXTRACT(b, '$.population')), key(c));")
tk.MustExec("select /*+ TIDB_INLJ(t2) */ t1.a, t1.c, t2.a from t t1, t t2 where t1.c=t2.c;")
}

func (s *testIntegrationSerialSuite) TestIssue17989(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
Expand Down Expand Up @@ -6763,3 +6786,16 @@ func (s *testIntegrationSerialSuite) TestIssue19045(c *C) {
tk.MustExec(`insert into t(a) values('101'),('101');`)
tk.MustQuery(`select ( SELECT t1.a FROM t1, t2 WHERE t1.b = t2.a AND t2.b = '03' AND t1.c = a.a) invode from t a ;`).Check(testkit.Rows("a011", "a011"))
}

func (s *testIntegrationSuite) TestIssue19504(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
tk.MustExec("drop table if exists t1;")
tk.MustExec("create table t1 (c_int int, primary key (c_int));")
tk.MustExec("insert into t1 values (1), (2), (3);")
tk.MustExec("drop table if exists t2;")
tk.MustExec("create table t2 (c_int int, primary key (c_int));")
tk.MustExec("insert into t2 values (1);")
tk.MustQuery("select (select count(c_int) from t2 where c_int = t1.c_int) c1, (select count(1) from t2 where c_int = t1.c_int) c2 from t1;").
Check(testkit.Rows("1 1", "0 0", "0 0"))
}
2 changes: 0 additions & 2 deletions planner/core/find_best_task.go
Original file line number Diff line number Diff line change
Expand Up @@ -731,7 +731,6 @@ func (ds *DataSource) buildIndexMergeTableScan(prop *property.PhysicalProperty,
physicalTableID: ds.physicalTableID,
}.Init(ds.ctx, ds.blockOffset)
ts.SetSchema(ds.schema.Clone())
ts.Columns = ExpandVirtualColumn(ts.Columns, ts.schema, ts.Table.Columns)
if ts.Table.PKIsHandle {
if pkColInfo := ts.Table.GetPkColInfo(); pkColInfo != nil {
if ds.statisticTable.Columns[pkColInfo.ID] != nil {
Expand Down Expand Up @@ -827,7 +826,6 @@ func (ds *DataSource) convertToIndexScan(prop *property.PhysicalProperty, candid
physicalTableID: ds.physicalTableID,
}.Init(ds.ctx, is.blockOffset)
ts.SetSchema(ds.schema.Clone())
ts.Columns = ExpandVirtualColumn(ts.Columns, ts.schema, ts.Table.Columns)
cop.tablePlan = ts
}
cop.cst = cost
Expand Down
8 changes: 8 additions & 0 deletions planner/core/initialize.go
Original file line number Diff line number Diff line change
Expand Up @@ -469,6 +469,14 @@ func (p BatchPointGetPlan) Init(ctx sessionctx.Context, stats *property.StatsInf
return &p
}

// Init initializes PointGetPlan.
func (p PointGetPlan) Init(ctx sessionctx.Context, stats *property.StatsInfo, offset int, props ...*property.PhysicalProperty) *PointGetPlan {
p.basePlan = newBasePlan(ctx, plancodec.TypePointGet, offset)
p.stats = stats
p.Columns = ExpandVirtualColumn(p.Columns, p.schema, p.TblInfo.Columns)
return &p
}

func flattenTreePlan(plan PhysicalPlan, plans []PhysicalPlan) []PhysicalPlan {
plans = append(plans, plan)
for _, child := range plan.Children() {
Expand Down
18 changes: 16 additions & 2 deletions planner/core/physical_plans.go
Original file line number Diff line number Diff line change
Expand Up @@ -287,6 +287,16 @@ func (ts *PhysicalTableScan) IsPartition() (bool, int64) {
// ExpandVirtualColumn expands the virtual column's dependent columns to ts's schema and column.
func ExpandVirtualColumn(columns []*model.ColumnInfo, schema *expression.Schema,
colsInfo []*model.ColumnInfo) []*model.ColumnInfo {
copyColumn := make([]*model.ColumnInfo, len(columns))
copy(copyColumn, columns)
var extraColumn *expression.Column
var extraColumnModel *model.ColumnInfo
if schema.Columns[len(schema.Columns)-1].ID == model.ExtraHandleID {
extraColumn = schema.Columns[len(schema.Columns)-1]
extraColumnModel = copyColumn[len(copyColumn)-1]
schema.Columns = schema.Columns[:len(schema.Columns)-1]
copyColumn = copyColumn[:len(copyColumn)-1]
}
schemaColumns := schema.Columns
for _, col := range schemaColumns {
if col.VirtualExpr == nil {
Expand All @@ -297,11 +307,15 @@ func ExpandVirtualColumn(columns []*model.ColumnInfo, schema *expression.Schema,
for _, baseCol := range baseCols {
if !schema.Contains(baseCol) {
schema.Columns = append(schema.Columns, baseCol)
columns = append(columns, FindColumnInfoByID(colsInfo, baseCol.ID))
copyColumn = append(copyColumn, FindColumnInfoByID(colsInfo, baseCol.ID))
}
}
}
return columns
if extraColumn != nil {
schema.Columns = append(schema.Columns, extraColumn)
copyColumn = append(copyColumn, extraColumnModel)
}
return copyColumn
}

//SetIsChildOfIndexLookUp is to set the bool if is a child of IndexLookUpReader
Expand Down
8 changes: 0 additions & 8 deletions planner/core/point_get_plan.go
Original file line number Diff line number Diff line change
Expand Up @@ -73,14 +73,6 @@ type nameValuePair struct {
param *driver.ParamMarkerExpr
}

// Init initializes PointGetPlan.
func (p PointGetPlan) Init(ctx sessionctx.Context, stats *property.StatsInfo, offset int, props ...*property.PhysicalProperty) *PointGetPlan {
p.basePlan = newBasePlan(ctx, plancodec.TypePointGet, offset)
p.stats = stats
p.Columns = ExpandVirtualColumn(p.Columns, p.schema, p.TblInfo.Columns)
return &p
}

// Schema implements the Plan interface.
func (p *PointGetPlan) Schema() *expression.Schema {
return p.schema
Expand Down
14 changes: 12 additions & 2 deletions planner/core/rule_decorrelate.go
Original file line number Diff line number Diff line change
Expand Up @@ -175,9 +175,19 @@ func (s *decorrelateSolver) optimize(ctx context.Context, p LogicalPlan) (Logica
outerCol.RetType = first.RetTp
outerColsInSchema = append(outerColsInSchema, outerCol)
}
newAggFuncs = append(newAggFuncs, agg.AggFuncs...)
agg.AggFuncs = newAggFuncs
apply.SetSchema(expression.MergeSchema(expression.NewSchema(outerColsInSchema...), innerPlan.Schema()))
resetNotNullFlag(apply.schema, outerPlan.Schema().Len(), apply.schema.Len())

for i, aggFunc := range agg.AggFuncs {
if idx := apply.schema.ColumnIndex(aggFunc.Args[0].(*expression.Column)); idx != -1 {
desc, err := aggregation.NewAggFuncDesc(agg.ctx, agg.AggFuncs[i].Name, []expression.Expression{apply.schema.Columns[idx]}, false)
if err != nil {
return nil, err
}
newAggFuncs = append(newAggFuncs, desc)
}
}
agg.AggFuncs = newAggFuncs
np, err := s.optimize(ctx, p)
if err != nil {
return nil, err
Expand Down
1 change: 1 addition & 0 deletions planner/core/rule_partition_processor.go
Original file line number Diff line number Diff line change
Expand Up @@ -707,6 +707,7 @@ func (s *partitionProcessor) makeUnionAllChildren(ds *DataSource, pi *model.Part
// Not a deep copy.
newDataSource := *ds
newDataSource.baseLogicalPlan = newBaseLogicalPlan(ds.SCtx(), plancodec.TypeTableScan, &newDataSource, ds.blockOffset)
newDataSource.schema = ds.schema.Clone()
newDataSource.isPartition = true
newDataSource.physicalTableID = pi.Definitions[i].ID
newDataSource.possibleAccessPaths = make([]*util.AccessPath, len(ds.possibleAccessPaths))
Expand Down
8 changes: 7 additions & 1 deletion planner/core/task.go
Original file line number Diff line number Diff line change
Expand Up @@ -711,6 +711,13 @@ func finishCopTask(ctx sessionctx.Context, task task) task {
// Network cost of transferring rows of table scan to TiDB.
if t.tablePlan != nil {
t.cst += t.count() * sessVars.NetworkFactor * t.tblColHists.GetAvgRowSize(ctx, t.tablePlan.Schema().Columns, false, false)

tp := t.tablePlan
for len(tp.Children()) > 0 {
tp = tp.Children()[0]
}
ts := tp.(*PhysicalTableScan)
ts.Columns = ExpandVirtualColumn(ts.Columns, ts.schema, ts.Table.Columns)
}
t.cst /= copIterWorkers
newTask := &rootTask{
Expand Down Expand Up @@ -744,7 +751,6 @@ func finishCopTask(ctx sessionctx.Context, task task) task {
StoreType: ts.StoreType,
}.Init(ctx, t.tablePlan.SelectBlockOffset())
p.stats = t.tablePlan.statsInfo()
ts.Columns = ExpandVirtualColumn(ts.Columns, ts.schema, ts.Table.Columns)
newTask.p = p
}

Expand Down

0 comments on commit 516918e

Please sign in to comment.