Skip to content

Commit

Permalink
ddl: Fix for unsigned partitioning expressions (#36830)
Browse files Browse the repository at this point in the history
close #35827
  • Loading branch information
mjonss authored Aug 18, 2022
1 parent ad0f3dd commit a1d1356
Show file tree
Hide file tree
Showing 3 changed files with 27 additions and 27 deletions.
4 changes: 4 additions & 0 deletions ddl/db_partition_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -333,6 +333,10 @@ partition by range (a)
partition p0 values less than (200),
partition p1 values less than (300),
partition p2 values less than maxvalue)`)

// Fix https://github.com/pingcap/tidb/issues/35827
tk.MustExec(`create table t37 (id tinyint unsigned, idpart tinyint, i varchar(255)) partition by range (idpart) (partition p1 values less than (-1));`)
tk.MustGetErrCode(`create table t38 (id tinyint unsigned, idpart tinyint unsigned, i varchar(255)) partition by range (idpart) (partition p1 values less than (-1));`, errno.ErrPartitionConstDomain)
}

func TestCreateTableWithHashPartition(t *testing.T) {
Expand Down
44 changes: 17 additions & 27 deletions ddl/partition.go
Original file line number Diff line number Diff line change
Expand Up @@ -53,6 +53,7 @@ import (
"github.com/pingcap/tidb/util/hack"
"github.com/pingcap/tidb/util/logutil"
"github.com/pingcap/tidb/util/mathutil"
"github.com/pingcap/tidb/util/mock"
"github.com/pingcap/tidb/util/slice"
"github.com/pingcap/tidb/util/sqlexec"
"github.com/pingcap/tidb/util/stringutil"
Expand Down Expand Up @@ -504,20 +505,6 @@ func buildTablePartitionInfo(ctx sessionctx.Context, s *ast.PartitionOptions, tb
return nil
}

func isPartExprUnsigned(ctx sessionctx.Context, tbInfo *model.TableInfo) bool {
expr, err := expression.ParseSimpleExprWithTableInfo(ctx, tbInfo.Partition.Expr, tbInfo)
if err != nil {
return false
}
pCols := expression.ExtractColumns(expr)
for _, col := range pCols {
if mysql.HasUnsignedFlag(col.GetType().GetFlag()) {
return true
}
}
return false
}

// getPartitionIntervalFromTable checks if a partitioned table matches a generated INTERVAL partitioned scheme
// will return nil if error occurs, i.e. not an INTERVAL partitioned table
func getPartitionIntervalFromTable(ctx sessionctx.Context, tbInfo *model.TableInfo) *ast.PartitionInterval {
Expand Down Expand Up @@ -554,7 +541,7 @@ func getPartitionIntervalFromTable(ctx sessionctx.Context, tbInfo *model.TableIn
return nil
}
} else {
if !isPartExprUnsigned(ctx, tbInfo) {
if !isPartExprUnsigned(tbInfo) {
minVal = "-9223372036854775808"
}
}
Expand Down Expand Up @@ -828,7 +815,7 @@ func generatePartitionDefinitionsFromInterval(ctx sessionctx.Context, partOption
if partCol != nil {
min = getLowerBoundInt(partCol)
} else {
if !isPartExprUnsigned(ctx, tbInfo) {
if !isPartExprUnsigned(tbInfo) {
min = math.MinInt64
}
}
Expand Down Expand Up @@ -1217,7 +1204,7 @@ func buildRangePartitionDefinitions(ctx sessionctx.Context, defs []*ast.Partitio

func checkPartitionValuesIsInt(ctx sessionctx.Context, defName interface{}, exprs []ast.ExprNode, tbInfo *model.TableInfo) error {
tp := types.NewFieldType(mysql.TypeLonglong)
if isColUnsigned(tbInfo.Columns, tbInfo.Partition) {
if isPartExprUnsigned(tbInfo) {
tp.AddFlag(mysql.UnsignedFlag)
}
for _, exp := range exprs {
Expand Down Expand Up @@ -1369,11 +1356,10 @@ func checkRangePartitionValue(ctx sessionctx.Context, tblInfo *model.TableInfo)
return nil
}

cols := tblInfo.Columns
if strings.EqualFold(defs[len(defs)-1].LessThan[0], partitionMaxValue) {
defs = defs[:len(defs)-1]
}
isUnsigned := isColUnsigned(cols, pi)
isUnsigned := isPartExprUnsigned(tblInfo)
var prevRangeValue interface{}
for i := 0; i < len(defs); i++ {
if strings.EqualFold(defs[i].LessThan[0], partitionMaxValue) {
Expand Down Expand Up @@ -1436,7 +1422,7 @@ func formatListPartitionValue(ctx sessionctx.Context, tblInfo *model.TableInfo)
cols := make([]*model.ColumnInfo, 0, len(pi.Columns))
if len(pi.Columns) == 0 {
tp := types.NewFieldType(mysql.TypeLonglong)
if isColUnsigned(tblInfo.Columns, tblInfo.Partition) {
if isPartExprUnsigned(tblInfo) {
tp.AddFlag(mysql.UnsignedFlag)
}
colTps = []*types.FieldType{tp}
Expand Down Expand Up @@ -2555,13 +2541,17 @@ func (cns columnNameSlice) At(i int) string {
return cns[i].Name.L
}

// isColUnsigned returns true if the partitioning key column is unsigned.
func isColUnsigned(cols []*model.ColumnInfo, pi *model.PartitionInfo) bool {
for _, col := range cols {
isUnsigned := mysql.HasUnsignedFlag(col.GetFlag())
if isUnsigned && strings.Contains(strings.ToLower(pi.Expr), col.Name.L) {
return true
}
func isPartExprUnsigned(tbInfo *model.TableInfo) bool {
// We should not rely on any configuration, system or session variables, so use a mock ctx!
// Same as in tables.newPartitionExpr
ctx := mock.NewContext()
expr, err := expression.ParseSimpleExprWithTableInfo(ctx, tbInfo.Partition.Expr, tbInfo)
if err != nil {
logutil.BgLogger().Error("isPartExpr failed parsing expression!", zap.Error(err))
return false
}
if mysql.HasUnsignedFlag(expr.GetType().GetFlag()) {
return true
}
return false
}
Expand Down
6 changes: 6 additions & 0 deletions table/tables/partition_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -512,6 +512,12 @@ func TestRangePartitionUnderNoUnsigned(t *testing.T) {
tk.MustExec("drop table if exists tu;")
defer tk.MustExec("drop table if exists t2;")
defer tk.MustExec("drop table if exists tu;")
tk.MustGetErrCode(`CREATE TABLE tu (c1 BIGINT UNSIGNED) PARTITION BY RANGE(c1 - 10) (
PARTITION p0 VALUES LESS THAN (-5),
PARTITION p1 VALUES LESS THAN (0),
PARTITION p2 VALUES LESS THAN (5),
PARTITION p3 VALUES LESS THAN (10),
PARTITION p4 VALUES LESS THAN (MAXVALUE));`, mysql.ErrPartitionConstDomain)
tk.MustExec("SET @@sql_mode='NO_UNSIGNED_SUBTRACTION';")
tk.MustExec(`create table t2 (a bigint unsigned) partition by range (a) (
partition p1 values less than (0),
Expand Down

0 comments on commit a1d1356

Please sign in to comment.