Skip to content

Commit

Permalink
Merge branch 'release-3.0' into automated-cherry-pick-of-pingcap#10837-…
Browse files Browse the repository at this point in the history
…upstream-release-3.0
  • Loading branch information
sre-bot authored Oct 18, 2019
2 parents 5f915f9 + f45c29b commit 0c6c6aa
Show file tree
Hide file tree
Showing 7 changed files with 131 additions and 19 deletions.
4 changes: 2 additions & 2 deletions ddl/column.go
Original file line number Diff line number Diff line change
Expand Up @@ -588,9 +588,9 @@ func generateOriginDefaultValue(col *model.ColumnInfo) (interface{}, error) {
return odValue, nil
}

func findColumnInIndexCols(c *model.ColumnInfo, cols []*ast.IndexColName) bool {
func findColumnInIndexCols(c string, cols []*ast.IndexColName) bool {
for _, c1 := range cols {
if c.Name.L == c1.Column.Name.L {
if c == c1.Column.Name.L {
return true
}
}
Expand Down
32 changes: 32 additions & 0 deletions ddl/db_partition_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1080,6 +1080,38 @@ func (s *testIntegrationSuite5) TestPartitionUniqueKeyNeedAllFieldsInPf(c *C) {
partition p2 values less than (15)
)`
assertErrorCode(c, tk, sql9, tmysql.ErrUniqueKeyNeedAllFieldsInPf)

sql10 := `create table part8 (
a int not null,
b int not null,
c int default null,
d int default null,
e int default null,
primary key (a, b),
unique key (c, d)
)
partition by range columns (b) (
partition p0 values less than (4),
partition p1 values less than (7),
partition p2 values less than (11)
)`
assertErrorCode(c, tk, sql10, tmysql.ErrUniqueKeyNeedAllFieldsInPf)

sql11 := `create table part9 (
a int not null,
b int not null,
c int default null,
d int default null,
e int default null,
primary key (a, b),
unique key (b, c, d)
)
partition by range columns (b, c) (
partition p0 values less than (4, 5),
partition p1 values less than (7, 9),
partition p2 values less than (11, 22)
)`
assertErrorCode(c, tk, sql11, tmysql.ErrUniqueKeyNeedAllFieldsInPf)
}

func (s *testIntegrationSuite3) TestPartitionDropIndex(c *C) {
Expand Down
57 changes: 46 additions & 11 deletions ddl/partition.go
Original file line number Diff line number Diff line change
Expand Up @@ -493,17 +493,22 @@ func getPartitionIDs(table *model.TableInfo) []int64 {
// checkRangePartitioningKeysConstraints checks that the range partitioning key is included in the table constraint.
func checkRangePartitioningKeysConstraints(sctx sessionctx.Context, s *ast.CreateTableStmt, tblInfo *model.TableInfo, constraints []*ast.Constraint) error {
// Returns directly if there is no constraint in the partition table.
// TODO: Remove the test 's.Partition.Expr == nil' when we support 'PARTITION BY RANGE COLUMNS'
if len(constraints) == 0 || s.Partition.Expr == nil {
if len(constraints) == 0 {
return nil
}

// Parse partitioning key, extract the column names in the partitioning key to slice.
buf := new(bytes.Buffer)
s.Partition.Expr.Format(buf)
partCols, err := extractPartitionColumns(buf.String(), tblInfo)
if err != nil {
return err
var partCols stringSlice
if s.Partition.Expr != nil {
// Parse partitioning key, extract the column names in the partitioning key to slice.
buf := new(bytes.Buffer)
s.Partition.Expr.Format(buf)
partColumns, err := extractPartitionColumns(buf.String(), tblInfo)
if err != nil {
return err
}
partCols = columnInfoSlice(partColumns)
} else if len(s.Partition.ColumnNames) > 0 {
partCols = columnNameSlice(s.Partition.ColumnNames)
}

// Checks that the partitioning key is included in the constraint.
Expand Down Expand Up @@ -549,7 +554,7 @@ func checkPartitionKeysConstraint(pi *model.PartitionInfo, idxColNames []*ast.In

// Every unique key on the table must use every column in the table's partitioning expression.
// See https://dev.mysql.com/doc/refman/5.7/en/partitioning-limitations-partitioning-keys-unique-keys.html
if !checkUniqueKeyIncludePartKey(partCols, idxColNames) {
if !checkUniqueKeyIncludePartKey(columnInfoSlice(partCols), idxColNames) {
return ErrUniqueKeyNeedAllFieldsInPf.GenWithStackByArgs("UNIQUE INDEX")
}
return nil
Expand Down Expand Up @@ -596,16 +601,46 @@ func extractPartitionColumns(partExpr string, tblInfo *model.TableInfo) ([]*mode
return extractor.extractedColumns, nil
}

// stringSlice is defined for checkUniqueKeyIncludePartKey.
// if Go supports covariance, the code shouldn't be so complex.
type stringSlice interface {
Len() int
At(i int) string
}

// checkUniqueKeyIncludePartKey checks that the partitioning key is included in the constraint.
func checkUniqueKeyIncludePartKey(partCols []*model.ColumnInfo, idxCols []*ast.IndexColName) bool {
for _, partCol := range partCols {
func checkUniqueKeyIncludePartKey(partCols stringSlice, idxCols []*ast.IndexColName) bool {
for i := 0; i < partCols.Len(); i++ {
partCol := partCols.At(i)
if !findColumnInIndexCols(partCol, idxCols) {
return false
}
}
return true
}

// columnInfoSlice implements the stringSlice interface.
type columnInfoSlice []*model.ColumnInfo

func (cis columnInfoSlice) Len() int {
return len(cis)
}

func (cis columnInfoSlice) At(i int) string {
return cis[i].Name.L
}

// columnNameSlice implements the stringSlice interface.
type columnNameSlice []*ast.ColumnName

func (cns columnNameSlice) Len() int {
return len(cns)
}

func (cns columnNameSlice) At(i int) string {
return cns[i].Name.L
}

// isRangePartitionColUnsignedBigint returns true if the partitioning key column type is unsigned bigint type.
func isRangePartitionColUnsignedBigint(cols []*table.Column, pi *model.PartitionInfo) bool {
for _, col := range cols {
Expand Down
2 changes: 1 addition & 1 deletion expression/expression.go
Original file line number Diff line number Diff line change
Expand Up @@ -275,7 +275,7 @@ func EvaluateExprWithNull(ctx sessionctx.Context, schema *Schema, expr Expressio
for i, arg := range x.GetArgs() {
args[i] = EvaluateExprWithNull(ctx, schema, arg)
}
return NewFunctionInternal(ctx, x.FuncName.L, types.NewFieldType(mysql.TypeTiny), args...)
return NewFunctionInternal(ctx, x.FuncName.L, x.RetType, args...)
case *Column:
if !schema.Contains(x) {
return x
Expand Down
22 changes: 22 additions & 0 deletions planner/core/integration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -122,3 +122,25 @@ func (s *testIntegrationSuite) TestApplyNotNullFlag(c *C) {

tk.MustQuery("select IFNULL((select t1.x from t1 where t1.x = t2.x), 'xxx') as col1 from t2").Check(testkit.Rows("xxx"))
}

func (s *testIntegrationSuite) TestSimplifyOuterJoinWithCast(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 not null, b datetime default null)")

var input []string
var output []struct {
SQL string
Plan []string
}
s.testData.GetTestCases(c, &input, &output)
for i, tt := range input {
s.testData.OnRecord(func() {
output[i].SQL = tt
output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows())
})
tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...))
}
}
17 changes: 12 additions & 5 deletions planner/core/testdata/integration_suite_in.json
Original file line number Diff line number Diff line change
Expand Up @@ -4,12 +4,12 @@
"cases": [
// Limit should be pushed down into IndexLookUpReader, row count of IndexLookUpReader and TableScan should be 1.00.
"explain select * from tbl use index(idx_b_c) where b > 1 limit 2,1",
// Projection atop IndexLookUpReader, Limit should be pushed down into IndexLookUpReader, and Projection should have row count 1.00 as well.
// Projection atop IndexLookUpReader, Limit should be pushed down into IndexLookUpReader, and Projection should have row count 1.00 as well.
"explain select * from tbl use index(idx_b_c) where b > 1 order by b desc limit 2,1",
// Limit should be pushed down into IndexLookUpReader when Selection on top of IndexScan.
"explain select * from tbl use index(idx_b_c) where b > 1 and c > 1 limit 2,1",
// Limit should NOT be pushed down into IndexLookUpReader when Selection on top of TableScan.
"explain select * from tbl use index(idx_b_c) where b > 1 and a > 1 limit 2,1"
// Limit should be pushed down into IndexLookUpReader when Selection on top of IndexScan.
"explain select * from tbl use index(idx_b_c) where b > 1 and c > 1 limit 2,1",
// Limit should NOT be pushed down into IndexLookUpReader when Selection on top of TableScan.
"explain select * from tbl use index(idx_b_c) where b > 1 and a > 1 limit 2,1"
]
},
{
Expand All @@ -18,5 +18,12 @@
// fix #12385
"explain select * from t t1 left join t t2 on t1.a=t2.a where from_unixtime(t2.b);"
]
},
{
"name": "TestSimplifyOuterJoinWithCast",
"cases": [
// LeftOuterJoin should no be simplified to InnerJoin.
"explain select * from t t1 left join t t2 on t1.a = t2.a where cast(t1.b as date) >= '2019-01-01'"
]
}
]
16 changes: 16 additions & 0 deletions planner/core/testdata/integration_suite_out.json
Original file line number Diff line number Diff line change
Expand Up @@ -20,5 +20,21 @@
]
}
]
},
{
"Name": "TestSimplifyOuterJoinWithCast",
"Cases": [
{
"SQL": "explain select * from t t1 left join t t2 on t1.a = t2.a where cast(t1.b as date) >= '2019-01-01'",
"Plan": [
"HashLeftJoin_8 10000.00 root left outer join, inner:TableReader_13, equal:[eq(test.t1.a, test.t2.a)]",
"├─Selection_9 8000.00 root ge(cast(test.t1.b), 2019-01-01 00:00:00.000000)",
"│ └─TableReader_11 10000.00 root data:TableScan_10",
"│ └─TableScan_10 10000.00 cop table:t1, range:[-inf,+inf], keep order:false, stats:pseudo",
"└─TableReader_13 10000.00 root data:TableScan_12",
" └─TableScan_12 10000.00 cop table:t2, range:[-inf,+inf], keep order:false, stats:pseudo"
]
}
]
}
]

0 comments on commit 0c6c6aa

Please sign in to comment.