Skip to content

Commit

Permalink
ddl:create table partition value must be strictly increasing (#6912)
Browse files Browse the repository at this point in the history
* ddl:create table partition value must be strictly increasing
  • Loading branch information
ciscoxll authored Jul 9, 2018
1 parent e28a818 commit 9331a07
Show file tree
Hide file tree
Showing 5 changed files with 283 additions and 66 deletions.
16 changes: 10 additions & 6 deletions ddl/ddl.go
Original file line number Diff line number Diff line change
Expand Up @@ -157,17 +157,19 @@ var (
// ErrNotAllowedTypeInPartition returns not allowed type error when creating table partiton with unsupport expression type.
ErrNotAllowedTypeInPartition = terror.ClassDDL.New(codeCantCreateTable, "Field '%s' is of a not allowed type for this type of partitioning")
// ErrPartitionsMustBeDefined returns each partition must be defined.
ErrPartitionsMustBeDefined = terror.ClassSchema.New(codePartitionsMustBeDefined, "For RANGE partitions each partition must be defined")
ErrPartitionsMustBeDefined = terror.ClassDDL.New(codePartitionsMustBeDefined, "For RANGE partitions each partition must be defined")
// ErrPartitionMgmtOnNonpartitioned returns it's not a partition table.
ErrPartitionMgmtOnNonpartitioned = terror.ClassSchema.New(codePartitionMgmtOnNonpartitioned, "Partition management on a not partitioned table is not possible")
ErrPartitionMgmtOnNonpartitioned = terror.ClassDDL.New(codePartitionMgmtOnNonpartitioned, "Partition management on a not partitioned table is not possible")
// ErrDropPartitionNonExistent returns error in list of partition.
ErrDropPartitionNonExistent = terror.ClassSchema.New(codeDropPartitionNonExistent, " Error in list of partitions to %s")
ErrDropPartitionNonExistent = terror.ClassDDL.New(codeDropPartitionNonExistent, " Error in list of partitions to %s")
// ErrSameNamePartition returns duplicate partition name.
ErrSameNamePartition = terror.ClassSchema.New(codeSameNamePartition, "Duplicate partition name %s")
ErrSameNamePartition = terror.ClassDDL.New(codeSameNamePartition, "Duplicate partition name %s")
// ErrRangeNotIncreasing returns values less than value must be strictly increasing for each partition.
ErrRangeNotIncreasing = terror.ClassSchema.New(codeRangeNotIncreasing, "VALUES LESS THAN value must be strictly increasing for each partition")
ErrRangeNotIncreasing = terror.ClassDDL.New(codeRangeNotIncreasing, "VALUES LESS THAN value must be strictly increasing for each partition")
// ErrPartitionMaxvalue returns maxvalue can only be used in last partition definition.
ErrPartitionMaxvalue = terror.ClassSchema.New(codePartitionMaxvalue, "MAXVALUE can only be used in last partition definition")
ErrPartitionMaxvalue = terror.ClassDDL.New(codePartitionMaxvalue, "MAXVALUE can only be used in last partition definition")
// ErrTooManyValues returns cannot have more than one value for this type of partitioning.
ErrTooManyValues = terror.ClassDDL.New(codeErrTooManyValues, mysql.MySQLErrName[mysql.ErrTooManyValues])
)

// DDL is responsible for updating schema in data store and maintaining in-memory InfoSchema cache.
Expand Down Expand Up @@ -571,6 +573,7 @@ const (
codeSameNamePartition = terror.ErrCode(mysql.ErrSameNamePartition)
codeRangeNotIncreasing = terror.ErrCode(mysql.ErrRangeNotIncreasing)
codePartitionMaxvalue = terror.ErrCode(mysql.ErrPartitionMaxvalue)
codeErrTooManyValues = terror.ErrCode(mysql.ErrTooManyValues)
)

func init() {
Expand Down Expand Up @@ -609,6 +612,7 @@ func init() {
codeSameNamePartition: mysql.ErrSameNamePartition,
codeRangeNotIncreasing: mysql.ErrRangeNotIncreasing,
codePartitionMaxvalue: mysql.ErrPartitionMaxvalue,
codeErrTooManyValues: mysql.ErrTooManyValues,
}
terror.ErrClassToMySQLCodes[terror.ClassDDL] = ddlMySQLErrCodes
}
56 changes: 14 additions & 42 deletions ddl/ddl_api.go
Original file line number Diff line number Diff line change
Expand Up @@ -838,48 +838,20 @@ func (d *ddl) CreateTable(ctx sessionctx.Context, s *ast.CreateTableStmt) (err e
if err != nil {
return errors.Trace(err)
}
if s.Partition != nil {
pi := &model.PartitionInfo{
Type: s.Partition.Tp,
Enable: ctx.GetSessionVars().EnableTablePartition,
}
if s.Partition.Expr != nil {
buf := new(bytes.Buffer)
s.Partition.Expr.Format(buf)
pi.Expr = buf.String()
if s.Partition.Tp == model.PartitionTypeRange {
if _, ok := s.Partition.Expr.(*ast.ColumnNameExpr); ok {
for _, col := range cols {
name := strings.Replace(col.Name.String(), ".", "`.`", -1)
if !(col.Tp == mysql.TypeLong || col.Tp == mysql.TypeLonglong) && fmt.Sprintf("`%s`", name) == pi.Expr {
return errors.Trace(ErrNotAllowedTypeInPartition.GenByArgs(pi.Expr))
}
}
}
}
} else if s.Partition.ColumnNames != nil {
pi.Columns = make([]model.CIStr, 0, len(s.Partition.ColumnNames))
for _, cn := range s.Partition.ColumnNames {
pi.Columns = append(pi.Columns, cn.Name)
}

pi, err := buildTablePartitionInfo(ctx, d, s, cols)
if err != nil {
return errors.Trace(err)
}

if pi != nil {
err = checkPartitionNameUnique(tbInfo, pi)
if err != nil {
return errors.Trace(err)
}
for _, def := range s.Partition.Definitions {
// TODO: generate multiple global ID for paritions.
pid, err1 := d.genGlobalID()
if err1 != nil {
return errors.Trace(err1)
}
piDef := model.PartitionDefinition{
Name: def.Name,
ID: pid,
Comment: def.Comment,
}
for _, expr := range def.LessThan {
buf := new(bytes.Buffer)
expr.Format(buf)
piDef.LessThan = append(piDef.LessThan, buf.String())
}
pi.Definitions = append(pi.Definitions, piDef)
err = checkCreatePartitionValue(pi)
if err != nil {
return errors.Trace(err)
}
tbInfo.Partition = pi
}
Expand Down Expand Up @@ -1253,7 +1225,7 @@ func (d *ddl) AddTablePartitions(ctx sessionctx.Context, ident ast.Ident, spec *
return errors.Trace(err)
}

err = checkPartitionNotExists(meta, partInfo)
err = checkPartitionNameUnique(meta, partInfo)
if err != nil {
return errors.Trace(err)
}
Expand Down
96 changes: 94 additions & 2 deletions ddl/ddl_db_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1554,12 +1554,14 @@ func (s *testDBSuite) TestCreateTable(c *C) {
}

func (s *testDBSuite) TestCreateTableWithPartition(c *C) {
s.tk.MustExec("use test")
s.tk = testkit.NewTestKit(c, s.store)
s.tk.MustExec("use test;")
s.tk.MustExec("drop table if exists tp;")
s.tk.MustExec(`CREATE TABLE tp (a int) PARTITION BY RANGE(a) (
PARTITION p0 VALUES LESS THAN (10),
PARTITION p1 VALUES LESS THAN (20),
PARTITION p2 VALUES LESS THAN (MAXVALUE)
);`)
);`)
ctx := s.tk.Se.(sessionctx.Context)
is := domain.GetDomain(ctx).InfoSchema()
tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("tp"))
Expand All @@ -1578,6 +1580,96 @@ func (s *testDBSuite) TestCreateTableWithPartition(c *C) {
c.Assert(part.Definitions[1].Name, Equals, "p1")
c.Assert(part.Definitions[2].LessThan[0], Equals, "MAXVALUE")
c.Assert(part.Definitions[2].Name, Equals, "p2")

s.tk.MustExec("drop table if exists employees;")
sql1 := `create table employees (
id int not null,
hired int not null
)
partition by range( hired ) (
partition p1 values less than (1991),
partition p2 values less than (1996),
partition p2 values less than (2001)
);`
s.testErrorCode(c, sql1, tmysql.ErrSameNamePartition)

sql2 := `create table employees (
id int not null,
hired int not null
)
partition by range( hired ) (
partition p1 values less than (1998),
partition p2 values less than (1996),
partition p3 values less than (2001)
);`
s.testErrorCode(c, sql2, tmysql.ErrRangeNotIncreasing)

sql3 := `create table employees (
id int not null,
hired int not null
)
partition by range( hired ) (
partition p1 values less than (1998),
partition p2 values less than maxvalue,
partition p3 values less than (2001)
);`
s.testErrorCode(c, sql3, tmysql.ErrPartitionMaxvalue)

sql4 := `create table t4 (
a int not null,
b int not null
)
partition by range( id ) (
partition p1 values less than maxvalue,
partition p2 values less than (1991),
partition p3 values less than (1995)
);`
s.testErrorCode(c, sql4, tmysql.ErrPartitionMaxvalue)

_, err = s.tk.Exec(`CREATE TABLE rc (
a INT NOT NULL,
b INT NOT NULL,
c INT NOT NULL
)
partition by range columns(a,b,c) (
partition p0 values less than (10,5,1),
partition p2 values less than (50,maxvalue,10),
partition p3 values less than (65,30,13),
partition p4 values less than (maxvalue,30,40)
);`)
c.Assert(err, IsNil)

sql6 := `create table employees (
id int not null,
hired int not null
)
partition by range( hired ) (
partition p0 values less than (6 , 10)
);`
s.testErrorCode(c, sql6, tmysql.ErrTooManyValues)

sql7 := `create table t7 (
a int not null,
b int not null
)
partition by range( id ) (
partition p1 values less than (1991),
partition p2 values less than maxvalue,
partition p3 values less than maxvalue,
partition p4 values less than (1995),
partition p5 values less than maxvalue
);`
s.testErrorCode(c, sql7, tmysql.ErrPartitionMaxvalue)

_, err = s.tk.Exec(`create table t8 (
a int not null,
b int not null
)
partition by range( id ) (
partition p1 values less than (19xx91),
partition p2 values less than maxvalue
);`)
c.Assert(ddl.ErrNotAllowedTypeInPartition.Equal(err), IsTrue)
}

func (s *testDBSuite) TestTableDDLWithFloatType(c *C) {
Expand Down
156 changes: 156 additions & 0 deletions ddl/partition.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,156 @@
// Copyright 2018 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 ddl

import (
"bytes"
"fmt"
"strconv"
"strings"

"github.com/juju/errors"
"github.com/pingcap/tidb/ast"
"github.com/pingcap/tidb/model"
"github.com/pingcap/tidb/mysql"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/table"
)

const (
partitionMaxValue = "MAXVALUE"
)

// buildTablePartitionInfo builds partition info and checks for some errors.
func buildTablePartitionInfo(ctx sessionctx.Context, d *ddl, s *ast.CreateTableStmt, cols []*table.Column) (*model.PartitionInfo, error) {
if s.Partition == nil {
return nil, nil
}
pi := &model.PartitionInfo{
Type: s.Partition.Tp,
Enable: ctx.GetSessionVars().EnableTablePartition,
}
if s.Partition.Expr != nil {
buf := new(bytes.Buffer)
s.Partition.Expr.Format(buf)
pi.Expr = buf.String()
if s.Partition.Tp == model.PartitionTypeRange {
for _, col := range cols {
name := strings.Replace(col.Name.String(), ".", "`.`", -1)
if _, ok := s.Partition.Expr.(*ast.ColumnNameExpr); ok {
// TODO: check that the expression returns an integer.
}
if _, ok := s.Partition.Expr.(ast.ExprNode); ok {
// Range partitioning key supported types: tinyint, smallint, mediumint, int and bigint.
if !validRangePartitionType(col) && fmt.Sprintf("`%s`", name) == pi.Expr {
return nil, errors.Trace(ErrNotAllowedTypeInPartition.GenByArgs(pi.Expr))
}
}
}
}
} else if s.Partition.ColumnNames != nil {
pi.Columns = make([]model.CIStr, 0, len(s.Partition.ColumnNames))
for _, cn := range s.Partition.ColumnNames {
pi.Columns = append(pi.Columns, cn.Name)
}
}
for _, def := range s.Partition.Definitions {
// TODO: generate multiple global ID for paritions, reduce the times of obtaining the global ID from the storage.
pid, err := d.genGlobalID()
if err != nil {
return nil, errors.Trace(err)
}
piDef := model.PartitionDefinition{
Name: def.Name,
ID: pid,
Comment: def.Comment,
}

if s.Partition.Tp == model.PartitionTypeRange {
if s.Partition.ColumnNames == nil && len(def.LessThan) != 1 {
return nil, ErrTooManyValues.GenByArgs(s.Partition.Tp.String())
}
buf := new(bytes.Buffer)
// Range columns partitions support multi-column partitions.
for _, expr := range def.LessThan {
expr.Format(buf)
piDef.LessThan = append(piDef.LessThan, buf.String())
buf.Reset()
}
pi.Definitions = append(pi.Definitions, piDef)
}
}
return pi, nil
}

func checkPartitionNameUnique(tbInfo *model.TableInfo, pi *model.PartitionInfo) error {
partNames := make(map[string]struct{})
if tbInfo.Partition != nil {
oldPars := tbInfo.Partition.Definitions
for _, oldPar := range oldPars {
partNames[strings.ToLower(oldPar.Name)] = struct{}{}
}
}
newPars := pi.Definitions
for _, newPar := range newPars {
if _, ok := partNames[strings.ToLower(newPar.Name)]; ok {
return ErrSameNamePartition.GenByArgs(newPar.Name)
}
partNames[strings.ToLower(newPar.Name)] = struct{}{}
}
return nil
}

// checkCreatePartitionValue checks whether `less than value` is strictly increasing for each partition.
func checkCreatePartitionValue(pi *model.PartitionInfo) error {
defs := pi.Definitions
if len(defs) <= 1 {
return nil
}

if strings.EqualFold(defs[len(defs)-1].LessThan[0], partitionMaxValue) {
defs = defs[:len(defs)-1]
}
var prevRangeValue int
for i := 0; i < len(defs); i++ {
if strings.EqualFold(defs[i].LessThan[0], partitionMaxValue) {
return errors.Trace(ErrPartitionMaxvalue)
}

currentRangeValue, err := strconv.Atoi(defs[i].LessThan[0])
if err != nil {
return ErrNotAllowedTypeInPartition.GenByArgs(defs[i].LessThan[0])
}

if i == 0 {
prevRangeValue = currentRangeValue
continue
}

if currentRangeValue <= prevRangeValue {
return errors.Trace(ErrRangeNotIncreasing)
}
prevRangeValue = currentRangeValue
}
return nil
}

// validRangePartitionType checks the type supported by the range partitioning key.
func validRangePartitionType(col *table.Column) bool {
switch col.Tp {
case mysql.TypeTiny, mysql.TypeShort, mysql.TypeInt24, mysql.TypeLong, mysql.TypeLonglong:
return true
default:
return false
}
}
Loading

0 comments on commit 9331a07

Please sign in to comment.