From e35594dbb90bc344a41ac7f7ac2cb95fee1989ba Mon Sep 17 00:00:00 2001 From: jwxiong Date: Mon, 9 Nov 2020 16:17:17 +0800 Subject: [PATCH 1/8] fix partition does consider collation --- ddl/db_partition_test.go | 20 ++++++++++ ddl/ddl_api.go | 9 ++++- planner/core/partition_pruning_test.go | 47 ++++++++++++++++++++++++ planner/core/rule_partition_processor.go | 8 ++-- 4 files changed, 80 insertions(+), 4 deletions(-) diff --git a/ddl/db_partition_test.go b/ddl/db_partition_test.go index 09a28bb9f94b0..aabe919eb4b8b 100644 --- a/ddl/db_partition_test.go +++ b/ddl/db_partition_test.go @@ -16,6 +16,7 @@ package ddl_test import ( "context" "fmt" + "github.com/pingcap/tidb/util/collate" "math" "math/rand" "strings" @@ -356,6 +357,8 @@ func (s *testIntegrationSuite2) TestCreateTableWithHashPartition(c *C) { } func (s *testIntegrationSuite1) TestCreateTableWithRangeColumnPartition(c *C) { + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test;") tk.MustExec("drop table if exists log_message_1;") @@ -476,6 +479,13 @@ create table log_message_1 ( "PARTITION p1 VALUES LESS THAN (20190906));", ddl.ErrWrongTypeColumnValue, }, + { + "create table t(a char(10) collate utf8mb4_bin) " + + "partition by range columns (a) (" + + "partition p0 values less than ('a'), " + + "partition p1 values less than ('G'));", + ddl.ErrRangeNotIncreasing, + }, } for i, t := range cases { _, err := tk.Exec(t.sql) @@ -492,6 +502,16 @@ create table log_message_1 ( tk.MustExec("create table t2 (a int, b char(3)) partition by range columns (b) (" + "partition p0 values less than ( 'a')," + "partition p1 values less than (maxvalue))") + + tk.MustExec("drop table if exists t;") + tk.MustExec(`create table t(a char(10) collate utf8mb4_unicode_ci) partition by range columns (a) ( + partition p0 values less than ('a'), + partition p1 values less than ('G'));`) + + tk.MustExec("drop table if exists t;") + tk.MustExec(`create table t(a int) partition by range columns (a) ( + partition p0 values less than (10), + partition p1 values less than (20));`) } func (s *testIntegrationSuite1) TestCreateTableWithListPartition(c *C) { diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 3917511e951e5..08e0d4721c49e 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -2110,7 +2110,14 @@ func checkTwoRangeColumns(ctx sessionctx.Context, curr, prev *model.PartitionDef // PARTITION p0 VALUES LESS THAN (5,10,'ggg') // PARTITION p1 VALUES LESS THAN (10,20,'mmm') // PARTITION p2 VALUES LESS THAN (15,30,'sss') - succ, err := parseAndEvalBoolExpr(ctx, fmt.Sprintf("(%s) > (%s)", curr.LessThan[i], prev.LessThan[i]), tbInfo) + var s string + colInfo := findColumnByName(pi.Columns[i].L, tbInfo) + if colInfo.EvalType() == types.ETString { + s = fmt.Sprintf("(%s) > (%s collate %s)", curr.LessThan[i], prev.LessThan[i], colInfo.Collate) + } else { + s = fmt.Sprintf("(%s) > (%s)", curr.LessThan[i], prev.LessThan[i]) + } + succ, err := parseAndEvalBoolExpr(ctx, s, tbInfo) if err != nil { return false, err } diff --git a/planner/core/partition_pruning_test.go b/planner/core/partition_pruning_test.go index 19fce4c99e9ef..1be250845ec91 100644 --- a/planner/core/partition_pruning_test.go +++ b/planner/core/partition_pruning_test.go @@ -21,6 +21,7 @@ import ( "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/mock" ) @@ -315,6 +316,52 @@ func (s *testPartitionPruningSuite) TestPartitionRangePrunner2VarChar(c *C) { } } +func (s *testPartitionPruningSuite) TestPartitionRangePrunner2CharWithCollation(c *C) { + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + tc := prepareTestCtx(c, + "create table t (a char(32) collate utf8mb4_unicode_ci)", + "a", + ) + lessThanDataInt := []string{"'c'", "'F'", "'h'", "'L'", "'t'"} + lessThan := make([]expression.Expression, len(lessThanDataInt)+1) // +1 for maxvalue + for i, str := range lessThanDataInt { + tmp, err := expression.ParseSimpleExprsWithNames(tc.sctx, str, tc.schema, tc.names) + c.Assert(err, IsNil) + lessThan[i] = tmp[0] + } + + prunner := &rangeColumnsPruner{lessThan, tc.columns[0], true} + cases := []struct { + input string + result partitionRangeOR + }{ + {"a > 'G'", partitionRangeOR{{2, 6}}}, + {"a > 'g'", partitionRangeOR{{2, 6}}}, + {"a < 'h'", partitionRangeOR{{0, 3}}}, + {"a >= 'M'", partitionRangeOR{{4, 6}}}, + {"a > 'm'", partitionRangeOR{{4, 6}}}, + {"a < 'F'", partitionRangeOR{{0, 2}}}, + {"a = 'C'", partitionRangeOR{{1, 2}}}, + {"a > 't'", partitionRangeOR{{5, 6}}}, + {"a > 'C' and a < 'q'", partitionRangeOR{{1, 5}}}, + {"a > 'c' and a < 'Q'", partitionRangeOR{{1, 5}}}, + {"a < 'l' or a >= 'W'", partitionRangeOR{{0, 4}, {5, 6}}}, + {"a is null", partitionRangeOR{{0, 1}}}, + {"'Mm' > a", partitionRangeOR{{0, 5}}}, + {"'f' <= a", partitionRangeOR{{2, 6}}}, + {"'f' >= a", partitionRangeOR{{0, 3}}}, + } + + for _, ca := range cases { + expr, err := expression.ParseSimpleExprsWithNames(tc.sctx, ca.input, tc.schema, tc.names) + c.Assert(err, IsNil) + result := fullRange(len(lessThan)) + result = partitionRangeForExpr(tc.sctx, expr[0], prunner, result) + c.Assert(equalPartitionRangeOR(ca.result, result), IsTrue, Commentf("unexpected:", ca.input)) + } +} + func (s *testPartitionPruningSuite) TestPartitionRangePrunner2Date(c *C) { tc := prepareTestCtx(c, "create table t (a date)", diff --git a/planner/core/rule_partition_processor.go b/planner/core/rule_partition_processor.go index 0d3a79ae82368..0cdeff2adfc75 100644 --- a/planner/core/rule_partition_processor.go +++ b/planner/core/rule_partition_processor.go @@ -1061,11 +1061,12 @@ func (p *rangeColumnsPruner) partitionRangeForExpr(sctx sessionctx.Context, expr return 0, len(p.data), false } - start, end := p.pruneUseBinarySearch(sctx, opName, con) + _, collation := op.CharsetAndCollation(sctx) + start, end := p.pruneUseBinarySearch(sctx, opName, con, collation) return start, end, true } -func (p *rangeColumnsPruner) pruneUseBinarySearch(sctx sessionctx.Context, op string, data *expression.Constant) (start int, end int) { +func (p *rangeColumnsPruner) pruneUseBinarySearch(sctx sessionctx.Context, op string, data *expression.Constant, collation string) (start int, end int) { var err error var isNull bool compare := func(ith int, op string, v *expression.Constant) bool { @@ -1075,7 +1076,8 @@ func (p *rangeColumnsPruner) pruneUseBinarySearch(sctx sessionctx.Context, op st } } var expr expression.Expression - expr, err = expression.NewFunction(sctx, op, types.NewFieldType(mysql.TypeLonglong), p.data[ith], v) + expr, err = expression.NewFunctionBase(sctx, op, types.NewFieldType(mysql.TypeLonglong), p.data[ith], v) + expr.SetCharsetAndCollation("", collation) var val int64 val, isNull, err = expr.EvalInt(sctx, chunk.Row{}) return val > 0 From a6b12fb3d752a65a40c81e27c9b492c3c637a20a Mon Sep 17 00:00:00 2001 From: jwxiong Date: Mon, 9 Nov 2020 17:07:46 +0800 Subject: [PATCH 2/8] make check --- ddl/db_partition_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ddl/db_partition_test.go b/ddl/db_partition_test.go index aabe919eb4b8b..55660de32f35e 100644 --- a/ddl/db_partition_test.go +++ b/ddl/db_partition_test.go @@ -16,7 +16,6 @@ package ddl_test import ( "context" "fmt" - "github.com/pingcap/tidb/util/collate" "math" "math/rand" "strings" @@ -44,6 +43,7 @@ import ( "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/admin" + "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/mock" "github.com/pingcap/tidb/util/testkit" ) From d205213847455713daa4ec5905700ef7f5fe3bfc Mon Sep 17 00:00:00 2001 From: jwxiong Date: Tue, 10 Nov 2020 16:21:49 +0800 Subject: [PATCH 3/8] serial test --- ddl/db_partition_test.go | 2 +- planner/core/partition_pruning_test.go | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/ddl/db_partition_test.go b/ddl/db_partition_test.go index 55660de32f35e..b6912194010f5 100644 --- a/ddl/db_partition_test.go +++ b/ddl/db_partition_test.go @@ -356,7 +356,7 @@ func (s *testIntegrationSuite2) TestCreateTableWithHashPartition(c *C) { tk.MustExec("create table t2 (a date, b datetime) partition by hash (EXTRACT(YEAR_MONTH FROM a)) partitions 7") } -func (s *testIntegrationSuite1) TestCreateTableWithRangeColumnPartition(c *C) { +func (s *testIntegrationSuite7) TestCreateTableWithRangeColumnPartition(c *C) { collate.SetNewCollationEnabledForTest(true) defer collate.SetNewCollationEnabledForTest(false) tk := testkit.NewTestKit(c, s.store) diff --git a/planner/core/partition_pruning_test.go b/planner/core/partition_pruning_test.go index 1be250845ec91..3e7f3fae593b5 100644 --- a/planner/core/partition_pruning_test.go +++ b/planner/core/partition_pruning_test.go @@ -25,7 +25,7 @@ import ( "github.com/pingcap/tidb/util/mock" ) -var _ = Suite(&testPartitionPruningSuite{}) +var _ = SerialSuites(&testPartitionPruningSuite{}) type testPartitionPruningSuite struct { partitionProcessor From 058dab83bf528a827ef72a834434f9ee47dc3f9b Mon Sep 17 00:00:00 2001 From: jwxiong Date: Tue, 10 Nov 2020 17:55:28 +0800 Subject: [PATCH 4/8] address comments --- ddl/ddl_api.go | 13 +++++-------- 1 file changed, 5 insertions(+), 8 deletions(-) diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 08e0d4721c49e..ed58ed5553888 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -2110,14 +2110,8 @@ func checkTwoRangeColumns(ctx sessionctx.Context, curr, prev *model.PartitionDef // PARTITION p0 VALUES LESS THAN (5,10,'ggg') // PARTITION p1 VALUES LESS THAN (10,20,'mmm') // PARTITION p2 VALUES LESS THAN (15,30,'sss') - var s string colInfo := findColumnByName(pi.Columns[i].L, tbInfo) - if colInfo.EvalType() == types.ETString { - s = fmt.Sprintf("(%s) > (%s collate %s)", curr.LessThan[i], prev.LessThan[i], colInfo.Collate) - } else { - s = fmt.Sprintf("(%s) > (%s)", curr.LessThan[i], prev.LessThan[i]) - } - succ, err := parseAndEvalBoolExpr(ctx, s, tbInfo) + succ, err := parseAndEvalBoolExpr(ctx, fmt.Sprintf("(%s) > (%s)", curr.LessThan[i], prev.LessThan[i]), tbInfo, colInfo) if err != nil { return false, err } @@ -2129,11 +2123,14 @@ func checkTwoRangeColumns(ctx sessionctx.Context, curr, prev *model.PartitionDef return false, nil } -func parseAndEvalBoolExpr(ctx sessionctx.Context, expr string, tbInfo *model.TableInfo) (bool, error) { +func parseAndEvalBoolExpr(ctx sessionctx.Context, expr string, tbInfo *model.TableInfo, colInfo *model.ColumnInfo) (bool, error) { e, err := expression.ParseSimpleExprWithTableInfo(ctx, expr, tbInfo) if err != nil { return false, err } + if colInfo.EvalType() == types.ETString { + e.SetCharsetAndCollation("", colInfo.Collate) + } res, _, err1 := e.EvalInt(ctx, chunk.Row{}) if err1 != nil { return false, err1 From 58aa4bea4b9c3ff3f0ab80dddc15e94d2bea0303 Mon Sep 17 00:00:00 2001 From: jwxiong Date: Tue, 10 Nov 2020 18:16:33 +0800 Subject: [PATCH 5/8] address comments --- ddl/db_partition_test.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/ddl/db_partition_test.go b/ddl/db_partition_test.go index b6912194010f5..a0b2716e1f592 100644 --- a/ddl/db_partition_test.go +++ b/ddl/db_partition_test.go @@ -495,10 +495,12 @@ create table log_message_1 ( )) } + tk.MustExec("drop table if exists t1;") tk.MustExec("create table t1 (a int, b char(3)) partition by range columns (a, b) (" + "partition p0 values less than (1, 'a')," + "partition p1 values less than (2, maxvalue))") + tk.MustExec("drop table if exists t2;") tk.MustExec("create table t2 (a int, b char(3)) partition by range columns (b) (" + "partition p0 values less than ( 'a')," + "partition p1 values less than (maxvalue))") From c1fe4b28b56af5c04db651336b60bc56a25e4277 Mon Sep 17 00:00:00 2001 From: jwxiong Date: Wed, 11 Nov 2020 10:40:41 +0800 Subject: [PATCH 6/8] remove unused code --- ddl/partition.go | 5 ----- 1 file changed, 5 deletions(-) diff --git a/ddl/partition.go b/ddl/partition.go index 1cee88f34aee7..2516c007c955b 100644 --- a/ddl/partition.go +++ b/ddl/partition.go @@ -327,11 +327,6 @@ func buildTablePartitionInfo(ctx sessionctx.Context, s *ast.CreateTableStmt) (*m } pi.Expr = buf.String() } else if s.Partition.ColumnNames != nil { - // TODO: Support multiple columns for 'PARTITION BY RANGE COLUMNS'. - if s.Partition.Tp == model.PartitionTypeRange && len(s.Partition.ColumnNames) != 1 { - pi.Enable = false - ctx.GetSessionVars().StmtCtx.AppendWarning(ErrUnsupportedPartitionByRangeColumns) - } pi.Columns = make([]model.CIStr, 0, len(s.Partition.ColumnNames)) for _, cn := range s.Partition.ColumnNames { pi.Columns = append(pi.Columns, cn.Name) From 83e4691a73793bf0458c8fde3e9ae7d6d3b505bb Mon Sep 17 00:00:00 2001 From: jwxiong Date: Fri, 20 Nov 2020 12:00:54 +0800 Subject: [PATCH 7/8] fix test --- ddl/ddl_api.go | 16 +++++++++++----- 1 file changed, 11 insertions(+), 5 deletions(-) diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index b4f5797c06b37..f1aafe24d615f 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -2122,7 +2122,7 @@ func checkTwoRangeColumns(ctx sessionctx.Context, curr, prev *model.PartitionDef // PARTITION p1 VALUES LESS THAN (10,20,'mmm') // PARTITION p2 VALUES LESS THAN (15,30,'sss') colInfo := findColumnByName(pi.Columns[i].L, tbInfo) - succ, err := parseAndEvalBoolExpr(ctx, fmt.Sprintf("(%s) > (%s)", curr.LessThan[i], prev.LessThan[i]), tbInfo, colInfo) + succ, err := parseAndEvalBoolExpr(ctx, curr.LessThan[i], prev.LessThan[i], colInfo.Collate, tbInfo) if err != nil { return false, err } @@ -2134,14 +2134,20 @@ func checkTwoRangeColumns(ctx sessionctx.Context, curr, prev *model.PartitionDef return false, nil } -func parseAndEvalBoolExpr(ctx sessionctx.Context, expr string, tbInfo *model.TableInfo, colInfo *model.ColumnInfo) (bool, error) { - e, err := expression.ParseSimpleExprWithTableInfo(ctx, expr, tbInfo) +func parseAndEvalBoolExpr(ctx sessionctx.Context, l, r string, coll string, tbInfo *model.TableInfo) (bool, error) { + lexpr, err := expression.ParseSimpleExprWithTableInfo(ctx, l, tbInfo) if err != nil { return false, err } - if colInfo.EvalType() == types.ETString { - e.SetCharsetAndCollation("", colInfo.Collate) + rexpr, err := expression.ParseSimpleExprWithTableInfo(ctx, r, tbInfo) + if err != nil { + return false, err + } + e, err := expression.NewFunctionBase(ctx, ast.GT, types.NewFieldType(mysql.TypeLonglong), lexpr, rexpr) + if err != nil { + return false, err } + e.SetCharsetAndCollation("", coll) res, _, err1 := e.EvalInt(ctx, chunk.Row{}) if err1 != nil { return false, err1 From cd11ad8d5b5054788c589ef397d4ad6fb3dee3ab Mon Sep 17 00:00:00 2001 From: jwxiong Date: Fri, 20 Nov 2020 18:45:17 +0800 Subject: [PATCH 8/8] address comments --- ddl/ddl_api.go | 6 +++--- planner/core/rule_partition_processor.go | 7 +++---- 2 files changed, 6 insertions(+), 7 deletions(-) diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index f1aafe24d615f..4c052655e3bb5 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -2122,7 +2122,7 @@ func checkTwoRangeColumns(ctx sessionctx.Context, curr, prev *model.PartitionDef // PARTITION p1 VALUES LESS THAN (10,20,'mmm') // PARTITION p2 VALUES LESS THAN (15,30,'sss') colInfo := findColumnByName(pi.Columns[i].L, tbInfo) - succ, err := parseAndEvalBoolExpr(ctx, curr.LessThan[i], prev.LessThan[i], colInfo.Collate, tbInfo) + succ, err := parseAndEvalBoolExpr(ctx, curr.LessThan[i], prev.LessThan[i], colInfo, tbInfo) if err != nil { return false, err } @@ -2134,7 +2134,7 @@ func checkTwoRangeColumns(ctx sessionctx.Context, curr, prev *model.PartitionDef return false, nil } -func parseAndEvalBoolExpr(ctx sessionctx.Context, l, r string, coll string, tbInfo *model.TableInfo) (bool, error) { +func parseAndEvalBoolExpr(ctx sessionctx.Context, l, r string, colInfo *model.ColumnInfo, tbInfo *model.TableInfo) (bool, error) { lexpr, err := expression.ParseSimpleExprWithTableInfo(ctx, l, tbInfo) if err != nil { return false, err @@ -2147,7 +2147,7 @@ func parseAndEvalBoolExpr(ctx sessionctx.Context, l, r string, coll string, tbIn if err != nil { return false, err } - e.SetCharsetAndCollation("", coll) + e.SetCharsetAndCollation(colInfo.Charset, colInfo.Collate) res, _, err1 := e.EvalInt(ctx, chunk.Row{}) if err1 != nil { return false, err1 diff --git a/planner/core/rule_partition_processor.go b/planner/core/rule_partition_processor.go index 418ea638925c5..bbe9d22e88d45 100644 --- a/planner/core/rule_partition_processor.go +++ b/planner/core/rule_partition_processor.go @@ -1155,12 +1155,11 @@ func (p *rangeColumnsPruner) partitionRangeForExpr(sctx sessionctx.Context, expr return 0, len(p.data), false } - _, collation := op.CharsetAndCollation(sctx) - start, end := p.pruneUseBinarySearch(sctx, opName, con, collation) + start, end := p.pruneUseBinarySearch(sctx, opName, con, op) return start, end, true } -func (p *rangeColumnsPruner) pruneUseBinarySearch(sctx sessionctx.Context, op string, data *expression.Constant, collation string) (start int, end int) { +func (p *rangeColumnsPruner) pruneUseBinarySearch(sctx sessionctx.Context, op string, data *expression.Constant, f *expression.ScalarFunction) (start int, end int) { var err error var isNull bool compare := func(ith int, op string, v *expression.Constant) bool { @@ -1171,7 +1170,7 @@ func (p *rangeColumnsPruner) pruneUseBinarySearch(sctx sessionctx.Context, op st } var expr expression.Expression expr, err = expression.NewFunctionBase(sctx, op, types.NewFieldType(mysql.TypeLonglong), p.data[ith], v) - expr.SetCharsetAndCollation("", collation) + expr.SetCharsetAndCollation(f.CharsetAndCollation(sctx)) var val int64 val, isNull, err = expr.EvalInt(sctx, chunk.Row{}) return val > 0