From f42c0391d2ef5a24c81d9cf0b57a4d42d2a43f09 Mon Sep 17 00:00:00 2001 From: kennytm Date: Sun, 2 Jun 2019 03:02:41 +0800 Subject: [PATCH 1/3] ddl: error or skip unsupported partition-related DDLs --- ddl/db_integration_test.go | 12 +++---- ddl/db_partition_test.go | 47 +++++++++++++++++++++++++--- ddl/ddl.go | 24 +++++++++----- ddl/ddl_api.go | 64 ++++++++++++++++++++++++++------------ ddl/partition.go | 47 +++++++++++++++------------- go.mod | 2 ++ go.sum | 4 +-- 7 files changed, 138 insertions(+), 62 deletions(-) diff --git a/ddl/db_integration_test.go b/ddl/db_integration_test.go index 49755bcac84e9..d2229951addc6 100644 --- a/ddl/db_integration_test.go +++ b/ddl/db_integration_test.go @@ -1444,17 +1444,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") diff --git a/ddl/db_partition_test.go b/ddl/db_partition_test.go index df7f20805339b..27104cb77403b 100644 --- a/ddl/db_partition_test.go +++ b/ddl/db_partition_test.go @@ -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" @@ -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, }, { @@ -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) (" + @@ -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 *testIntegrationSuite5) TestAlterTableDropPartition(c *C) { @@ -797,7 +814,7 @@ func (s *testIntegrationSuite5) 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) ) ( @@ -1453,3 +1470,25 @@ func (s *testIntegrationSuite3) 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") +} diff --git a/ddl/ddl.go b/ddl/ddl.go index 29c74bd972ff8..f736ad0979ed3 100644 --- a/ddl/ddl.go +++ b/ddl/ddl.go @@ -192,8 +192,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. @@ -204,14 +202,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. @@ -233,8 +227,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. @@ -731,6 +723,14 @@ const ( codeNotSupportedAlterOperation = terror.ErrCode(mysql.ErrAlterOperationNotSupportedReason) codeFieldNotFoundPart = terror.ErrCode(mysql.ErrFieldNotFoundPart) codePartitionColumnList = terror.ErrCode(mysql.ErrPartitionColumnList) + 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() { @@ -793,6 +793,14 @@ func init() { codePartitionColumnList: mysql.ErrPartitionColumnList, codeInvalidDefaultValue: mysql.ErrInvalidDefault, codeErrGeneratedColumnRefAutoInc: mysql.ErrGeneratedColumnRefAutoInc, + 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 } diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index f94454c93d24b..98a79cf765884 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -1517,12 +1517,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] @@ -1539,7 +1539,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. @@ -1747,8 +1747,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 } @@ -1835,6 +1834,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 { @@ -2060,10 +2062,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 { @@ -2115,20 +2113,27 @@ func (d *ddl) CoalescePartitions(ctx sessionctx.Context, ident ast.Ident, spec * return errors.Trace(ErrPartitionMgmtOnNonpartitioned) } + switch meta.Partition.Type { // Coalesce partition can only be used on hash/key partitions. - if meta.Partition.Type == model.PartitionTypeRange { + default: return errors.Trace(ErrCoalesceOnlyOnHashPartition) - } // We don't support coalesce partitions hash type partition now. - if meta.Partition.Type == model.PartitionTypeHash { + case model.PartitionTypeHash: return errors.Trace(ErrUnsupportedCoalescePartition) + + case model.PartitionTypeKey: } 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 { @@ -2144,7 +2149,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) } @@ -2166,6 +2171,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 { @@ -2179,7 +2189,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) } @@ -2189,7 +2201,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) @@ -3147,9 +3159,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, @@ -3158,7 +3176,12 @@ func buildPartitionInfo(meta *model.TableInfo, d *ddl, spec *ast.AlterTableSpec) } buf := new(bytes.Buffer) for _, def := range spec.PartDefinitions { - for _, expr := range def.LessThan { + if err := def.Clause.Validate(part.Type, len(part.Columns)); err != nil { + return nil, errors.Trace(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. @@ -3177,14 +3200,15 @@ func buildPartitionInfo(meta *model.TableInfo, d *ddl, spec *ast.AlterTableSpec) if err1 != nil { return nil, errors.Trace(err1) } + comment, _ := def.Comment() piDef := model.PartitionDefinition{ Name: def.Name, ID: pid, - 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() diff --git a/ddl/partition.go b/ddl/partition.go index 4826057be4bbb..4dc1dced8a060 100644 --- a/ddl/partition.go +++ b/ddl/partition.go @@ -42,6 +42,17 @@ func buildTablePartitionInfo(ctx sessionctx.Context, d *ddl, s *ast.CreateTableS if s.Partition == nil { return nil, nil } + + // force-discard the unsupported types, even when @@tidb_enable_table_partition = 'on' + switch s.Partition.Tp { + case model.PartitionTypeKey: + // can't create a warning for KEY partition, it will fail an integration test :/ + return nil, nil + case model.PartitionTypeList, model.PartitionTypeSystemTime: + ctx.GetSessionVars().StmtCtx.AppendWarning(errUnsupportedCreatePartition) + return nil, nil + } + var enable bool switch ctx.GetSessionVars().EnableTablePartition { case "on": @@ -112,7 +123,13 @@ func buildHashPartitionDefinitions(ctx sessionctx.Context, d *ddl, s *ast.Create return errors.Trace(err) } defs[i].ID = pid - defs[i].Name = model.NewCIStr(fmt.Sprintf("p%v", i)) + if len(s.Partition.Definitions) == 0 { + defs[i].Name = model.NewCIStr(fmt.Sprintf("p%v", i)) + } else { + def := s.Partition.Definitions[i] + defs[i].Name = def.Name + defs[i].Comment, _ = def.Comment() + } } pi.Definitions = defs return nil @@ -124,18 +141,16 @@ func buildRangePartitionDefinitions(ctx sessionctx.Context, d *ddl, s *ast.Creat if err != nil { return errors.Trace(err) } + comment, _ := def.Comment() piDef := model.PartitionDefinition{ Name: def.Name, ID: pid, - Comment: def.Comment, + Comment: comment, } - if s.Partition.ColumnNames == nil && len(def.LessThan) != 1 { - return ErrTooManyValues.GenWithStackByArgs(s.Partition.Tp.String()) - } buf := new(bytes.Buffer) // Range columns partitions support multi-column partitions. - for _, expr := range def.LessThan { + for _, expr := range def.Clause.(*ast.PartitionDefinitionClauseLessThan).Exprs { expr.Format(buf) piDef.LessThan = append(piDef.LessThan, buf.String()) buf.Reset() @@ -447,14 +462,14 @@ func checkAddPartitionTooManyPartitions(piDefs uint64) error { func checkNoHashPartitions(ctx sessionctx.Context, partitionNum uint64) error { if partitionNum == 0 { - return ErrNoParts.GenWithStackByArgs("partitions") + return ast.ErrNoParts.GenWithStackByArgs("partitions") } return nil } func checkNoRangePartitions(partitionNum int) error { if partitionNum == 0 { - return errors.Trace(ErrPartitionsMustBeDefined) + return ast.ErrPartitionsMustBeDefined.GenWithStackByArgs("RANGE") } return nil } @@ -555,20 +570,8 @@ func truncateTableByReassignPartitionIDs(t *meta.Meta, tblInfo *model.TableInfo) if err != nil { return errors.Trace(err) } - - var newDef model.PartitionDefinition - if tblInfo.Partition.Type == model.PartitionTypeHash { - newDef = model.PartitionDefinition{ - ID: pid, - } - } else if tblInfo.Partition.Type == model.PartitionTypeRange { - newDef = model.PartitionDefinition{ - ID: pid, - Name: def.Name, - LessThan: def.LessThan, - Comment: def.Comment, - } - } + newDef := def + newDef.ID = pid newDefs = append(newDefs, newDef) } tblInfo.Partition.Definitions = newDefs diff --git a/go.mod b/go.mod index 4586530b0b537..209357212e7a7 100644 --- a/go.mod +++ b/go.mod @@ -73,3 +73,5 @@ require ( sourcegraph.com/sourcegraph/appdash v0.0.0-20180531100431-4c381bd170b4 sourcegraph.com/sourcegraph/appdash-data v0.0.0-20151005221446-73f23eafcf67 ) + +replace github.com/pingcap/parser => github.com/kennytm/parser v0.0.0-20190602070424-b90c5fc67c0c diff --git a/go.sum b/go.sum index f351d80b7dfa1..a38ea37aa5513 100644 --- a/go.sum +++ b/go.sum @@ -100,6 +100,8 @@ github.com/jonboulle/clockwork v0.1.0 h1:VKV+ZcuP6l3yW9doeqz6ziZGgcynBVQO+obU0+0 github.com/jonboulle/clockwork v0.1.0/go.mod h1:Ii8DK3G1RaLaWxj9trq07+26W01tbo22gdxWY5EU2bo= github.com/juju/ratelimit v1.0.1 h1:+7AIFJVQ0EQgq/K9+0Krm7m530Du7tIz0METWzN0RgY= github.com/juju/ratelimit v1.0.1/go.mod h1:qapgC/Gy+xNh9UxzV13HGGl/6UXNN+ct+vwSgWNm/qk= +github.com/kennytm/parser v0.0.0-20190602070424-b90c5fc67c0c h1:rZZtnBel43d3+mquaOOtYTspODun0enEi8z/7zylLKU= +github.com/kennytm/parser v0.0.0-20190602070424-b90c5fc67c0c/go.mod h1:JYKo7zIr56SCa1kNeIslpEFHTwAVbXhsjGbwy84pfF0= github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= github.com/klauspost/cpuid v0.0.0-20170728055534-ae7887de9fa5 h1:2U0HzY8BJ8hVwDKIzp7y4voR9CX/nvcfymLmg2UiOio= github.com/klauspost/cpuid v0.0.0-20170728055534-ae7887de9fa5/go.mod h1:Pj4uuM528wm8OyEC2QMXAi2YiTZ96dNQPGgoMS4s3ek= @@ -160,8 +162,6 @@ github.com/pingcap/kvproto v0.0.0-20190528074401-b942b3f4108f/go.mod h1:QMdbTAXC github.com/pingcap/log v0.0.0-20190214045112-b37da76f67a7/go.mod h1:xsfkWVaFVV5B8e1K9seWfyJWFrIhbtUTAD8NV1Pq3+w= github.com/pingcap/log v0.0.0-20190307075452-bd41d9273596 h1:t2OQTpPJnrPDGlvA+3FwJptMTt6MEPdzK1Wt99oaefQ= github.com/pingcap/log v0.0.0-20190307075452-bd41d9273596/go.mod h1:WpHUKhNZ18v116SvGrmjkA9CBhYmuUTKL+p8JC9ANEw= -github.com/pingcap/parser v0.0.0-20190529103304-95494e4022ce h1:qpX1uGhjKSWCNiY1vy0/stirh/ud1nF7Hi3dZzFSuio= -github.com/pingcap/parser v0.0.0-20190529103304-95494e4022ce/go.mod h1:1FNvfp9+J0wvc4kl8eGNh7Rqrxveg15jJoWo/a0uHwA= github.com/pingcap/pd v0.0.0-20190424024702-bd1e2496a669 h1:ZoKjndm/Ig7Ru/wojrQkc/YLUttUdQXoH77gtuWCvL4= github.com/pingcap/pd v0.0.0-20190424024702-bd1e2496a669/go.mod h1:MUCxRzOkYiWZtlyi4MhxjCIj9PgQQ/j+BLNGm7aUsnM= github.com/pingcap/tidb-tools v2.1.3-0.20190321065848-1e8b48f5c168+incompatible h1:MkWCxgZpJBgY2f4HtwWMMFzSBb3+JPzeJgF3VrXE/bU= From 1370a58efc152bda4b574acaf73420117add2b2d Mon Sep 17 00:00:00 2001 From: kennytm Date: Tue, 11 Jun 2019 17:14:40 +0800 Subject: [PATCH 2/3] go.mod: stop replacing parser since the PR is merged --- go.mod | 4 +--- go.sum | 4 ++-- 2 files changed, 3 insertions(+), 5 deletions(-) diff --git a/go.mod b/go.mod index 209357212e7a7..0d091731c3848 100644 --- a/go.mod +++ b/go.mod @@ -39,7 +39,7 @@ require ( github.com/pingcap/goleveldb v0.0.0-20171020122428-b9ff6c35079e github.com/pingcap/kvproto v0.0.0-20190528074401-b942b3f4108f github.com/pingcap/log v0.0.0-20190307075452-bd41d9273596 - github.com/pingcap/parser v0.0.0-20190529103304-95494e4022ce + github.com/pingcap/parser v0.0.0-20190611090107-2bb2c0e8f340 github.com/pingcap/pd v0.0.0-20190424024702-bd1e2496a669 github.com/pingcap/tidb-tools v2.1.3-0.20190321065848-1e8b48f5c168+incompatible github.com/pingcap/tipb v0.0.0-20190428032612-535e1abaa330 @@ -73,5 +73,3 @@ require ( sourcegraph.com/sourcegraph/appdash v0.0.0-20180531100431-4c381bd170b4 sourcegraph.com/sourcegraph/appdash-data v0.0.0-20151005221446-73f23eafcf67 ) - -replace github.com/pingcap/parser => github.com/kennytm/parser v0.0.0-20190602070424-b90c5fc67c0c diff --git a/go.sum b/go.sum index a38ea37aa5513..7eed1012d267d 100644 --- a/go.sum +++ b/go.sum @@ -100,8 +100,6 @@ github.com/jonboulle/clockwork v0.1.0 h1:VKV+ZcuP6l3yW9doeqz6ziZGgcynBVQO+obU0+0 github.com/jonboulle/clockwork v0.1.0/go.mod h1:Ii8DK3G1RaLaWxj9trq07+26W01tbo22gdxWY5EU2bo= github.com/juju/ratelimit v1.0.1 h1:+7AIFJVQ0EQgq/K9+0Krm7m530Du7tIz0METWzN0RgY= github.com/juju/ratelimit v1.0.1/go.mod h1:qapgC/Gy+xNh9UxzV13HGGl/6UXNN+ct+vwSgWNm/qk= -github.com/kennytm/parser v0.0.0-20190602070424-b90c5fc67c0c h1:rZZtnBel43d3+mquaOOtYTspODun0enEi8z/7zylLKU= -github.com/kennytm/parser v0.0.0-20190602070424-b90c5fc67c0c/go.mod h1:JYKo7zIr56SCa1kNeIslpEFHTwAVbXhsjGbwy84pfF0= github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= github.com/klauspost/cpuid v0.0.0-20170728055534-ae7887de9fa5 h1:2U0HzY8BJ8hVwDKIzp7y4voR9CX/nvcfymLmg2UiOio= github.com/klauspost/cpuid v0.0.0-20170728055534-ae7887de9fa5/go.mod h1:Pj4uuM528wm8OyEC2QMXAi2YiTZ96dNQPGgoMS4s3ek= @@ -162,6 +160,8 @@ github.com/pingcap/kvproto v0.0.0-20190528074401-b942b3f4108f/go.mod h1:QMdbTAXC github.com/pingcap/log v0.0.0-20190214045112-b37da76f67a7/go.mod h1:xsfkWVaFVV5B8e1K9seWfyJWFrIhbtUTAD8NV1Pq3+w= github.com/pingcap/log v0.0.0-20190307075452-bd41d9273596 h1:t2OQTpPJnrPDGlvA+3FwJptMTt6MEPdzK1Wt99oaefQ= github.com/pingcap/log v0.0.0-20190307075452-bd41d9273596/go.mod h1:WpHUKhNZ18v116SvGrmjkA9CBhYmuUTKL+p8JC9ANEw= +github.com/pingcap/parser v0.0.0-20190611090107-2bb2c0e8f340 h1:WZGc0+xpKCnjuVdMX9YW7763/HO0Ey4Dn9nEbUfq5bc= +github.com/pingcap/parser v0.0.0-20190611090107-2bb2c0e8f340/go.mod h1:1FNvfp9+J0wvc4kl8eGNh7Rqrxveg15jJoWo/a0uHwA= github.com/pingcap/pd v0.0.0-20190424024702-bd1e2496a669 h1:ZoKjndm/Ig7Ru/wojrQkc/YLUttUdQXoH77gtuWCvL4= github.com/pingcap/pd v0.0.0-20190424024702-bd1e2496a669/go.mod h1:MUCxRzOkYiWZtlyi4MhxjCIj9PgQQ/j+BLNGm7aUsnM= github.com/pingcap/tidb-tools v2.1.3-0.20190321065848-1e8b48f5c168+incompatible h1:MkWCxgZpJBgY2f4HtwWMMFzSBb3+JPzeJgF3VrXE/bU= From 951b65feaf83e5d0b4768bba8c2cabea60309deb Mon Sep 17 00:00:00 2001 From: kennytm Date: Tue, 11 Jun 2019 23:04:20 +0800 Subject: [PATCH 3/3] ddl: addressed comment --- ddl/ddl_api.go | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 40b3ee09c6f13..ac3bc0e54f077 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -2107,15 +2107,16 @@ func (d *ddl) CoalescePartitions(ctx sessionctx.Context, ident ast.Ident, spec * } switch meta.Partition.Type { - // Coalesce partition can only be used on hash/key partitions. - default: - return errors.Trace(ErrCoalesceOnlyOnHashPartition) - // We don't support coalesce partitions hash type partition now. 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)