Skip to content

Commit

Permalink
ddl: error or skip unsupported partition-related DDLs (#10672)
Browse files Browse the repository at this point in the history
* ddl: error or skip unsupported partition-related DDLs

* go.mod: stop replacing parser since the PR is merged

* ddl: addressed comment
  • Loading branch information
kennytm committed Jul 22, 2019
1 parent 8dd4a27 commit ebcbe18
Show file tree
Hide file tree
Showing 7 changed files with 143 additions and 66 deletions.
12 changes: 6 additions & 6 deletions ddl/db_integration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1525,17 +1525,17 @@ func (s *testIntegrationSuite3) TestAlterAlgorithm(c *C) {
s.tk.MustExec("alter table t rename index idx_c to idx_c1, ALGORITHM=DEFAULT")

// partition.
s.assertAlterWarnExec(c, "alter table t truncate partition p1, ALGORITHM=COPY")
s.assertAlterErrorExec(c, "alter table t truncate partition p2, ALGORITHM=INPLACE")
s.tk.MustExec("alter table t truncate partition p3, ALGORITHM=INSTANT")
s.assertAlterWarnExec(c, "alter table t ALGORITHM=COPY, truncate partition p1")
s.assertAlterErrorExec(c, "alter table t ALGORITHM=INPLACE, truncate partition p2")
s.tk.MustExec("alter table t ALGORITHM=INSTANT, truncate partition p3")

s.assertAlterWarnExec(c, "alter table t add partition (partition p4 values less than (2002)), ALGORITHM=COPY")
s.assertAlterErrorExec(c, "alter table t add partition (partition p5 values less than (3002)), ALGORITHM=INPLACE")
s.tk.MustExec("alter table t add partition (partition p6 values less than (4002)), ALGORITHM=INSTANT")

s.assertAlterWarnExec(c, "alter table t drop partition p4, ALGORITHM=COPY")
s.assertAlterErrorExec(c, "alter table t drop partition p5, ALGORITHM=INPLACE")
s.tk.MustExec("alter table t drop partition p6, ALGORITHM=INSTANT")
s.assertAlterWarnExec(c, "alter table t ALGORITHM=COPY, drop partition p4")
s.assertAlterErrorExec(c, "alter table t ALGORITHM=INPLACE, drop partition p5")
s.tk.MustExec("alter table t ALGORITHM=INSTANT, drop partition p6")

// Table options
s.assertAlterWarnExec(c, "alter table t comment = 'test', ALGORITHM=COPY")
Expand Down
47 changes: 43 additions & 4 deletions ddl/db_partition_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ import (

. "github.com/pingcap/check"
"github.com/pingcap/errors"
"github.com/pingcap/parser/ast"
"github.com/pingcap/parser/model"
tmysql "github.com/pingcap/parser/mysql"
"github.com/pingcap/parser/terror"
Expand Down Expand Up @@ -324,14 +325,18 @@ create table log_message_1 (
cases := []testCase{
{
"create table t (id int) partition by range columns (id);",
ddl.ErrPartitionsMustBeDefined,
ast.ErrPartitionsMustBeDefined,
},
{
"create table t (id int) partition by range columns (id) (partition p0 values less than (1, 2));",
ddl.ErrPartitionColumnList,
ast.ErrPartitionColumnList,
},
{
"create table t (a int) partition by range columns (b) (partition p0 values less than (1, 2));",
ast.ErrPartitionColumnList,
},
{
"create table t (a int) partition by range columns (b) (partition p0 values less than (1));",
ddl.ErrFieldNotFoundPart,
},
{
Expand Down Expand Up @@ -371,7 +376,10 @@ create table log_message_1 (
}
for i, t := range cases {
_, err := tk.Exec(t.sql)
c.Assert(t.err.Equal(err), IsTrue, Commentf("case %d fail, sql = %s", i, t.sql))
c.Assert(t.err.Equal(err), IsTrue, Commentf(
"case %d fail, sql = `%s`\nexpected error = `%v`\n actual error = `%v`",
i, t.sql, t.err, err,
))
}

tk.MustExec("create table t1 (a int, b char(3)) partition by range columns (a, b) (" +
Expand Down Expand Up @@ -495,6 +503,15 @@ func (s *testIntegrationSuite5) TestAlterTableAddPartition(c *C) {
partition p5 values less than maxvalue
);`
assertErrorCode(c, tk, sql7, tmysql.ErrSameNamePartition)

sql8 := "alter table table3 add partition (partition p6);"
assertErrorCode(c, tk, sql8, tmysql.ErrPartitionRequiresValues)

sql9 := "alter table table3 add partition (partition p7 values in (2018));"
assertErrorCode(c, tk, sql9, tmysql.ErrPartitionWrongValues)

sql10 := "alter table table3 add partition partitions 4;"
assertErrorCode(c, tk, sql10, tmysql.ErrPartitionsMustBeDefined)
}

func (s *testIntegrationSuite6) TestAlterTableDropPartition(c *C) {
Expand Down Expand Up @@ -800,7 +817,7 @@ func (s *testIntegrationSuite6) TestTruncatePartitionAndDropTable(c *C) {
tk.MustExec("drop table if exists t5;")
tk.MustExec("set @@session.tidb_enable_table_partition=1;")
tk.MustExec(`create table t5(
id int, name varchar(50),
id int, name varchar(50),
purchased date
)
partition by range( year(purchased) ) (
Expand Down Expand Up @@ -1456,3 +1473,25 @@ func (s *testIntegrationSuite4) TestPartitionErrorCode(c *C) {
_, err = tk.Exec("alter table t_part coalesce partition 4;")
c.Assert(ddl.ErrCoalesceOnlyOnHashPartition.Equal(err), IsTrue)
}

func (s *testIntegrationSuite3) TestUnsupportedPartitionManagementDDLs(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test;")
tk.MustExec("drop table if exists test_1465;")
tk.MustExec(`
create table test_1465 (a int)
partition by range(a) (
partition p1 values less than (10),
partition p2 values less than (20),
partition p3 values less than (30)
);
`)

_, err := tk.Exec("alter table test_1465 truncate partition p1, p2")
c.Assert(err, ErrorMatches, ".*can't run multi schema change")
_, err = tk.Exec("alter table test_1465 drop partition p1, p2")
c.Assert(err, ErrorMatches, ".*can't run multi schema change")

_, err = tk.Exec("alter table test_1465 partition by hash(a)")
c.Assert(err, ErrorMatches, ".*alter table partition is unsupported")
}
24 changes: 16 additions & 8 deletions ddl/ddl.go
Original file line number Diff line number Diff line change
Expand Up @@ -194,8 +194,6 @@ var (

// ErrNotAllowedTypeInPartition returns not allowed type error when creating table partiton with unsupport expression type.
ErrNotAllowedTypeInPartition = terror.ClassDDL.New(codeErrFieldTypeNotAllowedAsPartitionField, mysql.MySQLErrName[mysql.ErrFieldTypeNotAllowedAsPartitionField])
// ErrPartitionsMustBeDefined returns 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.ClassDDL.New(codePartitionMgmtOnNonpartitioned, "Partition management on a not partitioned table is not possible")
// ErrDropPartitionNonExistent returns error in list of partition.
Expand All @@ -206,14 +204,10 @@ var (
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.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])
//ErrDropLastPartition returns cannot remove all partitions, use drop table instead.
ErrDropLastPartition = terror.ClassDDL.New(codeDropLastPartition, mysql.MySQLErrName[mysql.ErrDropLastPartition])
//ErrTooManyPartitions returns too many partitions were defined.
ErrTooManyPartitions = terror.ClassDDL.New(codeTooManyPartitions, mysql.MySQLErrName[mysql.ErrTooManyPartitions])
//ErrNoParts returns no partition were defined.
ErrNoParts = terror.ClassDDL.New(codeNoParts, mysql.MySQLErrName[mysql.ErrNoParts])
//ErrPartitionFunctionIsNotAllowed returns this partition function is not allowed.
ErrPartitionFunctionIsNotAllowed = terror.ClassDDL.New(codePartitionFunctionIsNotAllowed, mysql.MySQLErrName[mysql.ErrPartitionFunctionIsNotAllowed])
// ErrPartitionFuncNotAllowed returns partition function returns the wrong type.
Expand All @@ -235,8 +229,6 @@ var (
ErrTableCantHandleFt = terror.ClassDDL.New(codeErrTableCantHandleFt, mysql.MySQLErrName[mysql.ErrTableCantHandleFt])
// ErrFieldNotFoundPart returns an error when 'partition by columns' are not found in table columns.
ErrFieldNotFoundPart = terror.ClassDDL.New(codeFieldNotFoundPart, mysql.MySQLErrName[mysql.ErrFieldNotFoundPart])
// ErrPartitionColumnList returns "Inconsistency in usage of column lists for partitioning".
ErrPartitionColumnList = terror.ClassDDL.New(codePartitionColumnList, mysql.MySQLErrName[mysql.ErrPartitionColumnList])
)

// DDL is responsible for updating schema in data store and maintaining in-memory InfoSchema cache.
Expand Down Expand Up @@ -750,6 +742,14 @@ const (
codeFieldNotFoundPart = terror.ErrCode(mysql.ErrFieldNotFoundPart)
codePartitionColumnList = terror.ErrCode(mysql.ErrPartitionColumnList)
codeOnlyOnRangeListPartition = terror.ErrCode(mysql.ErrOnlyOnRangeListPartition)
codePartitionRequiresValues = terror.ErrCode(mysql.ErrPartitionRequiresValues)
codePartitionWrongNoPart = terror.ErrCode(mysql.ErrPartitionWrongNoPart)
codePartitionWrongNoSubpart = terror.ErrCode(mysql.ErrPartitionWrongNoSubpart)
codePartitionWrongValues = terror.ErrCode(mysql.ErrPartitionWrongValues)
codeRowSinglePartitionField = terror.ErrCode(mysql.ErrRowSinglePartitionField)
codeSubpartition = terror.ErrCode(mysql.ErrSubpartition)
codeSystemVersioningWrongPartitions = terror.ErrCode(mysql.ErrSystemVersioningWrongPartitions)
codeWrongPartitionTypeExpectedSystemTime = terror.ErrCode(mysql.ErrWrongPartitionTypeExpectedSystemTime)
)

func init() {
Expand Down Expand Up @@ -813,6 +813,14 @@ func init() {
codeInvalidDefaultValue: mysql.ErrInvalidDefault,
codeErrGeneratedColumnRefAutoInc: mysql.ErrGeneratedColumnRefAutoInc,
codeOnlyOnRangeListPartition: mysql.ErrOnlyOnRangeListPartition,
codePartitionRequiresValues: mysql.ErrPartitionRequiresValues,
codePartitionWrongNoPart: mysql.ErrPartitionWrongNoPart,
codePartitionWrongNoSubpart: mysql.ErrPartitionWrongNoSubpart,
codePartitionWrongValues: mysql.ErrPartitionWrongValues,
codeRowSinglePartitionField: mysql.ErrRowSinglePartitionField,
codeSubpartition: mysql.ErrSubpartition,
codeSystemVersioningWrongPartitions: mysql.ErrSystemVersioningWrongPartitions,
codeWrongPartitionTypeExpectedSystemTime: mysql.ErrWrongPartitionTypeExpectedSystemTime,
}
terror.ErrClassToMySQLCodes[terror.ClassDDL] = ddlMySQLErrCodes
}
71 changes: 48 additions & 23 deletions ddl/ddl_api.go
Original file line number Diff line number Diff line change
Expand Up @@ -1546,12 +1546,12 @@ func checkRangeColumnsPartitionValue(ctx sessionctx.Context, tbInfo *model.Table
// Range columns partition key supports multiple data types with integer、datetime、string.
defs := pi.Definitions
if len(defs) < 1 {
return errors.Trace(ErrPartitionsMustBeDefined)
return ast.ErrPartitionsMustBeDefined.GenWithStackByArgs("RANGE")
}

curr := &defs[0]
if len(curr.LessThan) != len(pi.Columns) {
return errors.Trace(ErrPartitionColumnList)
return errors.Trace(ast.ErrPartitionColumnList)
}
for i := 1; i < len(defs); i++ {
prev, curr := curr, &defs[i]
Expand All @@ -1568,7 +1568,7 @@ func checkRangeColumnsPartitionValue(ctx sessionctx.Context, tbInfo *model.Table

func checkTwoRangeColumns(ctx sessionctx.Context, curr, prev *model.PartitionDefinition, pi *model.PartitionInfo, tbInfo *model.TableInfo) (bool, error) {
if len(curr.LessThan) != len(pi.Columns) {
return false, errors.Trace(ErrPartitionColumnList)
return false, errors.Trace(ast.ErrPartitionColumnList)
}
for i := 0; i < len(pi.Columns); i++ {
// Special handling for MAXVALUE.
Expand Down Expand Up @@ -1775,8 +1775,7 @@ func resolveAlterTableSpec(ctx sessionctx.Context, specs []*ast.AlterTableSpec)
validSpecs = append(validSpecs, spec)
}

if len(validSpecs) != 1 {
// TODO: Hanlde len(validSpecs) == 0.
if len(validSpecs) > 1 {
// Now we only allow one schema changing at the same time.
return nil, errRunMultiSchemaChanges
}
Expand Down Expand Up @@ -1863,6 +1862,9 @@ func (d *ddl) AlterTable(ctx sessionctx.Context, ident ast.Ident, specs []*ast.A
err = ErrUnsupportedModifyPrimaryKey.GenWithStackByArgs("drop")
case ast.AlterTableRenameIndex:
err = d.RenameIndex(ctx, ident, spec)
case ast.AlterTablePartition:
// Prevent silent succeed if user executes ALTER TABLE x PARTITION BY ...
err = errors.New("alter table partition is unsupported")
case ast.AlterTableOption:
for i, opt := range spec.Options {
switch opt.Tp {
Expand Down Expand Up @@ -2089,10 +2091,6 @@ func (d *ddl) AddTablePartitions(ctx sessionctx.Context, ident ast.Ident, spec *
if meta.GetPartitionInfo() == nil {
return errors.Trace(ErrPartitionMgmtOnNonpartitioned)
}
// We don't support add hash type partition now.
if meta.Partition.Type == model.PartitionTypeHash {
return errors.Trace(ErrUnsupportedAddPartition)
}

partInfo, err := buildPartitionInfo(meta, d, spec)
if err != nil {
Expand Down Expand Up @@ -2144,20 +2142,28 @@ func (d *ddl) CoalescePartitions(ctx sessionctx.Context, ident ast.Ident, spec *
return errors.Trace(ErrPartitionMgmtOnNonpartitioned)
}

// Coalesce partition can only be used on hash/key partitions.
if meta.Partition.Type == model.PartitionTypeRange {
return errors.Trace(ErrCoalesceOnlyOnHashPartition)
}

switch meta.Partition.Type {
// We don't support coalesce partitions hash type partition now.
if meta.Partition.Type == model.PartitionTypeHash {
case model.PartitionTypeHash:
return errors.Trace(ErrUnsupportedCoalescePartition)

// Key type partition cannot be constructed currently, ignoring it for now.
case model.PartitionTypeKey:

// Coalesce partition can only be used on hash/key partitions.
default:
return errors.Trace(ErrCoalesceOnlyOnHashPartition)
}

return errors.Trace(err)
}

func (d *ddl) TruncateTablePartition(ctx sessionctx.Context, ident ast.Ident, spec *ast.AlterTableSpec) error {
// TODO: Support truncate multiple partitions
if len(spec.PartitionNames) != 1 {
return errRunMultiSchemaChanges
}

is := d.infoHandle.Get()
schema, ok := is.SchemaByName(ident.Schema)
if !ok {
Expand All @@ -2173,7 +2179,7 @@ func (d *ddl) TruncateTablePartition(ctx sessionctx.Context, ident ast.Ident, sp
}

var pid int64
pid, err = tables.FindPartitionByName(meta, spec.Name)
pid, err = tables.FindPartitionByName(meta, spec.PartitionNames[0].L)
if err != nil {
return errors.Trace(err)
}
Expand All @@ -2195,6 +2201,11 @@ func (d *ddl) TruncateTablePartition(ctx sessionctx.Context, ident ast.Ident, sp
}

func (d *ddl) DropTablePartition(ctx sessionctx.Context, ident ast.Ident, spec *ast.AlterTableSpec) error {
// TODO: Support drop multiple partitions
if len(spec.PartitionNames) != 1 {
return errRunMultiSchemaChanges
}

is := d.infoHandle.Get()
schema, ok := is.SchemaByName(ident.Schema)
if !ok {
Expand All @@ -2208,7 +2219,9 @@ func (d *ddl) DropTablePartition(ctx sessionctx.Context, ident ast.Ident, spec *
if meta.GetPartitionInfo() == nil {
return errors.Trace(ErrPartitionMgmtOnNonpartitioned)
}
err = checkDropTablePartition(meta, spec.Name)

partName := spec.PartitionNames[0].L
err = checkDropTablePartition(meta, partName)
if err != nil {
return errors.Trace(err)
}
Expand All @@ -2218,7 +2231,7 @@ func (d *ddl) DropTablePartition(ctx sessionctx.Context, ident ast.Ident, spec *
TableID: meta.ID,
Type: model.ActionDropTablePartition,
BinlogInfo: &model.HistoryInfo{},
Args: []interface{}{spec.Name},
Args: []interface{}{partName},
}

err = d.doDDLJob(ctx, job)
Expand Down Expand Up @@ -3180,9 +3193,15 @@ func validateCommentLength(vars *variable.SessionVars, comment string, maxLen in
}

func buildPartitionInfo(meta *model.TableInfo, d *ddl, spec *ast.AlterTableSpec) (*model.PartitionInfo, error) {
if meta.Partition.Type == model.PartitionTypeRange && len(spec.PartDefinitions) == 0 {
return nil, errors.Trace(ErrPartitionsMustBeDefined)
if meta.Partition.Type == model.PartitionTypeRange {
if len(spec.PartDefinitions) == 0 {
return nil, ast.ErrPartitionsMustBeDefined.GenWithStackByArgs(meta.Partition.Type)
}
} else {
// we don't support ADD PARTITION for all other partition types yet.
return nil, errors.Trace(ErrUnsupportedAddPartition)
}

part := &model.PartitionInfo{
Type: meta.Partition.Type,
Expr: meta.Partition.Expr,
Expand All @@ -3195,7 +3214,12 @@ func buildPartitionInfo(meta *model.TableInfo, d *ddl, spec *ast.AlterTableSpec)
}
buf := new(bytes.Buffer)
for ith, def := range spec.PartDefinitions {
for _, expr := range def.LessThan {
if err := def.Clause.Validate(part.Type, len(part.Columns)); err != nil {
return nil, err
}
// For RANGE partition only VALUES LESS THAN should be possible.
clause := def.Clause.(*ast.PartitionDefinitionClauseLessThan)
for _, expr := range clause.Exprs {
tp := expr.GetType().Tp
if len(part.Columns) == 0 {
// Partition by range.
Expand All @@ -3210,14 +3234,15 @@ func buildPartitionInfo(meta *model.TableInfo, d *ddl, spec *ast.AlterTableSpec)
}
// Partition by range columns if len(part.Columns) != 0.
}
comment, _ := def.Comment()
piDef := model.PartitionDefinition{
Name: def.Name,
ID: genIDs[ith],
Comment: def.Comment,
Comment: comment,
}

buf := new(bytes.Buffer)
for _, expr := range def.LessThan {
for _, expr := range clause.Exprs {
expr.Format(buf)
piDef.LessThan = append(piDef.LessThan, buf.String())
buf.Reset()
Expand Down
Loading

0 comments on commit ebcbe18

Please sign in to comment.