Skip to content

Commit

Permalink
cherry pick pingcap#20937 to release-4.0
Browse files Browse the repository at this point in the history
Signed-off-by: ti-srebot <ti-srebot@pingcap.com>
  • Loading branch information
xiongjiwei authored and ti-srebot committed Nov 21, 2020
1 parent cc61a9f commit 4b114e6
Show file tree
Hide file tree
Showing 5 changed files with 91 additions and 8 deletions.
24 changes: 23 additions & 1 deletion ddl/db_partition_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -41,6 +41,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"
)
Expand Down Expand Up @@ -335,7 +336,9 @@ func (s *testIntegrationSuite2) TestCreateTableWithHashPartition(c *C) {
tk.MustGetErrCode("select * from t_sub partition (p0)", tmysql.ErrPartitionClauseOnNonpartitioned)
}

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)
tk.MustExec("use test;")
tk.MustExec("drop table if exists log_message_1;")
Expand Down Expand Up @@ -456,6 +459,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)
Expand All @@ -465,13 +475,25 @@ 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))")

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 *testIntegrationSuite3) TestCreateTableWithKeyPartition(c *C) {
Expand Down
16 changes: 13 additions & 3 deletions ddl/ddl_api.go
Original file line number Diff line number Diff line change
Expand Up @@ -1831,7 +1831,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')
succ, err := parseAndEvalBoolExpr(ctx, fmt.Sprintf("(%s) > (%s)", curr.LessThan[i], prev.LessThan[i]), tbInfo)
colInfo := findColumnByName(pi.Columns[i].L, tbInfo)
succ, err := parseAndEvalBoolExpr(ctx, curr.LessThan[i], prev.LessThan[i], colInfo, tbInfo)
if err != nil {
return false, err
}
Expand All @@ -1843,11 +1844,20 @@ func checkTwoRangeColumns(ctx sessionctx.Context, curr, prev *model.PartitionDef
return false, nil
}

func parseAndEvalBoolExpr(ctx sessionctx.Context, expr string, tbInfo *model.TableInfo) (bool, error) {
e, err := expression.ParseSimpleExprWithTableInfo(ctx, expr, tbInfo)
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
}
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(colInfo.Charset, colInfo.Collate)
res, _, err1 := e.EvalInt(ctx, chunk.Row{})
if err1 != nil {
return false, err1
Expand Down
3 changes: 3 additions & 0 deletions ddl/partition.go
Original file line number Diff line number Diff line change
Expand Up @@ -317,11 +317,14 @@ func buildTablePartitionInfo(ctx sessionctx.Context, s *ast.CreateTableStmt) (*m
}
pi.Expr = buf.String()
} else if s.Partition.ColumnNames != nil {
<<<<<<< HEAD
// TODO: Support multiple columns for 'PARTITION BY RANGE COLUMNS'.
if len(s.Partition.ColumnNames) != 1 {
pi.Enable = false
ctx.GetSessionVars().StmtCtx.AppendWarning(ErrUnsupportedPartitionByRangeColumns)
}
=======
>>>>>>> fbaab3ecb... ddl, planner: make partition table consider collation in create table and select sentence (#20937)
pi.Columns = make([]model.CIStr, 0, len(s.Partition.ColumnNames))
for _, cn := range s.Partition.ColumnNames {
pi.Columns = append(pi.Columns, cn.Name)
Expand Down
49 changes: 48 additions & 1 deletion planner/core/partition_pruning_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,10 +21,11 @@ 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"
)

var _ = Suite(&testPartitionPruningSuite{})
var _ = SerialSuites(&testPartitionPruningSuite{})

type testPartitionPruningSuite struct {
partitionProcessor
Expand Down Expand Up @@ -314,6 +315,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)",
Expand Down
7 changes: 4 additions & 3 deletions planner/core/rule_partition_processor.go
Original file line number Diff line number Diff line change
Expand Up @@ -939,11 +939,11 @@ func (p *rangeColumnsPruner) partitionRangeForExpr(sctx sessionctx.Context, expr
return 0, len(p.data), false
}

start, end := p.pruneUseBinarySearch(sctx, opName, con)
start, end := p.pruneUseBinarySearch(sctx, opName, con, op)
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, f *expression.ScalarFunction) (start int, end int) {
var err error
var isNull bool
compare := func(ith int, op string, v *expression.Constant) bool {
Expand All @@ -953,7 +953,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(f.CharsetAndCollation(sctx))
var val int64
val, isNull, err = expr.EvalInt(sctx, chunk.Row{})
return val > 0
Expand Down

0 comments on commit 4b114e6

Please sign in to comment.