Skip to content

Commit

Permalink
*: remove ddl dependency (pingcap#32910)
Browse files Browse the repository at this point in the history
  • Loading branch information
hawkingrei authored Mar 9, 2022
1 parent 7bf5e4e commit a1d8f2f
Show file tree
Hide file tree
Showing 42 changed files with 1,417 additions and 1,155 deletions.
5 changes: 3 additions & 2 deletions ddl/backfilling.go
Original file line number Diff line number Diff line change
Expand Up @@ -38,6 +38,7 @@ import (
"github.com/pingcap/tidb/table"
"github.com/pingcap/tidb/tablecodec"
"github.com/pingcap/tidb/util"
"github.com/pingcap/tidb/util/dbterror"
"github.com/pingcap/tidb/util/logutil"
decoder "github.com/pingcap/tidb/util/rowDecoder"
"github.com/pingcap/tidb/util/timeutil"
Expand Down Expand Up @@ -292,7 +293,7 @@ func (w *backfillWorker) handleBackfillTask(d *ddlCtx, task *reorgBackfillTask,
func (w *backfillWorker) run(d *ddlCtx, bf backfiller, job *model.Job) {
logutil.BgLogger().Info("[ddl] backfill worker start", zap.Int("workerID", w.id))
defer func() {
w.resultCh <- &backfillResult{err: errReorgPanic}
w.resultCh <- &backfillResult{err: dbterror.ErrReorgPanic}
}()
defer util.Recover(metrics.LabelDDL, "backfillWorker.run", nil, false)
for {
Expand Down Expand Up @@ -343,7 +344,7 @@ func splitTableRanges(t table.PhysicalTable, store kv.Storage, startKey, endKey
}
if len(ranges) == 0 {
errMsg := fmt.Sprintf("cannot find region in range [%s, %s]", startKey.String(), endKey.String())
return nil, errors.Trace(errInvalidSplitRegionRanges.GenWithStackByArgs(errMsg))
return nil, errors.Trace(dbterror.ErrInvalidSplitRegionRanges.GenWithStackByArgs(errMsg))
}
return ranges, nil
}
Expand Down
58 changes: 27 additions & 31 deletions ddl/column.go
Original file line number Diff line number Diff line change
Expand Up @@ -43,7 +43,7 @@ import (
"github.com/pingcap/tidb/tablecodec"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util"
"github.com/pingcap/tidb/util/collate"
"github.com/pingcap/tidb/util/dbterror"
"github.com/pingcap/tidb/util/logutil"
decoder "github.com/pingcap/tidb/util/rowDecoder"
"github.com/pingcap/tidb/util/sqlexec"
Expand Down Expand Up @@ -250,7 +250,7 @@ func onAddColumn(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, err error)
job.FinishTableJob(model.JobStateDone, model.StatePublic, ver, tblInfo)
asyncNotifyEvent(d, &ddlutil.Event{Tp: model.ActionAddColumn, TableInfo: tblInfo, ColumnInfos: []*model.ColumnInfo{columnInfo}})
default:
err = ErrInvalidDDLState.GenWithStackByArgs("column", columnInfo.State)
err = dbterror.ErrInvalidDDLState.GenWithStackByArgs("column", columnInfo.State)
}

return ver, errors.Trace(err)
Expand Down Expand Up @@ -413,7 +413,7 @@ func onAddColumns(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, err error
job.FinishTableJob(model.JobStateDone, model.StatePublic, ver, tblInfo)
asyncNotifyEvent(d, &ddlutil.Event{Tp: model.ActionAddColumns, TableInfo: tblInfo, ColumnInfos: columnInfos})
default:
err = ErrInvalidDDLState.GenWithStackByArgs("column", columnInfos[0].State)
err = dbterror.ErrInvalidDDLState.GenWithStackByArgs("column", columnInfos[0].State)
}

return ver, errors.Trace(err)
Expand Down Expand Up @@ -490,7 +490,7 @@ func onDropColumns(t *meta.Meta, job *model.Job) (ver int64, _ error) {
job.Args = append(job.Args, getPartitionIDs(tblInfo))
}
default:
err = errInvalidDDLJob.GenWithStackByArgs("table", tblInfo.State)
err = dbterror.ErrInvalidDDLJob.GenWithStackByArgs("table", tblInfo.State)
}
return ver, errors.Trace(err)
}
Expand Down Expand Up @@ -525,7 +525,7 @@ func checkDropColumns(t *meta.Meta, job *model.Job) (*model.TableInfo, []*model.
continue
}
job.State = model.JobStateCancelled
return nil, nil, 0, nil, ErrCantDropFieldOrKey.GenWithStack("column %s doesn't exist", colName)
return nil, nil, 0, nil, dbterror.ErrCantDropFieldOrKey.GenWithStack("column %s doesn't exist", colName)
}
if err = isDroppableColumn(job.MultiSchemaInfo != nil, tblInfo, colName); err != nil {
job.State = model.JobStateCancelled
Expand Down Expand Up @@ -632,7 +632,7 @@ func onDropColumn(t *meta.Meta, job *model.Job) (ver int64, _ error) {
job.Args = append(job.Args, getPartitionIDs(tblInfo))
}
default:
err = errInvalidDDLJob.GenWithStackByArgs("table", tblInfo.State)
err = dbterror.ErrInvalidDDLJob.GenWithStackByArgs("table", tblInfo.State)
}
return ver, errors.Trace(err)
}
Expand All @@ -656,7 +656,7 @@ func checkDropColumn(t *meta.Meta, job *model.Job) (*model.TableInfo, *model.Col
colInfo := model.FindColumnInfo(tblInfo.Columns, colName.L)
if colInfo == nil || colInfo.Hidden {
job.State = model.JobStateCancelled
return nil, nil, nil, ErrCantDropFieldOrKey.GenWithStack("column %s doesn't exist", colName)
return nil, nil, nil, dbterror.ErrCantDropFieldOrKey.GenWithStack("column %s doesn't exist", colName)
}
if err = isDroppableColumn(job.MultiSchemaInfo != nil, tblInfo, colName); err != nil {
job.State = model.JobStateCancelled
Expand Down Expand Up @@ -742,13 +742,9 @@ func needChangeColumnData(oldCol, newCol *model.ColumnInfo) bool {
return true
}

// Column type conversion between varchar to char need reorganization because
// 1. varchar -> char: char type is stored with the padding removed. All the indexes need to be rewritten.
// 2. char -> varchar: the index value encoding of secondary index on clustered primary key tables is different.
// These secondary indexes need to be rewritten.
// TODO: it is used for plugins. so change plugin's using and remove it.
func convertBetweenCharAndVarchar(oldCol, newCol byte) bool {
return (types.IsTypeVarchar(oldCol) && newCol == mysql.TypeString) ||
(oldCol == mysql.TypeString && types.IsTypeVarchar(newCol) && collate.NewCollationEnabled())
return types.ConvertBetweenCharAndVarchar(oldCol, newCol)
}

func isElemsChangedToModifyColumn(oldElems, newElems []string) bool {
Expand Down Expand Up @@ -887,7 +883,7 @@ func (w *worker) onModifyColumn(d *ddlCtx, t *meta.Meta, job *model.Job) (ver in
if mysql.HasPriKeyFlag(oldCol.Flag) {
job.State = model.JobStateCancelled
msg := "this column has primary key flag"
return ver, errUnsupportedModifyColumn.GenWithStackByArgs(msg)
return ver, dbterror.ErrUnsupportedModifyColumn.GenWithStackByArgs(msg)
}

jobParam.changingCol = jobParam.newCol.Clone()
Expand Down Expand Up @@ -975,7 +971,7 @@ func (w *worker) doModifyColumnTypeWithData(
// Introduce the `mysql.PreventNullInsertFlag` flag to prevent users from inserting or updating null values.
err := modifyColsFromNull2NotNull(w, dbInfo, tblInfo, []*model.ColumnInfo{oldCol}, targetCol, oldCol.Tp != changingCol.Tp)
if err != nil {
if ErrWarnDataTruncated.Equal(err) || errInvalidUseOfNull.Equal(err) {
if dbterror.ErrWarnDataTruncated.Equal(err) || dbterror.ErrInvalidUseOfNull.Equal(err) {
job.State = model.JobStateRollingback
}
return ver, err
Expand Down Expand Up @@ -1014,7 +1010,7 @@ func (w *worker) doModifyColumnTypeWithData(
// Introduce the `mysql.PreventNullInsertFlag` flag to prevent users from inserting or updating null values.
err := modifyColsFromNull2NotNull(w, dbInfo, tblInfo, []*model.ColumnInfo{oldCol}, targetCol, oldCol.Tp != changingCol.Tp)
if err != nil {
if ErrWarnDataTruncated.Equal(err) || errInvalidUseOfNull.Equal(err) {
if dbterror.ErrWarnDataTruncated.Equal(err) || dbterror.ErrInvalidUseOfNull.Equal(err) {
job.State = model.JobStateRollingback
}
return ver, err
Expand Down Expand Up @@ -1058,15 +1054,15 @@ func (w *worker) doModifyColumnTypeWithData(
err = w.runReorgJob(t, reorgInfo, tbl.Meta(), d.lease, func() (addIndexErr error) {
defer util.Recover(metrics.LabelDDL, "onModifyColumn",
func() {
addIndexErr = errCancelledDDLJob.GenWithStack("modify table `%v` column `%v` panic", tblInfo.Name, oldCol.Name)
addIndexErr = dbterror.ErrCancelledDDLJob.GenWithStack("modify table `%v` column `%v` panic", tblInfo.Name, oldCol.Name)
}, false)
// Use old column name to generate less confusing error messages.
changingColCpy := changingCol.Clone()
changingColCpy.Name = oldCol.Name
return w.updateColumnAndIndexes(tbl, oldCol, changingColCpy, changingIdxs, reorgInfo)
})
if err != nil {
if errWaitReorgTimeout.Equal(err) {
if dbterror.ErrWaitReorgTimeout.Equal(err) {
// If timeout, we should return, check for the owner and re-wait job done.
return ver, nil
}
Expand Down Expand Up @@ -1130,7 +1126,7 @@ func (w *worker) doModifyColumnTypeWithData(
job.Args = []interface{}{oldIdxIDs, getPartitionIDs(tblInfo)}
asyncNotifyEvent(d, &ddlutil.Event{Tp: model.ActionModifyColumn, TableInfo: tblInfo, ColumnInfos: []*model.ColumnInfo{changingCol}})
default:
err = ErrInvalidDDLState.GenWithStackByArgs("column", changingCol.State)
err = dbterror.ErrInvalidDDLState.GenWithStackByArgs("column", changingCol.State)
}

return ver, errors.Trace(err)
Expand Down Expand Up @@ -1164,7 +1160,7 @@ func (w *worker) updateColumnAndIndexes(t table.Table, oldCol, col *model.Column
time.Sleep(30 * time.Millisecond)
if w.reorgCtx.isReorgCanceled() {
// Job is cancelled. So it can't be done.
failpoint.Return(errCancelledDDLJob)
failpoint.Return(dbterror.ErrCancelledDDLJob)
}
}
}
Expand Down Expand Up @@ -1325,7 +1321,7 @@ func (w *updateColumnWorker) getRowRecord(handle kv.Handle, recordKey []byte, ra
sysTZ := w.sessCtx.GetSessionVars().StmtCtx.TimeZone
_, err := w.rowDecoder.DecodeTheExistedColumnMap(w.sessCtx, handle, rawRow, sysTZ, w.rowMap)
if err != nil {
return errors.Trace(errCantDecodeRecord.GenWithStackByArgs("column", err))
return errors.Trace(dbterror.ErrCantDecodeRecord.GenWithStackByArgs("column", err))
}

if _, ok := w.rowMap[w.newColInfo.ID]; ok {
Expand Down Expand Up @@ -1363,7 +1359,7 @@ func (w *updateColumnWorker) getRowRecord(handle kv.Handle, recordKey []byte, ra
failpoint.Inject("MockReorgTimeoutInOneRegion", func(val failpoint.Value) {
if val.(bool) {
if handle.IntValue() == 3000 && atomic.CompareAndSwapInt32(&TestCheckReorgTimeout, 0, 1) {
failpoint.Return(errors.Trace(errWaitReorgTimeout))
failpoint.Return(errors.Trace(dbterror.ErrWaitReorgTimeout))
}
}
})
Expand Down Expand Up @@ -1477,7 +1473,7 @@ func (w *worker) doModifyColumn(
newCol, oldCol *model.ColumnInfo, pos *ast.ColumnPosition) (ver int64, _ error) {
if oldCol.ID != newCol.ID {
job.State = model.JobStateRollingback
return ver, errKeyColumnDoesNotExits.GenWithStack("column %s id %d does not exist, this column may have been updated by other DDL ran in parallel", oldCol.Name, newCol.ID)
return ver, dbterror.ErrKeyColumnDoesNotExits.GenWithStack("column %s id %d does not exist, this column may have been updated by other DDL ran in parallel", oldCol.Name, newCol.ID)
}
// Column from null to not null.
if !mysql.HasNotNullFlag(oldCol.Flag) && mysql.HasNotNullFlag(newCol.Flag) {
Expand All @@ -1492,7 +1488,7 @@ func (w *worker) doModifyColumn(
// Introduce the `mysql.PreventNullInsertFlag` flag to prevent users from inserting or updating null values.
err := modifyColsFromNull2NotNull(w, dbInfo, tblInfo, []*model.ColumnInfo{oldCol}, newCol, oldCol.Tp != newCol.Tp)
if err != nil {
if ErrWarnDataTruncated.Equal(err) || errInvalidUseOfNull.Equal(err) {
if dbterror.ErrWarnDataTruncated.Equal(err) || dbterror.ErrInvalidUseOfNull.Equal(err) {
job.State = model.JobStateRollingback
}
return ver, err
Expand Down Expand Up @@ -1637,7 +1633,7 @@ func checkNewAutoRandomBits(idAccessors meta.AutoIDAccessors, oldCol *model.Colu
if usedBits > newLayout.IncrementalBits {
overflowCnt := usedBits - newLayout.IncrementalBits
errMsg := fmt.Sprintf(autoid.AutoRandomOverflowErrMsg, newAutoRandBits-overflowCnt, newAutoRandBits, oldCol.Name.O)
return ErrInvalidAutoRandom.GenWithStackByArgs(errMsg)
return dbterror.ErrInvalidAutoRandom.GenWithStackByArgs(errMsg)
}
return nil
}
Expand All @@ -1654,7 +1650,7 @@ func applyNewAutoRandomBits(d *ddlCtx, m *meta.Meta, dbInfo *model.DBInfo,
autoRandAlloc := autoid.NewAllocatorsFromTblInfo(d.store, dbInfo.ID, tblInfo).Get(autoid.AutoRandomType)
if autoRandAlloc == nil {
errMsg := fmt.Sprintf(autoid.AutoRandomAllocatorNotFound, dbInfo.Name.O, tblInfo.Name.O)
return ErrInvalidAutoRandom.GenWithStackByArgs(errMsg)
return dbterror.ErrInvalidAutoRandom.GenWithStackByArgs(errMsg)
}
idAcc := m.GetAutoIDAccessors(dbInfo.ID, tblInfo.ID).RowID()
nextAutoIncID, err := idAcc.Get()
Expand Down Expand Up @@ -1706,9 +1702,9 @@ func checkForNullValue(ctx context.Context, sctx sessionctx.Context, isDataTrunc
rowCount := len(rows)
if rowCount != 0 {
if isDataTruncated {
return ErrWarnDataTruncated.GenWithStackByArgs(newCol.Name.L, rowCount)
return dbterror.ErrWarnDataTruncated.GenWithStackByArgs(newCol.Name.L, rowCount)
}
return errInvalidUseOfNull
return dbterror.ErrInvalidUseOfNull
}
return nil
}
Expand Down Expand Up @@ -1754,12 +1750,12 @@ func isColumnCanDropWithIndex(isMultiSchemaChange bool, colName string, indices
if indexInfo.Primary || len(indexInfo.Columns) > 1 {
for _, col := range indexInfo.Columns {
if col.Name.L == colName {
return errCantDropColWithIndex.GenWithStack("can't drop column %s with composite index covered or Primary Key covered now", colName)
return dbterror.ErrCantDropColWithIndex.GenWithStack("can't drop column %s with composite index covered or Primary Key covered now", colName)
}
}
}
if len(indexInfo.Columns) == 1 && indexInfo.Columns[0].Name.L == colName && !isMultiSchemaChange {
return errCantDropColWithIndex.GenWithStack("can't drop column %s with tidb_enable_change_multi_schema is disable", colName)
return dbterror.ErrCantDropColWithIndex.GenWithStack("can't drop column %s with tidb_enable_change_multi_schema is disable", colName)
}
}
return nil
Expand Down Expand Up @@ -1793,7 +1789,7 @@ func allocateColumnID(tblInfo *model.TableInfo) int64 {

func checkAddColumnTooManyColumns(colNum int) error {
if uint32(colNum) > atomic.LoadUint32(&config.GetGlobalConfig().TableColumnCountLimit) {
return errTooManyFields
return dbterror.ErrTooManyFields
}
return nil
}
Expand Down
15 changes: 8 additions & 7 deletions ddl/column_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,7 @@ import (
"github.com/pingcap/tidb/table/tables"
"github.com/pingcap/tidb/tablecodec"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/dbterror"
"github.com/stretchr/testify/require"
"github.com/stretchr/testify/suite"
)
Expand Down Expand Up @@ -1193,7 +1194,7 @@ func TestModifyColumn(t *testing.T) {
{"int", "int unsigned", nil},
{"varchar(10)", "text", nil},
{"varbinary(10)", "blob", nil},
{"text", "blob", errUnsupportedModifyCharset.GenWithStackByArgs("charset from utf8mb4 to binary")},
{"text", "blob", dbterror.ErrUnsupportedModifyCharset.GenWithStackByArgs("charset from utf8mb4 to binary")},
{"varchar(10)", "varchar(8)", nil},
{"varchar(10)", "varchar(11)", nil},
{"varchar(10) character set utf8 collate utf8_bin", "varchar(10) character set utf8", nil},
Expand All @@ -1203,12 +1204,12 @@ func TestModifyColumn(t *testing.T) {
{"decimal(2,1)", "int", nil},
{"decimal", "int", nil},
{"decimal(2,1)", "bigint", nil},
{"int", "varchar(10) character set gbk", errUnsupportedModifyCharset.GenWithStackByArgs("charset from binary to gbk")},
{"varchar(10) character set gbk", "int", errUnsupportedModifyCharset.GenWithStackByArgs("charset from gbk to binary")},
{"varchar(10) character set gbk", "varchar(10) character set utf8", errUnsupportedModifyCharset.GenWithStackByArgs("charset from gbk to utf8")},
{"varchar(10) character set gbk", "char(10) character set utf8", errUnsupportedModifyCharset.GenWithStackByArgs("charset from gbk to utf8")},
{"varchar(10) character set utf8", "char(10) character set gbk", errUnsupportedModifyCharset.GenWithStackByArgs("charset from utf8 to gbk")},
{"varchar(10) character set utf8", "varchar(10) character set gbk", errUnsupportedModifyCharset.GenWithStackByArgs("charset from utf8 to gbk")},
{"int", "varchar(10) character set gbk", dbterror.ErrUnsupportedModifyCharset.GenWithStackByArgs("charset from binary to gbk")},
{"varchar(10) character set gbk", "int", dbterror.ErrUnsupportedModifyCharset.GenWithStackByArgs("charset from gbk to binary")},
{"varchar(10) character set gbk", "varchar(10) character set utf8", dbterror.ErrUnsupportedModifyCharset.GenWithStackByArgs("charset from gbk to utf8")},
{"varchar(10) character set gbk", "char(10) character set utf8", dbterror.ErrUnsupportedModifyCharset.GenWithStackByArgs("charset from gbk to utf8")},
{"varchar(10) character set utf8", "char(10) character set gbk", dbterror.ErrUnsupportedModifyCharset.GenWithStackByArgs("charset from utf8 to gbk")},
{"varchar(10) character set utf8", "varchar(10) character set gbk", dbterror.ErrUnsupportedModifyCharset.GenWithStackByArgs("charset from utf8 to gbk")},
{"varchar(10) character set gbk", "varchar(255) character set gbk", nil},
}
for _, tt := range tests {
Expand Down
4 changes: 2 additions & 2 deletions ddl/db_cache_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,14 +18,14 @@ import (
"testing"
"time"

"github.com/pingcap/tidb/ddl"
"github.com/pingcap/tidb/domain"
"github.com/pingcap/tidb/errno"
"github.com/pingcap/tidb/parser/model"
"github.com/pingcap/tidb/parser/terror"
"github.com/pingcap/tidb/session"
"github.com/pingcap/tidb/store/mockstore"
"github.com/pingcap/tidb/testkit"
"github.com/pingcap/tidb/util/dbterror"
"github.com/stretchr/testify/require"
)

Expand Down Expand Up @@ -189,7 +189,7 @@ func TestAlterTableCache(t *testing.T) {
tk.MustExec("create global temporary table tmp1 " +
"(id int not null primary key, code int not null, value int default null, unique key code(code))" +
"on commit delete rows")
tk.MustGetErrMsg("alter table tmp1 cache", ddl.ErrOptOnTemporaryTable.GenWithStackByArgs("alter temporary table cache").Error())
tk.MustGetErrMsg("alter table tmp1 cache", dbterror.ErrOptOnTemporaryTable.GenWithStackByArgs("alter temporary table cache").Error())
}

func TestCacheTableSizeLimit(t *testing.T) {
Expand Down
14 changes: 7 additions & 7 deletions ddl/db_integration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,6 @@ import (

"github.com/pingcap/errors"
"github.com/pingcap/tidb/config"
"github.com/pingcap/tidb/ddl"
"github.com/pingcap/tidb/domain"
"github.com/pingcap/tidb/errno"
"github.com/pingcap/tidb/infoschema"
Expand All @@ -47,6 +46,7 @@ import (
"github.com/pingcap/tidb/testkit"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/collate"
"github.com/pingcap/tidb/util/dbterror"
"github.com/pingcap/tidb/util/mock"
"github.com/stretchr/testify/require"
"github.com/tikv/client-go/v2/testutils"
Expand Down Expand Up @@ -1813,7 +1813,7 @@ func assertWarningExec(tk *testkit.TestKit, t *testing.T, sql string, expectedWa
}

func assertAlterWarnExec(tk *testkit.TestKit, t *testing.T, sql string) {
assertWarningExec(tk, t, sql, ddl.ErrAlterOperationNotSupported)
assertWarningExec(tk, t, sql, dbterror.ErrAlterOperationNotSupported)
}

func TestAlterAlgorithm(t *testing.T) {
Expand Down Expand Up @@ -1921,8 +1921,8 @@ func TestFulltextIndexIgnore(t *testing.T) {
tk.MustExec("drop table if exists t_ft")
defer tk.MustExec("drop table if exists t_ft")
// Make sure that creating and altering to add a fulltext key gives the correct warning
assertWarningExec(tk, t, "create table t_ft (a text, fulltext key (a))", ddl.ErrTableCantHandleFt)
assertWarningExec(tk, t, "alter table t_ft add fulltext key (a)", ddl.ErrTableCantHandleFt)
assertWarningExec(tk, t, "create table t_ft (a text, fulltext key (a))", dbterror.ErrTableCantHandleFt)
assertWarningExec(tk, t, "alter table t_ft add fulltext key (a)", dbterror.ErrTableCantHandleFt)

// Make sure table t_ft still has no indexes even after it was created and altered
r := tk.MustQuery("show index from t_ft")
Expand Down Expand Up @@ -3908,16 +3908,16 @@ func TestInvalidPartitionNameWhenCreateTable(t *testing.T) {

_, err := tk.Exec("create table t(a int) partition by range (a) (partition p0 values less than (0), partition `p1 ` values less than (3))")
require.Error(t, err)
require.Truef(t, terror.ErrorEqual(err, ddl.ErrWrongPartitionName), "err %v", err)
require.Truef(t, terror.ErrorEqual(err, dbterror.ErrWrongPartitionName), "err %v", err)

_, err = tk.Exec("create table t(a int) partition by range (a) (partition `` values less than (0), partition `p1` values less than (3))")
require.Error(t, err)
require.Truef(t, terror.ErrorEqual(err, ddl.ErrWrongPartitionName), "err %v", err)
require.Truef(t, terror.ErrorEqual(err, dbterror.ErrWrongPartitionName), "err %v", err)

tk.MustExec("create table t(a int) partition by range (a) (partition `p0` values less than (0), partition `p1` values less than (3))")
_, err = tk.Exec("alter table t add partition (partition `p2 ` values less than (5))")
require.Error(t, err)
require.Truef(t, terror.ErrorEqual(err, ddl.ErrWrongPartitionName), "err %v", err)
require.Truef(t, terror.ErrorEqual(err, dbterror.ErrWrongPartitionName), "err %v", err)
}

func TestDDLLastInfo(t *testing.T) {
Expand Down
Loading

0 comments on commit a1d8f2f

Please sign in to comment.