diff --git a/ddl/db_integration_test.go b/ddl/db_integration_test.go
index 05be31f16d3f4..b9435bc5194b1 100644
--- a/ddl/db_integration_test.go
+++ b/ddl/db_integration_test.go
@@ -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")
diff --git a/ddl/db_partition_test.go b/ddl/db_partition_test.go
index c9dd0dad36bda..a1f3f6995c155 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 *testIntegrationSuite6) TestAlterTableDropPartition(c *C) {
@@ -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) ) (
@@ -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")
+}
diff --git a/ddl/ddl.go b/ddl/ddl.go
index 8cc1b5d840769..a5bb7b6637088 100644
--- a/ddl/ddl.go
+++ b/ddl/ddl.go
@@ -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.
@@ -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.
@@ -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.
@@ -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() {
@@ -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
 }
diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go
index 890d1a721a29c..56fe7e7c50020 100644
--- a/ddl/ddl_api.go
+++ b/ddl/ddl_api.go
@@ -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]
@@ -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.
@@ -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
 	}
@@ -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 {
@@ -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 {
@@ -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 {
@@ -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)
 	}
@@ -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 {
@@ -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)
 	}
@@ -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)
@@ -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,
@@ -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.
@@ -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()
diff --git a/ddl/partition.go b/ddl/partition.go
index 7686e848312a3..8b66ced4a855c 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":
@@ -111,30 +122,34 @@ func buildHashPartitionDefinitions(ctx sessionctx.Context, d *ddl, s *ast.Create
 	defs := make([]model.PartitionDefinition, pi.Num)
 	for i := 0; i < len(defs); i++ {
 		defs[i].ID = genIDs[i]
-		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
 }
 
 func buildRangePartitionDefinitions(ctx sessionctx.Context, d *ddl, s *ast.CreateTableStmt, pi *model.PartitionInfo) error {
-	genIDs, err := d.genGlobalIDs(len(s.Partition.Definitions))
+	genIDs, err := d.genGlobalIDs(int(pi.Num))
 	if err != nil {
 		return err
 	}
 	for ith, def := range s.Partition.Definitions {
+		comment, _ := def.Comment()
 		piDef := model.PartitionDefinition{
 			Name:    def.Name,
 			ID:      genIDs[ith],
-			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()
@@ -450,14 +465,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
 }
@@ -558,20 +573,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 0fbd9a208bb3a..609caf5890359 100644
--- a/go.mod
+++ b/go.mod
@@ -78,3 +78,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-20190722175442-bd2bba5ae990
diff --git a/go.sum b/go.sum
index 1bc02a5605926..7fd8dc1b9e571 100644
--- a/go.sum
+++ b/go.sum
@@ -106,6 +106,8 @@ github.com/json-iterator/go v1.1.6 h1:MrUvLMLTMxbqFJ9kzlvat/rYZqZnW3u4wkLzWTaFwK
 github.com/json-iterator/go v1.1.6/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU=
 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-20190722175442-bd2bba5ae990 h1:Pob0uX7dh8E7ulEVyqhtYD8ooJ1CZYTsilSDU4QdM04=
+github.com/kennytm/parser v0.0.0-20190722175442-bd2bba5ae990/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=
@@ -166,8 +168,6 @@ github.com/pingcap/kvproto v0.0.0-20190703131923-d9830856b531/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-20190712081837-c3bdffe5d00e h1:bFfuLDvmMO6QvxkCAEsfJihxBkcrk58MYJRVpM/1Ujk=
-github.com/pingcap/parser v0.0.0-20190712081837-c3bdffe5d00e/go.mod h1:1FNvfp9+J0wvc4kl8eGNh7Rqrxveg15jJoWo/a0uHwA=
 github.com/pingcap/pd v0.0.0-20190711034019-ee98bf9063e9 h1:sqqiviE8oEYXJh3Aq59HO/AhxjsvcRb9ETh0ivFOHXc=
 github.com/pingcap/pd v0.0.0-20190711034019-ee98bf9063e9/go.mod h1:3DlDlFT7EF64A1bmb/tulZb6wbPSagm5G4p1AlhaEDs=
 github.com/pingcap/tidb-tools v2.1.3-0.20190321065848-1e8b48f5c168+incompatible h1:MkWCxgZpJBgY2f4HtwWMMFzSBb3+JPzeJgF3VrXE/bU=