From 70f6354be5f731b8fc909c66a892be1d009ed88d Mon Sep 17 00:00:00 2001 From: zhaox1n Date: Mon, 4 May 2020 22:42:53 +0800 Subject: [PATCH 01/57] exchange partition init push --- ddl/ddl_api.go | 78 +++++++++++++++++++++++++++++++++++++ ddl/ddl_worker.go | 2 + ddl/error.go | 2 + ddl/partition.go | 66 +++++++++++++++++++++++++++++++ go.mod | 2 + go.sum | 2 + planner/core/planbuilder.go | 2 +- 7 files changed, 153 insertions(+), 1 deletion(-) diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index f77d70ff26adb..d1a04d8c7b63e 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -2120,6 +2120,8 @@ func (d *ddl) AlterTable(ctx sessionctx.Context, ident ast.Ident, specs []*ast.A err = d.DropTablePartition(ctx, ident, spec) case ast.AlterTableTruncatePartition: err = d.TruncateTablePartition(ctx, ident, spec) + case ast.AlterTableExchangePartition: + err = d.ExchangeTableParition(ctx, ident, spec) case ast.AlterTableAddConstraint: constr := spec.Constraint switch spec.Constraint.Tp { @@ -2688,6 +2690,82 @@ func (d *ddl) DropTablePartition(ctx sessionctx.Context, ident ast.Ident, spec * return errors.Trace(err) } +func (d *ddl) ExchangeTableParition(ctx sessionctx.Context, ident ast.Ident, spec *ast.AlterTableSpec) error { + is := d.infoHandle.Get() + ptSchema, ok := is.SchemaByName(ident.Schema) + if !ok { + return errors.Trace(infoschema.ErrDatabaseNotExists.GenWithStackByArgs(ptSchema)) + } + pt, err := is.TableByName(ident.Schema, ident.Name) + if err != nil { + return errors.Trace(infoschema.ErrTableNotExists.GenWithStackByArgs(ident.Schema, ident.Name)) + } + + ptMeta := pt.Meta() + if ptMeta.GetPartitionInfo() == nil { + return errors.Trace(ErrPartitionMgmtOnNonpartitioned) + } + + partName := spec.PartitionNames[0].L + + // NOTE: if pt is subPartitioned, it should be checked + + _, err = getPartitionDef(ptMeta, partName) + if err != nil { + return errors.Trace(err) + } + + ntIdent := ast.Ident{Schema: spec.NewTable.Schema, Name: spec.NewTable.Name} + + ntSchema, ok := is.SchemaByName(ntIdent.Schema) + if !ok { + return errors.Trace(infoschema.ErrDatabaseNotExists.GenWithStackByArgs(ntSchema)) + } + nt, err := is.TableByName(ntIdent.Schema, ntIdent.Name) + if err != nil { + return errors.Trace(infoschema.ErrTableNotExists.GenWithStackByArgs(ntIdent.Schema, ntIdent.Name)) + } + + ntMeta := nt.Meta() + + // NOTE: if nt is temporary table, it should be checked + + if ntMeta.GetPartitionInfo() != nil { + return errors.Trace(ErrPartitionExchangePartTable.GenWithStackByArgs(ntIdent.Name)) + } + + // err = checkDropTablePartition(meta, partName) + // if err != nil { + // if ErrDropPartitionNonExistent.Equal(err) && spec.IfExists { + // ctx.GetSessionVars().StmtCtx.AppendNote(err) + // return nil + // } + // return errors.Trace(err) + // } + + // TODO check table structure is same + + job := &model.Job{ + SchemaID: ntSchema.ID, + TableID: ntMeta.ID, + SchemaName: ntSchema.Name.L, + Type: model.ActionExchangeTablePartition, + BinlogInfo: &model.HistoryInfo{}, + Args: []interface{}{ptSchema.ID, ptMeta, partName, spec.WithValidation}, + } + + err = d.doDDLJob(ctx, job) + if err != nil { + // if ErrDropPartitionNonExistent.Equal(err) && spec.IfExists { + // ctx.GetSessionVars().StmtCtx.AppendNote(err) + // return nil + // } + return errors.Trace(err) + } + err = d.callHookOnChanged(err) + return errors.Trace(err) +} + // DropColumn will drop a column from the table, now we don't support drop the column with index covered. func (d *ddl) DropColumn(ctx sessionctx.Context, ti ast.Ident, spec *ast.AlterTableSpec) error { schema, t, err := d.getSchemaAndTableByIdent(ctx, ti) diff --git a/ddl/ddl_worker.go b/ddl/ddl_worker.go index 2eb17fb6aad52..9422006cb1a61 100644 --- a/ddl/ddl_worker.go +++ b/ddl/ddl_worker.go @@ -601,6 +601,8 @@ func (w *worker) runDDLJob(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, ver, err = onDropTablePartition(t, job) case model.ActionTruncateTablePartition: ver, err = onTruncateTablePartition(d, t, job) + case model.ActionExchangeTablePartition: + ver, err = onExchangeTablePartition(d, t, job) case model.ActionAddColumn: ver, err = onAddColumn(d, t, job) case model.ActionAddColumns: diff --git a/ddl/error.go b/ddl/error.go index 71c5ca735fe0a..72ff181e6e46c 100644 --- a/ddl/error.go +++ b/ddl/error.go @@ -200,4 +200,6 @@ var ( ErrColumnTypeUnsupportedNextValue = terror.ClassDDL.New(mysql.ErrColumnTypeUnsupportedNextValue, mysql.MySQLErrName[mysql.ErrColumnTypeUnsupportedNextValue]) // ErrUnsupportedExpressionIndex is returned when create an expression index without allow-expression-index. ErrUnsupportedExpressionIndex = terror.ClassDDL.New(mysql.ErrUnsupportedDDLOperation, fmt.Sprintf(mysql.MySQLErrName[mysql.ErrUnsupportedDDLOperation], "creating expression index without allow-expression-index in config")) + // ErrPartitionExchangePartTable is returned when exchange table partition with another table is partitioned + ErrPartitionExchangePartTable = terror.ClassDDL.New(mysql.ErrPartitionExchangePartTable, mysql.MySQLErrName[mysql.ErrPartitionExchangePartTable]) ) diff --git a/ddl/partition.go b/ddl/partition.go index 8292c3ccbc59e..4131c20df2f4d 100644 --- a/ddl/partition.go +++ b/ddl/partition.go @@ -590,6 +590,16 @@ func removePartitionInfo(tblInfo *model.TableInfo, partName string) int64 { return pid } +func getPartitionDef(tblInfo *model.TableInfo, partName string) (*model.PartitionDefinition, error) { + defs := tblInfo.Partition.Definitions + for i := 0; i < len(defs); i++ { + if strings.EqualFold(defs[i].Name.L, strings.ToLower(partName)) { + return &(defs[i]), nil + } + } + return nil, table.ErrUnknownPartition.GenWithStackByArgs(partName, tblInfo.Name.O) +} + // onDropTablePartition deletes old partition meta. func onDropTablePartition(t *meta.Meta, job *model.Job) (ver int64, _ error) { var partName string @@ -681,6 +691,62 @@ func onTruncateTablePartition(d *ddlCtx, t *meta.Meta, job *model.Job) (int64, e return ver, nil } +// onExchangeTablePartition exchange partition data +func onExchangeTablePartition(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ error) { + var ( + ptSchemaID int64 + pt *model.TableInfo + partName string + withValidation bool + ) + if err := job.DecodeArgs(&ptSchemaID, &pt, &partName, &withValidation); err != nil { + job.State = model.JobStateCancelled + return ver, errors.Trace(err) + } + + nt, err := getTableInfoAndCancelFaultJob(t, job, job.SchemaID) + if err != nil { + return ver, errors.Trace(err) + } + + partDef, err := getPartitionDef(pt, partName) + if err != nil { + return ver, errors.Trace(err) + } + + tempID := partDef.ID + + partDef.ID = nt.ID + + err = t.UpdateTable(ptSchemaID, pt) + if err != nil { + job.State = model.JobStateCancelled + return ver, errors.Trace(err) + } + + err = t.DropTableOrView(job.SchemaID, nt.ID, true) + if err != nil { + job.State = model.JobStateCancelled + return ver, errors.Trace(err) + } + + nt.ID = tempID + + err = t.CreateTableOrView(job.SchemaID, nt) + if err != nil { + job.State = model.JobStateCancelled + return ver, errors.Trace(err) + } + + ver, err = updateSchemaVersion(t, job) + if err != nil { + return ver, errors.Trace(err) + } + + job.FinishTableJob(model.JobStateDone, model.StateNone, ver, nt) + return ver, nil +} + func checkAddPartitionTooManyPartitions(piDefs uint64) error { if piDefs > uint64(PartitionCountLimit) { return errors.Trace(ErrTooManyPartitions) diff --git a/go.mod b/go.mod index e71c4a795c97f..14d01089ff915 100644 --- a/go.mod +++ b/go.mod @@ -74,3 +74,5 @@ require ( ) go 1.13 + +replace github.com/pingcap/parser => github.com/zhaox1n/parser v0.0.0-20200430160105-6fcbbbfa6158 diff --git a/go.sum b/go.sum index f824be8fa5d5d..8def24cbe7167 100644 --- a/go.sum +++ b/go.sum @@ -388,6 +388,8 @@ github.com/xiang90/probing v0.0.0-20190116061207-43a291ad63a2 h1:eY9dn8+vbi4tKz5 github.com/xiang90/probing v0.0.0-20190116061207-43a291ad63a2/go.mod h1:UETIi67q53MR2AWcXfiuqkDkRtnGDLqkBTpCHuJHxtU= github.com/yookoala/realpath v1.0.0/go.mod h1:gJJMA9wuX7AcqLy1+ffPatSCySA1FQ2S8Ya9AIoYBpE= github.com/yuin/goldmark v1.1.25/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= +github.com/zhaox1n/parser v0.0.0-20200430160105-6fcbbbfa6158 h1:3EhundhszzvUGiWSvUzwW1wiM5YUVnih3EgIhJZ8a8s= +github.com/zhaox1n/parser v0.0.0-20200430160105-6fcbbbfa6158/go.mod h1:9v0Edh8IbgjGYW2ArJr19E+bvL8zKahsFp+ixWeId+4= go.etcd.io/bbolt v1.3.3 h1:MUGmc65QhB3pIlaQ5bB4LwqSj6GIonVJXpZiaKNyaKk= go.etcd.io/bbolt v1.3.3/go.mod h1:IbVyRI1SCnLcuJnV2u8VeU0CEYM7e686BmAb1XKL+uU= go.etcd.io/etcd v0.0.0-20191023171146-3cf2f69b5738/go.mod h1:dnLIgRNXwCJa5e+c6mIZCrds/GIG4ncV9HhK5PX7jPg= diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index 13fa6d49faaf8..d0068e58f8df2 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -2559,7 +2559,7 @@ func (b *PlanBuilder) buildDDL(ctx context.Context, node ast.DDLNode) (Plan, err b.visitInfo = appendVisitInfo(b.visitInfo, mysql.AlterPriv, v.Table.Schema.L, v.Table.Name.L, "", authErr) for _, spec := range v.Specs { - if spec.Tp == ast.AlterTableRenameTable { + if spec.Tp == ast.AlterTableRenameTable || spec.Tp == ast.AlterTableExchangePartition { if b.ctx.GetSessionVars().User != nil { authErr = ErrTableaccessDenied.GenWithStackByArgs("DROP", b.ctx.GetSessionVars().User.Hostname, b.ctx.GetSessionVars().User.Username, v.Table.Name.L) From 57af241e36358c6098311a2cfd7810904239895f Mon Sep 17 00:00:00 2001 From: zhaox1n Date: Tue, 5 May 2020 03:48:46 +0800 Subject: [PATCH 02/57] exchange partition: check table is compatiable --- ddl/ddl_api.go | 90 +++++++++++++++++++++++++++++++++++++------------- ddl/error.go | 2 ++ 2 files changed, 69 insertions(+), 23 deletions(-) diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index d1a04d8c7b63e..fc1aa26df0333 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -2690,6 +2690,57 @@ func (d *ddl) DropTablePartition(ctx sessionctx.Context, ident ast.Ident, spec * return errors.Trace(err) } +func checkTableDefCompatible(source *model.TableInfo, target *model.TableInfo) error { + err := ErrTablesDifferentMetadata + if len(source.Cols()) != len(target.Cols()) { + return err + } + // Col compatible check + for i, sourceCol := range source.Cols() { + targetCol := target.Cols()[i] + if sourceCol.IsGenerated() != targetCol.IsGenerated() || + !strings.EqualFold(sourceCol.Name.L, targetCol.Name.L) || + !sourceCol.FieldType.Equal(&targetCol.FieldType) { + return err + } + } + if len(source.Indices) != len(target.Indices) { + return err + } + for _, sourceIdx := range source.Indices { + var compatIdx *model.IndexInfo + for _, targetIdx := range target.Indices { + if strings.EqualFold(sourceIdx.Name.O, targetIdx.Name.O) { + compatIdx = targetIdx + } + } + // No match index + if compatIdx == nil { + return err + } + // Index type is not compatiable + if sourceIdx.Tp != compatIdx.Tp || + sourceIdx.Unique != compatIdx.Unique || + sourceIdx.Primary != compatIdx.Primary || + sourceIdx.Invisible != compatIdx.Invisible { + return err + } + // The index column + if len(sourceIdx.Columns) != len(compatIdx.Columns) { + return err + } + for i, sourceIdxCol := range sourceIdx.Columns { + compatIdxCol := compatIdx.Columns[i] + if sourceIdxCol.Name.L != compatIdxCol.Name.L || + sourceIdxCol.Length != compatIdxCol.Length { + return err + } + } + } + + return nil +} + func (d *ddl) ExchangeTableParition(ctx sessionctx.Context, ident ast.Ident, spec *ast.AlterTableSpec) error { is := d.infoHandle.Get() ptSchema, ok := is.SchemaByName(ident.Schema) @@ -2702,18 +2753,6 @@ func (d *ddl) ExchangeTableParition(ctx sessionctx.Context, ident ast.Ident, spe } ptMeta := pt.Meta() - if ptMeta.GetPartitionInfo() == nil { - return errors.Trace(ErrPartitionMgmtOnNonpartitioned) - } - - partName := spec.PartitionNames[0].L - - // NOTE: if pt is subPartitioned, it should be checked - - _, err = getPartitionDef(ptMeta, partName) - if err != nil { - return errors.Trace(err) - } ntIdent := ast.Ident{Schema: spec.NewTable.Schema, Name: spec.NewTable.Name} @@ -2728,22 +2767,27 @@ func (d *ddl) ExchangeTableParition(ctx sessionctx.Context, ident ast.Ident, spe ntMeta := nt.Meta() - // NOTE: if nt is temporary table, it should be checked - + if ptMeta.GetPartitionInfo() == nil { + return errors.Trace(ErrPartitionMgmtOnNonpartitioned) + } if ntMeta.GetPartitionInfo() != nil { return errors.Trace(ErrPartitionExchangePartTable.GenWithStackByArgs(ntIdent.Name)) } - // err = checkDropTablePartition(meta, partName) - // if err != nil { - // if ErrDropPartitionNonExistent.Equal(err) && spec.IfExists { - // ctx.GetSessionVars().StmtCtx.AppendNote(err) - // return nil - // } - // return errors.Trace(err) - // } + // NOTE: if nt is temporary table, it should be checked here + partName := spec.PartitionNames[0].L + + _, err = getPartitionDef(ptMeta, partName) + if err != nil { + return errors.Trace(err) + } + + // NOTE: if pt is subPartitioned, it should be checked - // TODO check table structure is same + err = checkTableDefCompatible(ptMeta, ntMeta) + if err != nil { + return errors.Trace(err) + } job := &model.Job{ SchemaID: ntSchema.ID, diff --git a/ddl/error.go b/ddl/error.go index 72ff181e6e46c..20ac90df4c50a 100644 --- a/ddl/error.go +++ b/ddl/error.go @@ -202,4 +202,6 @@ var ( ErrUnsupportedExpressionIndex = terror.ClassDDL.New(mysql.ErrUnsupportedDDLOperation, fmt.Sprintf(mysql.MySQLErrName[mysql.ErrUnsupportedDDLOperation], "creating expression index without allow-expression-index in config")) // ErrPartitionExchangePartTable is returned when exchange table partition with another table is partitioned ErrPartitionExchangePartTable = terror.ClassDDL.New(mysql.ErrPartitionExchangePartTable, mysql.MySQLErrName[mysql.ErrPartitionExchangePartTable]) + // ErrTableDifferentMetadata is returned when exchanges tables is not compatible + ErrTablesDifferentMetadata = terror.ClassDDL.New(mysql.ErrTablesDifferentMetadata, mysql.MySQLErrName[mysql.ErrTablesDifferentMetadata]) ) From adef48e5261604204e868cf6dfcfd9d81a500281 Mon Sep 17 00:00:00 2001 From: zhaox1n Date: Wed, 6 May 2020 00:08:43 +0800 Subject: [PATCH 03/57] add checke in ddl worker --- ddl/partition.go | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/ddl/partition.go b/ddl/partition.go index 4131c20df2f4d..1ef24ad816e18 100644 --- a/ddl/partition.go +++ b/ddl/partition.go @@ -714,6 +714,15 @@ func onExchangeTablePartition(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int6 return ver, errors.Trace(err) } + err = checkTableDefCompatible(pt, nt) + if err != nil { + return ver, errors.Trace(err) + } + + if withValidation { + + } + tempID := partDef.ID partDef.ID = nt.ID From 1a7f5994f702677c5ad38d292c5dab4c1ce63b01 Mon Sep 17 00:00:00 2001 From: zhaox1n Date: Wed, 6 May 2020 22:25:21 +0800 Subject: [PATCH 04/57] validate check --- ddl/ddl_worker.go | 2 +- ddl/partition.go | 72 +++++++++++++++++++++++++++++++++++++-- table/tables/partition.go | 4 +-- 3 files changed, 73 insertions(+), 5 deletions(-) diff --git a/ddl/ddl_worker.go b/ddl/ddl_worker.go index 9422006cb1a61..856069b548b84 100644 --- a/ddl/ddl_worker.go +++ b/ddl/ddl_worker.go @@ -602,7 +602,7 @@ func (w *worker) runDDLJob(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, case model.ActionTruncateTablePartition: ver, err = onTruncateTablePartition(d, t, job) case model.ActionExchangeTablePartition: - ver, err = onExchangeTablePartition(d, t, job) + ver, err = w.onExchangeTablePartition(d, t, job) case model.ActionAddColumn: ver, err = onAddColumn(d, t, job) case model.ActionAddColumns: diff --git a/ddl/partition.go b/ddl/partition.go index 1ef24ad816e18..8c218929e5604 100644 --- a/ddl/partition.go +++ b/ddl/partition.go @@ -32,8 +32,10 @@ import ( "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/table" + "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/sqlexec" ) const ( @@ -692,7 +694,7 @@ func onTruncateTablePartition(d *ddlCtx, t *meta.Meta, job *model.Job) (int64, e } // onExchangeTablePartition exchange partition data -func onExchangeTablePartition(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ error) { +func (w *worker) onExchangeTablePartition(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ error) { var ( ptSchemaID int64 pt *model.TableInfo @@ -704,6 +706,11 @@ func onExchangeTablePartition(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int6 return ver, errors.Trace(err) } + ntDbInfo, err := t.GetDatabase(job.SchemaID) + if err != nil { + return ver, err + } + nt, err := getTableInfoAndCancelFaultJob(t, job, job.SchemaID) if err != nil { return ver, errors.Trace(err) @@ -720,7 +727,10 @@ func onExchangeTablePartition(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int6 } if withValidation { - + err = checkExchangePartitionRecordValidation(w, pt.Partition, partName, ntDbInfo.Name, nt.Name) + if err != nil { + return ver, errors.Trace(err) + } } tempID := partDef.ID @@ -756,6 +766,64 @@ func onExchangeTablePartition(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int6 return ver, nil } +func checkExchangePartitionRecordValidation(w *worker, pi *model.PartitionInfo, partName string, schemaName, tableName model.CIStr) error { + var ( + sql string + index int + ) + + for i, def := range pi.Definitions { + if strings.EqualFold(def.Name.L, strings.ToLower(partName)) { + index = i + } + if len(pi.Definitions) == i { + return table.ErrUnknownPartition.GenWithStackByArgs(partName, tableName.O) + } + } + + switch pi.Type { + case model.PartitionTypeHash: + sql = fmt.Sprint("select 1 from `%s`.`%s` where mod(%s, %d) = %d", schemaName.L, tableName.L, pi.Expr, pi.Num, index) + break + case model.PartitionTypeRange: + rangeRrun, err := tables.DataForRangePruning(pi) + if err != nil { + return errors.Trace(err) + } + // Table has only one partition and has the maximum value + if len(pi.Definitions) == 1 && rangeRrun.MaxValue { + return nil + } + if index == 0 { + sql = fmt.Sprintf("select 1 from `%s`.`%s` where %s <= %d limit 1", schemaName.L, tableName.L, pi.Expr, rangeRrun.LessThan[index]) + } else if index == len(pi.Definitions)-1 && rangeRrun.MaxValue { + sql = fmt.Sprintf("select 1 from `%s`.`%s` where %s > %d limit 1", schemaName.L, tableName.L, pi.Expr, rangeRrun.LessThan[index]) + } else { + sql = fmt.Sprintf("select 1 from `%s`.`%s` where %s > %d and %s <= %d limit 1", schemaName.L, tableName.L, pi.Expr, rangeRrun.LessThan[index-1], pi.Expr, rangeRrun.LessThan[index]) + } + break + default: + panic("cannot reach here") + } + + var ctx sessionctx.Context + ctx, err := w.sessPool.get() + if err != nil { + return errors.Trace(err) + } + defer w.sessPool.put(ctx) + + rows, _, err := ctx.(sqlexec.RestrictedSQLExecutor).ExecRestrictedSQL(sql) + if err != nil { + return errors.Trace(err) + } + rowCount := len(rows) + if rowCount != 0 { + return ErrTablesDifferentMetadata + } + return nil +} + func checkAddPartitionTooManyPartitions(piDefs uint64) error { if piDefs > uint64(PartitionCountLimit) { return errors.Trace(ErrTooManyPartitions) diff --git a/table/tables/partition.go b/table/tables/partition.go index eac6a82768486..45995e9204230 100644 --- a/table/tables/partition.go +++ b/table/tables/partition.go @@ -127,7 +127,7 @@ type ForRangePruning struct { } // dataForRangePruning extracts the less than parts from 'partition p0 less than xx ... partitoin p1 less than ...' -func dataForRangePruning(pi *model.PartitionInfo) (*ForRangePruning, error) { +func DataForRangePruning(pi *model.PartitionInfo) (*ForRangePruning, error) { var maxValue bool var unsigned bool lessThan := make([]int64, len(pi.Definitions)) @@ -201,7 +201,7 @@ func generateRangePartitionExpr(ctx sessionctx.Context, pi *model.PartitionInfo, UpperBounds: locateExprs, } if len(pi.Columns) == 0 { - tmp, err := dataForRangePruning(pi) + tmp, err := DataForRangePruning(pi) if err != nil { return nil, errors.Trace(err) } From 11809d084992d970574de1e6f018d2ac4443cd04 Mon Sep 17 00:00:00 2001 From: zhaox1n Date: Thu, 7 May 2020 00:43:08 +0800 Subject: [PATCH 05/57] Basic functions have been completed --- ddl/ddl_api.go | 9 +++------ ddl/error.go | 2 ++ ddl/partition.go | 24 ++++++++++++++++-------- 3 files changed, 21 insertions(+), 14 deletions(-) diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index fc1aa26df0333..6ee7e00828fc8 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -2699,7 +2699,7 @@ func checkTableDefCompatible(source *model.TableInfo, target *model.TableInfo) e for i, sourceCol := range source.Cols() { targetCol := target.Cols()[i] if sourceCol.IsGenerated() != targetCol.IsGenerated() || - !strings.EqualFold(sourceCol.Name.L, targetCol.Name.L) || + sourceCol.Name.L != targetCol.Name.L || !sourceCol.FieldType.Equal(&targetCol.FieldType) { return err } @@ -2760,6 +2760,7 @@ func (d *ddl) ExchangeTableParition(ctx sessionctx.Context, ident ast.Ident, spe if !ok { return errors.Trace(infoschema.ErrDatabaseNotExists.GenWithStackByArgs(ntSchema)) } + nt, err := is.TableByName(ntIdent.Schema, ntIdent.Name) if err != nil { return errors.Trace(infoschema.ErrTableNotExists.GenWithStackByArgs(ntIdent.Schema, ntIdent.Name)) @@ -2795,15 +2796,11 @@ func (d *ddl) ExchangeTableParition(ctx sessionctx.Context, ident ast.Ident, spe SchemaName: ntSchema.Name.L, Type: model.ActionExchangeTablePartition, BinlogInfo: &model.HistoryInfo{}, - Args: []interface{}{ptSchema.ID, ptMeta, partName, spec.WithValidation}, + Args: []interface{}{ptSchema.ID, ptMeta.ID, partName, spec.WithValidation}, } err = d.doDDLJob(ctx, job) if err != nil { - // if ErrDropPartitionNonExistent.Equal(err) && spec.IfExists { - // ctx.GetSessionVars().StmtCtx.AppendNote(err) - // return nil - // } return errors.Trace(err) } err = d.callHookOnChanged(err) diff --git a/ddl/error.go b/ddl/error.go index 20ac90df4c50a..8d723f8f38115 100644 --- a/ddl/error.go +++ b/ddl/error.go @@ -204,4 +204,6 @@ var ( ErrPartitionExchangePartTable = terror.ClassDDL.New(mysql.ErrPartitionExchangePartTable, mysql.MySQLErrName[mysql.ErrPartitionExchangePartTable]) // ErrTableDifferentMetadata is returned when exchanges tables is not compatible ErrTablesDifferentMetadata = terror.ClassDDL.New(mysql.ErrTablesDifferentMetadata, mysql.MySQLErrName[mysql.ErrTablesDifferentMetadata]) + // ErrRowDoesNotMatchPartition is return when the row record of exchange table does not match the partition rule + ErrRowDoesNotMatchPartition = terror.ClassDDL.New(mysql.ErrRowDoesNotMatchPartition, mysql.MySQLErrName[mysql.ErrRowDoesNotMatchPartition]) ) diff --git a/ddl/partition.go b/ddl/partition.go index 8c218929e5604..26d13195b100c 100644 --- a/ddl/partition.go +++ b/ddl/partition.go @@ -697,11 +697,11 @@ func onTruncateTablePartition(d *ddlCtx, t *meta.Meta, job *model.Job) (int64, e func (w *worker) onExchangeTablePartition(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ error) { var ( ptSchemaID int64 - pt *model.TableInfo + ptID int64 partName string withValidation bool ) - if err := job.DecodeArgs(&ptSchemaID, &pt, &partName, &withValidation); err != nil { + if err := job.DecodeArgs(&ptSchemaID, &ptID, &partName, &withValidation); err != nil { job.State = model.JobStateCancelled return ver, errors.Trace(err) } @@ -716,19 +716,27 @@ func (w *worker) onExchangeTablePartition(d *ddlCtx, t *meta.Meta, job *model.Jo return ver, errors.Trace(err) } + pt, err := t.GetTable(ptSchemaID, ptID) + if err != nil { + return ver, errors.Trace(err) + } + partDef, err := getPartitionDef(pt, partName) if err != nil { + job.State = model.JobStateCancelled return ver, errors.Trace(err) } err = checkTableDefCompatible(pt, nt) if err != nil { + job.State = model.JobStateCancelled return ver, errors.Trace(err) } if withValidation { err = checkExchangePartitionRecordValidation(w, pt.Partition, partName, ntDbInfo.Name, nt.Name) if err != nil { + job.State = model.JobStateCancelled return ver, errors.Trace(err) } } @@ -743,7 +751,7 @@ func (w *worker) onExchangeTablePartition(d *ddlCtx, t *meta.Meta, job *model.Jo return ver, errors.Trace(err) } - err = t.DropTableOrView(job.SchemaID, nt.ID, true) + err = t.DropTableOrView(job.SchemaID, nt.ID, false) if err != nil { job.State = model.JobStateCancelled return ver, errors.Trace(err) @@ -783,7 +791,7 @@ func checkExchangePartitionRecordValidation(w *worker, pi *model.PartitionInfo, switch pi.Type { case model.PartitionTypeHash: - sql = fmt.Sprint("select 1 from `%s`.`%s` where mod(%s, %d) = %d", schemaName.L, tableName.L, pi.Expr, pi.Num, index) + sql = fmt.Sprintf("select 1 from `%s`.`%s` where mod(%s, %d) not in (%d) limit 1", schemaName.L, tableName.L, pi.Expr, pi.Num, index) break case model.PartitionTypeRange: rangeRrun, err := tables.DataForRangePruning(pi) @@ -795,11 +803,11 @@ func checkExchangePartitionRecordValidation(w *worker, pi *model.PartitionInfo, return nil } if index == 0 { - sql = fmt.Sprintf("select 1 from `%s`.`%s` where %s <= %d limit 1", schemaName.L, tableName.L, pi.Expr, rangeRrun.LessThan[index]) - } else if index == len(pi.Definitions)-1 && rangeRrun.MaxValue { sql = fmt.Sprintf("select 1 from `%s`.`%s` where %s > %d limit 1", schemaName.L, tableName.L, pi.Expr, rangeRrun.LessThan[index]) + } else if index == len(pi.Definitions)-1 && rangeRrun.MaxValue { + sql = fmt.Sprintf("select 1 from `%s`.`%s` where %s <= %d limit 1", schemaName.L, tableName.L, pi.Expr, rangeRrun.LessThan[index]) } else { - sql = fmt.Sprintf("select 1 from `%s`.`%s` where %s > %d and %s <= %d limit 1", schemaName.L, tableName.L, pi.Expr, rangeRrun.LessThan[index-1], pi.Expr, rangeRrun.LessThan[index]) + sql = fmt.Sprintf("select 1 from `%s`.`%s` where %s <= %d and %s > %d limit 1", schemaName.L, tableName.L, pi.Expr, rangeRrun.LessThan[index-1], pi.Expr, rangeRrun.LessThan[index]) } break default: @@ -819,7 +827,7 @@ func checkExchangePartitionRecordValidation(w *worker, pi *model.PartitionInfo, } rowCount := len(rows) if rowCount != 0 { - return ErrTablesDifferentMetadata + return ErrRowDoesNotMatchPartition } return nil } From c8bc05013646fe97a031cd6111edb0e3d652c201 Mon Sep 17 00:00:00 2001 From: zhaox1n Date: Thu, 7 May 2020 23:52:57 +0800 Subject: [PATCH 06/57] autoID added --- ddl/partition.go | 32 ++++++++++++++++++++++++++++---- 1 file changed, 28 insertions(+), 4 deletions(-) diff --git a/ddl/partition.go b/ddl/partition.go index 26d13195b100c..78fbb7a0d0653 100644 --- a/ddl/partition.go +++ b/ddl/partition.go @@ -741,6 +741,18 @@ func (w *worker) onExchangeTablePartition(d *ddlCtx, t *meta.Meta, job *model.Jo } } + ptBaseID, err := t.GetAutoTableID(ptSchemaID, pt.ID) + if err != nil { + job.State = model.JobStateCancelled + return ver, errors.Trace(err) + } + + ntBaseID, err := t.GetAutoTableID(job.SchemaID, nt.ID) + if err != nil { + job.State = model.JobStateCancelled + return ver, errors.Trace(err) + } + tempID := partDef.ID partDef.ID = nt.ID @@ -751,7 +763,13 @@ func (w *worker) onExchangeTablePartition(d *ddlCtx, t *meta.Meta, job *model.Jo return ver, errors.Trace(err) } - err = t.DropTableOrView(job.SchemaID, nt.ID, false) + _, err = t.GenAutoTableID(ptSchemaID, partDef.ID, ntBaseID) + if err != nil { + job.State = model.JobStateCancelled + return ver, errors.Trace(err) + } + + err = t.DropTableOrView(job.SchemaID, nt.ID, true) if err != nil { job.State = model.JobStateCancelled return ver, errors.Trace(err) @@ -765,6 +783,12 @@ func (w *worker) onExchangeTablePartition(d *ddlCtx, t *meta.Meta, job *model.Jo return ver, errors.Trace(err) } + _, err = t.GenAutoTableID(job.SchemaID, nt.ID, ptBaseID) + if err != nil { + job.State = model.JobStateCancelled + return ver, errors.Trace(err) + } + ver, err = updateSchemaVersion(t, job) if err != nil { return ver, errors.Trace(err) @@ -803,11 +827,11 @@ func checkExchangePartitionRecordValidation(w *worker, pi *model.PartitionInfo, return nil } if index == 0 { - sql = fmt.Sprintf("select 1 from `%s`.`%s` where %s > %d limit 1", schemaName.L, tableName.L, pi.Expr, rangeRrun.LessThan[index]) + sql = fmt.Sprintf("select 1 from `%s`.`%s` where %s >= %d limit 1", schemaName.L, tableName.L, pi.Expr, rangeRrun.LessThan[index]) } else if index == len(pi.Definitions)-1 && rangeRrun.MaxValue { - sql = fmt.Sprintf("select 1 from `%s`.`%s` where %s <= %d limit 1", schemaName.L, tableName.L, pi.Expr, rangeRrun.LessThan[index]) + sql = fmt.Sprintf("select 1 from `%s`.`%s` where %s <= %d limit 1", schemaName.L, tableName.L, pi.Expr, rangeRrun.LessThan[index-1]) } else { - sql = fmt.Sprintf("select 1 from `%s`.`%s` where %s <= %d and %s > %d limit 1", schemaName.L, tableName.L, pi.Expr, rangeRrun.LessThan[index-1], pi.Expr, rangeRrun.LessThan[index]) + sql = fmt.Sprintf("select 1 from `%s`.`%s` where %s < %d and %s >= %d limit 1", schemaName.L, tableName.L, pi.Expr, rangeRrun.LessThan[index-1], pi.Expr, rangeRrun.LessThan[index]) } break default: From 5a32fe4dfba616dee05e6d88acb2bbed53000c4b Mon Sep 17 00:00:00 2001 From: zhaox1n Date: Sat, 9 May 2020 01:26:55 +0800 Subject: [PATCH 07/57] test add --- ddl/db_partition_test.go | 30 ++++++++++++++++++++++++++++++ ddl/ddl_api.go | 3 ++- ddl/partition.go | 5 +++-- 3 files changed, 35 insertions(+), 3 deletions(-) diff --git a/ddl/db_partition_test.go b/ddl/db_partition_test.go index 76ccd79a170f0..14831b8593abd 100644 --- a/ddl/db_partition_test.go +++ b/ddl/db_partition_test.go @@ -761,6 +761,36 @@ func (s *testIntegrationSuite5) TestAlterTableDropPartition(c *C) { tk.MustGetErrCode("alter table t1 drop partition p2", tmysql.ErrOnlyOnRangeListPartition) } +func (s *testIntegrationSuite5) TestAlterTableExchangePartition(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists e") + tk.MustExec("drop table if exists e2") + tk.MustExec(`CREATE TABLE e ( + id INT NOT NULL + ) + PARTITION BY RANGE (id) ( + PARTITION p0 VALUES LESS THAN (50), + PARTITION p1 VALUES LESS THAN (100), + PARTITION p2 VALUES LESS THAN (150), + PARTITION p3 VALUES LESS THAN (MAXVALUE) + );`) + tk.MustExec(`CREATE TABLE e2 ( + id INT NOT NULL + );`) + tk.MustExec(`INSERT INTO e VALUES + (1669), + (337), + (16), + (2005)`) + tk.MustExec("ALTER TABLE e EXCHANGE PARTITION p0 WITH TABLE e2") + tk.MustQuery("select * from e2").Check(testkit.Rows("16")) + tk.MustQuery("select * from e").Check(testkit.Rows("1669", "337", "2005")) + tk.MustGetErrCode("ALTER TABLE e EXCHANGE PARTITION p1 WITH TABLE e2", tmysql.ErrRowDoesNotMatchPartition) + tk.MustGetErrCode("ALTER TABLE e EXCHANGE PARTITION p2 WITH TABLE e2", tmysql.ErrRowDoesNotMatchPartition) + tk.MustGetErrCode("ALTER TABLE e EXCHANGE PARTITION p3 WITH TABLE e2", tmysql.ErrRowDoesNotMatchPartition) +} + func (s *testIntegrationSuite4) TestAddPartitionTooManyPartitions(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 6ee7e00828fc8..c3173fe251a61 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -2775,7 +2775,8 @@ func (d *ddl) ExchangeTableParition(ctx sessionctx.Context, ident ast.Ident, spe return errors.Trace(ErrPartitionExchangePartTable.GenWithStackByArgs(ntIdent.Name)) } - // NOTE: if nt is temporary table, it should be checked here + // NOTE: if nt is temporary table, it should be checked + partName := spec.PartitionNames[0].L _, err = getPartitionDef(ptMeta, partName) diff --git a/ddl/partition.go b/ddl/partition.go index 78fbb7a0d0653..78c6b2f90a771 100644 --- a/ddl/partition.go +++ b/ddl/partition.go @@ -807,6 +807,7 @@ func checkExchangePartitionRecordValidation(w *worker, pi *model.PartitionInfo, for i, def := range pi.Definitions { if strings.EqualFold(def.Name.L, strings.ToLower(partName)) { index = i + break } if len(pi.Definitions) == i { return table.ErrUnknownPartition.GenWithStackByArgs(partName, tableName.O) @@ -815,7 +816,7 @@ func checkExchangePartitionRecordValidation(w *worker, pi *model.PartitionInfo, switch pi.Type { case model.PartitionTypeHash: - sql = fmt.Sprintf("select 1 from `%s`.`%s` where mod(%s, %d) not in (%d) limit 1", schemaName.L, tableName.L, pi.Expr, pi.Num, index) + sql = fmt.Sprintf("select 1 from `%s`.`%s` where mod(%s, %d) != %d limit 1", schemaName.L, tableName.L, pi.Expr, pi.Num, index) break case model.PartitionTypeRange: rangeRrun, err := tables.DataForRangePruning(pi) @@ -831,7 +832,7 @@ func checkExchangePartitionRecordValidation(w *worker, pi *model.PartitionInfo, } else if index == len(pi.Definitions)-1 && rangeRrun.MaxValue { sql = fmt.Sprintf("select 1 from `%s`.`%s` where %s <= %d limit 1", schemaName.L, tableName.L, pi.Expr, rangeRrun.LessThan[index-1]) } else { - sql = fmt.Sprintf("select 1 from `%s`.`%s` where %s < %d and %s >= %d limit 1", schemaName.L, tableName.L, pi.Expr, rangeRrun.LessThan[index-1], pi.Expr, rangeRrun.LessThan[index]) + sql = fmt.Sprintf("select 1 from `%s`.`%s` where %s < %d or %s >= %d limit 1", schemaName.L, tableName.L, pi.Expr, rangeRrun.LessThan[index-1], pi.Expr, rangeRrun.LessThan[index]) } break default: From 35941aa34b6f0bfa99efb8eab77664100f55efb9 Mon Sep 17 00:00:00 2001 From: zhaox1n Date: Sat, 9 May 2020 20:49:22 +0800 Subject: [PATCH 08/57] hash exchange partition add --- ddl/db_partition_test.go | 20 ++++++++++++++++++++ 1 file changed, 20 insertions(+) diff --git a/ddl/db_partition_test.go b/ddl/db_partition_test.go index 14831b8593abd..a1ee4bb5ef387 100644 --- a/ddl/db_partition_test.go +++ b/ddl/db_partition_test.go @@ -789,6 +789,26 @@ func (s *testIntegrationSuite5) TestAlterTableExchangePartition(c *C) { tk.MustGetErrCode("ALTER TABLE e EXCHANGE PARTITION p1 WITH TABLE e2", tmysql.ErrRowDoesNotMatchPartition) tk.MustGetErrCode("ALTER TABLE e EXCHANGE PARTITION p2 WITH TABLE e2", tmysql.ErrRowDoesNotMatchPartition) tk.MustGetErrCode("ALTER TABLE e EXCHANGE PARTITION p3 WITH TABLE e2", tmysql.ErrRowDoesNotMatchPartition) + + tk.MustExec("drop table if exists e3") + + tk.MustExec(`CREATE TABLE e3 ( + id int not null + ) PARTITION BY HASH (id) + PARTITIONS 4;`) + tk.MustGetErrCode("ALTER TABLE e EXCHANGE PARTITION p1 WITH TABLE e2;", tmysql.ErrPartitionExchangePartTable) + tk.MustExec("truncate tabl e2") + tk.MustExec(`INSERT INTO e3 VALUES (1),(5)`) + + tk.MustExec("ALTER TABLE e3 exchange partition p1 with tabl e2;") + tk.MustGetErrCode("ALTER TABLE e3 EXCHANGE PARTITION p0 WITH TABLE e2", tmysql.ErrRowDoesNotMatchPartition) + tk.MustGetErrCode("ALTER TABLE e3 EXCHANGE PARTITION p2 WITH TABLE e2", tmysql.ErrRowDoesNotMatchPartition) + tk.MustGetErrCode("ALTER TABLE e3 EXCHANGE PARTITION p3 WITH TABLE e2", tmysql.ErrRowDoesNotMatchPartition) + tk.MustExec("ALTER TABLE e3 EXCHANGE PARTITION p0 with TABLE e2 WITHOUT VALIDATION") + + tk.MustQuery("select * from e3 partition(p0)").Check(testkit.Rows("1", "5")) + tk.MustQuery("select * from e2").Check(testkit.Rows()) + } func (s *testIntegrationSuite4) TestAddPartitionTooManyPartitions(c *C) { From 7468924c8cc127efc0b5d1fdf767e7280fc3908e Mon Sep 17 00:00:00 2001 From: zhaox1n Date: Sat, 9 May 2020 20:50:52 +0800 Subject: [PATCH 09/57] format --- ddl/db_partition_test.go | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/ddl/db_partition_test.go b/ddl/db_partition_test.go index a1ee4bb5ef387..9e3e37f19c634 100644 --- a/ddl/db_partition_test.go +++ b/ddl/db_partition_test.go @@ -778,11 +778,7 @@ func (s *testIntegrationSuite5) TestAlterTableExchangePartition(c *C) { tk.MustExec(`CREATE TABLE e2 ( id INT NOT NULL );`) - tk.MustExec(`INSERT INTO e VALUES - (1669), - (337), - (16), - (2005)`) + tk.MustExec(`INSERT INTO e VALUES (1669),(337),(16),(2005)`) tk.MustExec("ALTER TABLE e EXCHANGE PARTITION p0 WITH TABLE e2") tk.MustQuery("select * from e2").Check(testkit.Rows("16")) tk.MustQuery("select * from e").Check(testkit.Rows("1669", "337", "2005")) From 64fd70912ba007eef990665d308ddc1c371fc20b Mon Sep 17 00:00:00 2001 From: zhaox1n Date: Sat, 9 May 2020 22:38:53 +0800 Subject: [PATCH 10/57] test add --- ddl/db_partition_test.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/ddl/db_partition_test.go b/ddl/db_partition_test.go index 9e3e37f19c634..c35f4b932053e 100644 --- a/ddl/db_partition_test.go +++ b/ddl/db_partition_test.go @@ -792,11 +792,11 @@ func (s *testIntegrationSuite5) TestAlterTableExchangePartition(c *C) { id int not null ) PARTITION BY HASH (id) PARTITIONS 4;`) - tk.MustGetErrCode("ALTER TABLE e EXCHANGE PARTITION p1 WITH TABLE e2;", tmysql.ErrPartitionExchangePartTable) - tk.MustExec("truncate tabl e2") + tk.MustGetErrCode("ALTER TABLE e EXCHANGE PARTITION p1 WITH TABLE e3;", tmysql.ErrPartitionExchangePartTable) + tk.MustExec("truncate table e2") tk.MustExec(`INSERT INTO e3 VALUES (1),(5)`) - tk.MustExec("ALTER TABLE e3 exchange partition p1 with tabl e2;") + tk.MustExec("ALTER TABLE e3 EXCHANGE PARTITION p1 WITH TABLE e2;") tk.MustGetErrCode("ALTER TABLE e3 EXCHANGE PARTITION p0 WITH TABLE e2", tmysql.ErrRowDoesNotMatchPartition) tk.MustGetErrCode("ALTER TABLE e3 EXCHANGE PARTITION p2 WITH TABLE e2", tmysql.ErrRowDoesNotMatchPartition) tk.MustGetErrCode("ALTER TABLE e3 EXCHANGE PARTITION p3 WITH TABLE e2", tmysql.ErrRowDoesNotMatchPartition) From 71cf840f6ad92a7426c833f4f03d2f6ba87284ca Mon Sep 17 00:00:00 2001 From: zhaox1n Date: Sun, 10 May 2020 17:08:39 +0800 Subject: [PATCH 11/57] test add --- ddl/ddl_algorithm_test.go | 1 + ddl/ddl_api.go | 2 +- ddl/ddl_worker_test.go | 22 ++++++++++++++++++++++ ddl/partition.go | 13 ++++++++----- ddl/rollingback.go | 3 ++- go.mod | 6 +++--- go.sum | 9 +++++++-- 7 files changed, 44 insertions(+), 12 deletions(-) diff --git a/ddl/ddl_algorithm_test.go b/ddl/ddl_algorithm_test.go index 0457c64ae8606..e0cc7fba286b8 100644 --- a/ddl/ddl_algorithm_test.go +++ b/ddl/ddl_algorithm_test.go @@ -60,6 +60,7 @@ func (s *testDDLAlgorithmSuite) TestFindAlterAlgorithm(c *C) { {ast.AlterTableSpec{Tp: ast.AlterTableAddPartitions}, instantAlgorithm, ast.AlgorithmTypeInstant}, {ast.AlterTableSpec{Tp: ast.AlterTableDropPartition}, instantAlgorithm, ast.AlgorithmTypeInstant}, {ast.AlterTableSpec{Tp: ast.AlterTableTruncatePartition}, instantAlgorithm, ast.AlgorithmTypeInstant}, + {ast.AlterTableSpec{Tp: ast.AlterTableExchangePartition}, instantAlgorithm, ast.AlgorithmTypeInstant}, // TODO: after we support lock a table, change the below case. {ast.AlterTableSpec{Tp: ast.AlterTableLock}, instantAlgorithm, ast.AlgorithmTypeInstant}, diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 3c9599aa36a42..2c327c63eb66e 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -2829,7 +2829,7 @@ func (d *ddl) ExchangeTableParition(ctx sessionctx.Context, ident ast.Ident, spe partName := spec.PartitionNames[0].L - _, err = getPartitionDef(ptMeta, partName) + _, err = tables.FindPartitionByName(ptMeta, partName) if err != nil { return errors.Trace(err) } diff --git a/ddl/ddl_worker_test.go b/ddl/ddl_worker_test.go index 6f1b13280e568..dede210da70c4 100644 --- a/ddl/ddl_worker_test.go +++ b/ddl/ddl_worker_test.go @@ -488,6 +488,9 @@ func buildCancelJobTests(firstID int64) []testCancelJob { {act: model.ActionAlterIndexVisibility, jobIDs: []int64{firstID + 46}, cancelRetErrs: noErrs, cancelState: model.StateNone}, {act: model.ActionAlterIndexVisibility, jobIDs: []int64{firstID + 47}, cancelRetErrs: []error{admin.ErrCancelFinishedDDLJob.GenWithStackByArgs(firstID + 47)}, cancelState: model.StatePublic}, + + {act: model.ActionExchangeTablePartition, jobIDs: []int64{firstID + 48}, cancelRetErrs: noErrs, cancelState: model.StateNone}, + {act: model.ActionExchangeTablePartition, jobIDs: []int64{firstID + 49}, cancelRetErrs: []error{admin.ErrCancelFinishedDDLJob.GenWithStackByArgs(firstID + 49)}, cancelState: model.StatePublic}, } return tests @@ -986,6 +989,25 @@ func (s *testDDLSuite) TestCancelJob(c *C) { c.Check(checkErr, IsNil) changedTable = testGetTable(c, d, dbInfo.ID, tblInfo.ID) c.Assert(checkIdxVisibility(changedTable, indexName, true), IsTrue) + + // test exchange partition failed caused by canceled + updateTest(&tests[43]) + exchangeTablePartition := []interface{}{dbInfo.ID, partitionTblInfo.ID, "p0", true} + doDDLJobErrWithSchemaState(ctx, d, c, dbInfo.ID, tblInfo.ID, test.act, exchangeTablePartition, &test.cancelState) + changedNtTable := testGetTable(c, d, dbInfo.ID, tblInfo.ID) + changedPtTable := testGetTable(c, d, dbInfo.ID, partitionTblInfo.ID) + c.Assert(changedNtTable.Meta().ID == tblInfo.ID, IsTrue) + c.Assert(changedPtTable.Meta().Partition.Definitions[0].ID == partitionTblInfo.Partition.Definitions[0].ID, IsTrue) + + // canel exchange partition successfully + updateTest(&tests[44]) + doDDLJobSuccess(ctx, d, c, dbInfo.ID, tblInfo.ID, test.act, exchangeTablePartition) + c.Check(checkErr, IsNil) + changedNtTable = testGetTable(c, d, dbInfo.ID, tblInfo.ID) + changedPtTable = testGetTable(c, d, dbInfo.ID, partitionTblInfo.ID) + c.Assert(changedNtTable.Meta().ID == tblInfo.ID, IsFalse) + c.Assert(changedPtTable.Meta().Partition.Definitions[0].ID == partitionTblInfo.Partition.Definitions[0].ID, IsFalse) + } func (s *testDDLSuite) TestIgnorableSpec(c *C) { diff --git a/ddl/partition.go b/ddl/partition.go index 78c6b2f90a771..b16ed338bbb17 100644 --- a/ddl/partition.go +++ b/ddl/partition.go @@ -706,18 +706,21 @@ func (w *worker) onExchangeTablePartition(d *ddlCtx, t *meta.Meta, job *model.Jo return ver, errors.Trace(err) } - ntDbInfo, err := t.GetDatabase(job.SchemaID) + nt, err := getTableInfoAndCancelFaultJob(t, job, job.SchemaID) if err != nil { - return ver, err + return ver, errors.Trace(err) } - nt, err := getTableInfoAndCancelFaultJob(t, job, job.SchemaID) + ntDbInfo, err := t.GetDatabase(job.SchemaID) if err != nil { - return ver, errors.Trace(err) + return ver, err } - pt, err := t.GetTable(ptSchemaID, ptID) + pt, err := getTableInfo(t, ptID, ptSchemaID) if err != nil { + if infoschema.ErrDatabaseNotExists.Equal(err) || infoschema.ErrTableNotExists.Equal(err) { + job.State = model.JobStateCancelled + } return ver, errors.Trace(err) } diff --git a/ddl/rollingback.go b/ddl/rollingback.go index 207b9218d1197..bed7c9f310ced 100644 --- a/ddl/rollingback.go +++ b/ddl/rollingback.go @@ -350,7 +350,8 @@ func convertJob2RollbackJob(w *worker, d *ddlCtx, t *meta.Meta, job *model.Job) model.ActionDropForeignKey, model.ActionRenameTable, model.ActionModifyTableCharsetAndCollate, model.ActionTruncateTablePartition, model.ActionModifySchemaCharsetAndCollate, model.ActionRepairTable, - model.ActionModifyTableAutoIdCache, model.ActionAlterIndexVisibility: + model.ActionModifyTableAutoIdCache, model.ActionAlterIndexVisibility, + model.ActionExchangeTablePartition: ver, err = cancelOnlyNotHandledJob(job) default: job.State = model.JobStateCancelled diff --git a/go.mod b/go.mod index 80a642c0cd383..a28a541f8b96f 100644 --- a/go.mod +++ b/go.mod @@ -30,7 +30,7 @@ require ( github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989 github.com/pingcap/kvproto v0.0.0-20200424032552-6650270c39c3 github.com/pingcap/log v0.0.0-20200117041106-d28c14d3b1cd - github.com/pingcap/parser v0.0.0-20200507022230-f3bf29096657 + github.com/pingcap/parser v3.1.1+incompatible github.com/pingcap/pd/v4 v4.0.0-rc.1.0.20200422143320-428acd53eba2 github.com/pingcap/sysutil v0.0.0-20200408114249-ed3bd6f7fdb1 github.com/pingcap/tidb-tools v4.0.0-rc.1.0.20200421113014-507d2bb3a15e+incompatible @@ -49,7 +49,7 @@ require ( go.etcd.io/etcd v0.5.0-alpha.5.0.20191023171146-3cf2f69b5738 go.uber.org/atomic v1.6.0 go.uber.org/automaxprocs v1.2.0 - go.uber.org/zap v1.14.1 + go.uber.org/zap v1.15.0 golang.org/x/net v0.0.0-20200324143707-d3edc9973b7e golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e golang.org/x/sys v0.0.0-20200323222414-85ca7c5b95cd @@ -63,4 +63,4 @@ require ( go 1.13 -replace github.com/pingcap/parser => github.com/zhaox1n/parser v0.0.0-20200430160105-6fcbbbfa6158 +replace github.com/pingcap/parser => github.com/zhaox1n/parser v0.0.0-20200510040745-d20d92603e29 diff --git a/go.sum b/go.sum index d67de5a9d91a4..661667a2cc097 100644 --- a/go.sum +++ b/go.sum @@ -370,8 +370,7 @@ github.com/pingcap/log v0.0.0-20200117041106-d28c14d3b1cd h1:CV3VsP3Z02MVtdpTMfE github.com/pingcap/log v0.0.0-20200117041106-d28c14d3b1cd/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= github.com/pingcap/log v0.0.0-20200117041106-d28c14d3b1cd/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= github.com/pingcap/parser v0.0.0-20200424075042-8222d8b724a4/go.mod h1:9v0Edh8IbgjGYW2ArJr19E+bvL8zKahsFp+ixWeId+4= -github.com/pingcap/parser v0.0.0-20200507022230-f3bf29096657 h1:2ceTso30kmgMeddZ4iZ6zrK8N9eFF8zmCa1hSSE1tXc= -github.com/pingcap/parser v0.0.0-20200507022230-f3bf29096657/go.mod h1:9v0Edh8IbgjGYW2ArJr19E+bvL8zKahsFp+ixWeId+4= +github.com/pingcap/parser v3.1.1+incompatible/go.mod h1:1FNvfp9+J0wvc4kl8eGNh7Rqrxveg15jJoWo/a0uHwA= github.com/pingcap/pd/v4 v4.0.0-rc.1.0.20200422143320-428acd53eba2 h1:JTzYYukREvxVSKW/ncrzNjFitd8snoQ/Xz32pw8i+s8= github.com/pingcap/pd/v4 v4.0.0-rc.1.0.20200422143320-428acd53eba2/go.mod h1:s+utZtXDznOiL24VK0qGmtoHjjXNsscJx3m1n8cC56s= github.com/pingcap/sysutil v0.0.0-20200206130906-2bfa6dc40bcd/go.mod h1:EB/852NMQ+aRKioCpToQ94Wl7fktV+FNnxf3CX/TTXI= @@ -504,6 +503,10 @@ github.com/yookoala/realpath v1.0.0/go.mod h1:gJJMA9wuX7AcqLy1+ffPatSCySA1FQ2S8Y github.com/yuin/goldmark v1.1.25/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/zhaox1n/parser v0.0.0-20200430160105-6fcbbbfa6158 h1:3EhundhszzvUGiWSvUzwW1wiM5YUVnih3EgIhJZ8a8s= github.com/zhaox1n/parser v0.0.0-20200430160105-6fcbbbfa6158/go.mod h1:9v0Edh8IbgjGYW2ArJr19E+bvL8zKahsFp+ixWeId+4= +github.com/zhaox1n/parser v0.0.0-20200510040745-d20d92603e29 h1:uI3SH5eGCzeTSi6Knl2YxsZgdyK9yI+Od//OYbc4Bdc= +github.com/zhaox1n/parser v0.0.0-20200510040745-d20d92603e29/go.mod h1:9v0Edh8IbgjGYW2ArJr19E+bvL8zKahsFp+ixWeId+4= +github.com/zhaox1n/parser v3.0.12+incompatible h1:4HqB/EJuvIasw1Io39UX1UM2FutJ2UTsNm1wptuxeGw= +github.com/zhaox1n/parser v3.0.12+incompatible/go.mod h1:/xFSvLQ9LsJ9t+fZfxb6/FqVrulSjEb5glcqR4fmZao= go.etcd.io/bbolt v1.3.3 h1:MUGmc65QhB3pIlaQ5bB4LwqSj6GIonVJXpZiaKNyaKk= go.etcd.io/bbolt v1.3.3/go.mod h1:IbVyRI1SCnLcuJnV2u8VeU0CEYM7e686BmAb1XKL+uU= go.etcd.io/etcd v0.0.0-20191023171146-3cf2f69b5738/go.mod h1:dnLIgRNXwCJa5e+c6mIZCrds/GIG4ncV9HhK5PX7jPg= @@ -537,6 +540,8 @@ go.uber.org/zap v1.12.0/go.mod h1:zwrFLgMcdUuIBviXEYEH1YKNaOBnKXsx2IPda5bBwHM= go.uber.org/zap v1.13.0/go.mod h1:zwrFLgMcdUuIBviXEYEH1YKNaOBnKXsx2IPda5bBwHM= go.uber.org/zap v1.14.1 h1:nYDKopTbvAPq/NrUVZwT15y2lpROBiLLyoRTbXOYWOo= go.uber.org/zap v1.14.1/go.mod h1:Mb2vm2krFEG5DV0W9qcHBYFtp/Wku1cvYaqPsS/WYfc= +go.uber.org/zap v1.15.0 h1:ZZCA22JRF2gQE5FoNmhmrf7jeJJ2uhqDUNRYKm8dvmM= +go.uber.org/zap v1.15.0/go.mod h1:Mb2vm2krFEG5DV0W9qcHBYFtp/Wku1cvYaqPsS/WYfc= golang.org/x/crypto v0.0.0-20180904163835-0709b304e793/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20181203042331-505ab145d0a9/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= From a902cdc07f7c87149fe98f0cc7eba1012b8e5cb2 Mon Sep 17 00:00:00 2001 From: zhaox1n Date: Mon, 11 May 2020 00:42:52 +0800 Subject: [PATCH 12/57] auto id fix --- ddl/ddl_worker_test.go | 35 ++++++++++++++++++++++------------- ddl/partition.go | 30 +++++++++++++++--------------- 2 files changed, 37 insertions(+), 28 deletions(-) diff --git a/ddl/ddl_worker_test.go b/ddl/ddl_worker_test.go index dede210da70c4..5b8fefd7c84df 100644 --- a/ddl/ddl_worker_test.go +++ b/ddl/ddl_worker_test.go @@ -375,10 +375,11 @@ func doDDLJobErrWithSchemaState(ctx sessionctx.Context, d *ddl, c *C, schemaID, Args: args, BinlogInfo: &model.HistoryInfo{}, } - err := d.doDDLJob(ctx, job) + // err := d.doDDLJob(ctx, job) + d.doDDLJob(ctx, job) // TODO: Add the detail error check. - c.Assert(err, NotNil, Commentf("err:%v", err)) - testCheckJobCancelled(c, d, job, state) + // c.Assert(err, NotNil, Commentf("err:%v", err)) + // testCheckJobCancelled(c, d, job, state) return job } @@ -991,22 +992,30 @@ func (s *testDDLSuite) TestCancelJob(c *C) { c.Assert(checkIdxVisibility(changedTable, indexName, true), IsTrue) // test exchange partition failed caused by canceled + pt := testTableInfoWithPartition(c, d, "pt", 5) + nt := testTableInfo(c, d, "nt", 5) + testCreateTable(c, ctx, d, dbInfo, pt) + testCreateTable(c, ctx, d, dbInfo, nt) + updateTest(&tests[43]) - exchangeTablePartition := []interface{}{dbInfo.ID, partitionTblInfo.ID, "p0", true} - doDDLJobErrWithSchemaState(ctx, d, c, dbInfo.ID, tblInfo.ID, test.act, exchangeTablePartition, &test.cancelState) - changedNtTable := testGetTable(c, d, dbInfo.ID, tblInfo.ID) - changedPtTable := testGetTable(c, d, dbInfo.ID, partitionTblInfo.ID) + exchangeTablePartition := []interface{}{dbInfo.ID, pt.ID, "p0", true} + c.Assert(test.act, Equals, model.ActionExchangeTablePartition) + job6 := doDDLJobErrWithSchemaState(ctx, d, c, dbInfo.ID, nt.ID, test.act, exchangeTablePartition, &test.cancelState) + c.Assert(job6.ID, Equals, test.jobIDs[0]) + c.Check(checkErr, IsNil) + changedNtTable := testGetTable(c, d, dbInfo.ID, nt.ID) + changedPtTable := testGetTable(c, d, dbInfo.ID, pt.ID) c.Assert(changedNtTable.Meta().ID == tblInfo.ID, IsTrue) - c.Assert(changedPtTable.Meta().Partition.Definitions[0].ID == partitionTblInfo.Partition.Definitions[0].ID, IsTrue) + c.Assert(changedPtTable.Meta().Partition.Definitions[0].ID == pt.Partition.Definitions[0].ID, IsTrue) // canel exchange partition successfully updateTest(&tests[44]) - doDDLJobSuccess(ctx, d, c, dbInfo.ID, tblInfo.ID, test.act, exchangeTablePartition) + doDDLJobSuccess(ctx, d, c, dbInfo.ID, nt.ID, test.act, exchangeTablePartition) c.Check(checkErr, IsNil) - changedNtTable = testGetTable(c, d, dbInfo.ID, tblInfo.ID) - changedPtTable = testGetTable(c, d, dbInfo.ID, partitionTblInfo.ID) - c.Assert(changedNtTable.Meta().ID == tblInfo.ID, IsFalse) - c.Assert(changedPtTable.Meta().Partition.Definitions[0].ID == partitionTblInfo.Partition.Definitions[0].ID, IsFalse) + changedNtTable = testGetTable(c, d, dbInfo.ID, pt.Partition.Definitions[0].ID) + changedPtTable = testGetTable(c, d, dbInfo.ID, pt.ID) + c.Assert(changedNtTable.Meta().ID == nt.ID, IsFalse) + c.Assert(changedPtTable.Meta().Partition.Definitions[0].ID == nt.ID, IsTrue) } diff --git a/ddl/partition.go b/ddl/partition.go index b16ed338bbb17..742c7ebb6d281 100644 --- a/ddl/partition.go +++ b/ddl/partition.go @@ -701,19 +701,20 @@ func (w *worker) onExchangeTablePartition(d *ddlCtx, t *meta.Meta, job *model.Jo partName string withValidation bool ) + if err := job.DecodeArgs(&ptSchemaID, &ptID, &partName, &withValidation); err != nil { job.State = model.JobStateCancelled return ver, errors.Trace(err) } - nt, err := getTableInfoAndCancelFaultJob(t, job, job.SchemaID) + ntDbInfo, err := t.GetDatabase(job.SchemaID) if err != nil { return ver, errors.Trace(err) } - ntDbInfo, err := t.GetDatabase(job.SchemaID) + nt, err := getTableInfoAndCancelFaultJob(t, job, job.SchemaID) if err != nil { - return ver, err + return ver, errors.Trace(err) } pt, err := getTableInfo(t, ptID, ptSchemaID) @@ -724,12 +725,6 @@ func (w *worker) onExchangeTablePartition(d *ddlCtx, t *meta.Meta, job *model.Jo return ver, errors.Trace(err) } - partDef, err := getPartitionDef(pt, partName) - if err != nil { - job.State = model.JobStateCancelled - return ver, errors.Trace(err) - } - err = checkTableDefCompatible(pt, nt) if err != nil { job.State = model.JobStateCancelled @@ -756,17 +751,16 @@ func (w *worker) onExchangeTablePartition(d *ddlCtx, t *meta.Meta, job *model.Jo return ver, errors.Trace(err) } - tempID := partDef.ID - - partDef.ID = nt.ID - - err = t.UpdateTable(ptSchemaID, pt) + partDef, err := getPartitionDef(pt, partName) if err != nil { job.State = model.JobStateCancelled return ver, errors.Trace(err) } - _, err = t.GenAutoTableID(ptSchemaID, partDef.ID, ntBaseID) + tempID := partDef.ID + partDef.ID = nt.ID + + err = t.UpdateTable(ptSchemaID, pt) if err != nil { job.State = model.JobStateCancelled return ver, errors.Trace(err) @@ -786,6 +780,12 @@ func (w *worker) onExchangeTablePartition(d *ddlCtx, t *meta.Meta, job *model.Jo return ver, errors.Trace(err) } + _, err = t.GenAutoTableID(ptSchemaID, pt.ID, ntBaseID) + if err != nil { + job.State = model.JobStateCancelled + return ver, errors.Trace(err) + } + _, err = t.GenAutoTableID(job.SchemaID, nt.ID, ptBaseID) if err != nil { job.State = model.JobStateCancelled From a228468f18573de1806b3ba5a6ce7e50008b04ea Mon Sep 17 00:00:00 2001 From: zhaox1n Date: Mon, 11 May 2020 01:26:33 +0800 Subject: [PATCH 13/57] add cancel job test --- ddl/ddl_worker_test.go | 17 +++++++---------- 1 file changed, 7 insertions(+), 10 deletions(-) diff --git a/ddl/ddl_worker_test.go b/ddl/ddl_worker_test.go index 5b8fefd7c84df..1d143b8aeec00 100644 --- a/ddl/ddl_worker_test.go +++ b/ddl/ddl_worker_test.go @@ -375,11 +375,10 @@ func doDDLJobErrWithSchemaState(ctx sessionctx.Context, d *ddl, c *C, schemaID, Args: args, BinlogInfo: &model.HistoryInfo{}, } - // err := d.doDDLJob(ctx, job) - d.doDDLJob(ctx, job) + err := d.doDDLJob(ctx, job) // TODO: Add the detail error check. - // c.Assert(err, NotNil, Commentf("err:%v", err)) - // testCheckJobCancelled(c, d, job, state) + c.Assert(err, NotNil, Commentf("err:%v", err)) + testCheckJobCancelled(c, d, job, state) return job } @@ -490,8 +489,8 @@ func buildCancelJobTests(firstID int64) []testCancelJob { {act: model.ActionAlterIndexVisibility, jobIDs: []int64{firstID + 46}, cancelRetErrs: noErrs, cancelState: model.StateNone}, {act: model.ActionAlterIndexVisibility, jobIDs: []int64{firstID + 47}, cancelRetErrs: []error{admin.ErrCancelFinishedDDLJob.GenWithStackByArgs(firstID + 47)}, cancelState: model.StatePublic}, - {act: model.ActionExchangeTablePartition, jobIDs: []int64{firstID + 48}, cancelRetErrs: noErrs, cancelState: model.StateNone}, - {act: model.ActionExchangeTablePartition, jobIDs: []int64{firstID + 49}, cancelRetErrs: []error{admin.ErrCancelFinishedDDLJob.GenWithStackByArgs(firstID + 49)}, cancelState: model.StatePublic}, + {act: model.ActionExchangeTablePartition, jobIDs: []int64{firstID + 53}, cancelRetErrs: noErrs, cancelState: model.StateNone}, + {act: model.ActionExchangeTablePartition, jobIDs: []int64{firstID + 54}, cancelRetErrs: []error{admin.ErrCancelFinishedDDLJob.GenWithStackByArgs(firstID + 54)}, cancelState: model.StatePublic}, } return tests @@ -999,13 +998,11 @@ func (s *testDDLSuite) TestCancelJob(c *C) { updateTest(&tests[43]) exchangeTablePartition := []interface{}{dbInfo.ID, pt.ID, "p0", true} - c.Assert(test.act, Equals, model.ActionExchangeTablePartition) - job6 := doDDLJobErrWithSchemaState(ctx, d, c, dbInfo.ID, nt.ID, test.act, exchangeTablePartition, &test.cancelState) - c.Assert(job6.ID, Equals, test.jobIDs[0]) + doDDLJobErrWithSchemaState(ctx, d, c, dbInfo.ID, nt.ID, test.act, exchangeTablePartition, &test.cancelState) c.Check(checkErr, IsNil) changedNtTable := testGetTable(c, d, dbInfo.ID, nt.ID) changedPtTable := testGetTable(c, d, dbInfo.ID, pt.ID) - c.Assert(changedNtTable.Meta().ID == tblInfo.ID, IsTrue) + c.Assert(changedNtTable.Meta().ID == nt.ID, IsTrue) c.Assert(changedPtTable.Meta().Partition.Definitions[0].ID == pt.Partition.Definitions[0].ID, IsTrue) // canel exchange partition successfully From 9ff0790da72b7cad53030b182c2dab26d9ffe2f4 Mon Sep 17 00:00:00 2001 From: zhaox1n Date: Tue, 12 May 2020 01:26:11 +0800 Subject: [PATCH 14/57] add exchange partition compatiable test --- ddl/db_partition_test.go | 46 ++++++++++++++++++++++++++++++++++++++++ ddl/ddl_api.go | 21 +++++++++++++++++- 2 files changed, 66 insertions(+), 1 deletion(-) diff --git a/ddl/db_partition_test.go b/ddl/db_partition_test.go index c35f4b932053e..52bbd4443d0fb 100644 --- a/ddl/db_partition_test.go +++ b/ddl/db_partition_test.go @@ -807,6 +807,52 @@ func (s *testIntegrationSuite5) TestAlterTableExchangePartition(c *C) { } +func (s *testIntegrationSuite4) TestExchangePartitionTableCompatiable(c *C) { + + type testCase struct { + pt string + nt string + exchange string + ErrTablesDifferentMetadata bool + } + cases := []testCase{ + { + "create table pt (id int not null) partition by hash (id) partitions 4;", + "create table nt (id int(1) not null);", + "alter table pt exchange partition p0 with table nt;", + false, + }, + { + "create table pt1 (id int not null, fname varchar(3)) partition by hash (id) partitions 4;", + "create table nt1 (id int not null, fname varchar(4));", + "alter table pt1 exchange partition p0 with table nt1;", + true, + }, + { + "create table pt2 (id int not null, salary decimal) partition by hash(id) partitions 4", + "create table nt2 (id int not null, salary decimal(3,2));", + "alter table pt2 exchange partition p0 with table nt2;", + false, + }, + } + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + for i, t := range cases { + tk.MustExec(t.pt) + tk.MustExec(t.nt) + if t.ErrTablesDifferentMetadata { + _, err := tk.Exec(t.exchange) + c.Assert(ddl.ErrTablesDifferentMetadata.Equal(err), IsTrue, Commentf( + "case %d fail, sql = `%s`\nexpected error = `%v`\n actual error = `%v`", + i, t.exchange, ddl.ErrTablesDifferentMetadata, err, + )) + } else { + tk.MustExec(t.exchange) + } + } + +} + func (s *testIntegrationSuite4) TestAddPartitionTooManyPartitions(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 2c327c63eb66e..b81fca648b848 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -2740,6 +2740,25 @@ func (d *ddl) DropTablePartition(ctx sessionctx.Context, ident ast.Ident, spec * return errors.Trace(err) } +func checkFielTypeCompatible(ft *types.FieldType, other *types.FieldType) bool { + // int(1) could match the type with int(8) + partialEqual := ft.Tp == other.Tp && + ft.Decimal == other.Decimal && + ft.Charset == other.Charset && + ft.Collate == other.Collate && + (ft.Flen == other.Flen || (ft.StorageLength() == other.StorageLength() && ft.StorageLength() != types.VarStorageLen && other.StorageLength() != types.VarStorageLen)) && + mysql.HasUnsignedFlag(ft.Flag) == mysql.HasUnsignedFlag(other.Flag) + if !partialEqual || len(ft.Elems) != len(other.Elems) { + return false + } + for i := range ft.Elems { + if ft.Elems[i] != other.Elems[i] { + return false + } + } + return true +} + func checkTableDefCompatible(source *model.TableInfo, target *model.TableInfo) error { err := ErrTablesDifferentMetadata if len(source.Cols()) != len(target.Cols()) { @@ -2750,7 +2769,7 @@ func checkTableDefCompatible(source *model.TableInfo, target *model.TableInfo) e targetCol := target.Cols()[i] if sourceCol.IsGenerated() != targetCol.IsGenerated() || sourceCol.Name.L != targetCol.Name.L || - !sourceCol.FieldType.Equal(&targetCol.FieldType) { + !checkFielTypeCompatible(&sourceCol.FieldType, &targetCol.FieldType) { return err } } From dda9d753fea16d0fade732a7dc8becf01800bfc2 Mon Sep 17 00:00:00 2001 From: zhaox1n Date: Wed, 13 May 2020 00:34:24 +0800 Subject: [PATCH 15/57] add test of partition --- ddl/db_partition_test.go | 58 ++++++++++++++++++++++++++++++++-------- ddl/ddl_api.go | 5 ++-- 2 files changed, 49 insertions(+), 14 deletions(-) diff --git a/ddl/db_partition_test.go b/ddl/db_partition_test.go index 52bbd4443d0fb..5569290c48fbd 100644 --- a/ddl/db_partition_test.go +++ b/ddl/db_partition_test.go @@ -810,10 +810,10 @@ func (s *testIntegrationSuite5) TestAlterTableExchangePartition(c *C) { func (s *testIntegrationSuite4) TestExchangePartitionTableCompatiable(c *C) { type testCase struct { - pt string - nt string - exchange string - ErrTablesDifferentMetadata bool + ptSQL string + ntSQL string + exchangeSQL string + err bool } cases := []testCase{ { @@ -829,25 +829,61 @@ func (s *testIntegrationSuite4) TestExchangePartitionTableCompatiable(c *C) { true, }, { - "create table pt2 (id int not null, salary decimal) partition by hash(id) partitions 4", + "create table pt2 (id int not null, salary decimal) partition by hash(id) partitions 4;", "create table nt2 (id int not null, salary decimal(3,2));", "alter table pt2 exchange partition p0 with table nt2;", + true, + }, + { + "create table pt3 (id int not null, salary decimal) partition by hash(id) partitions 1;", + "create table nt3 (id int not null, salary decimal(10, 1));", + "alter table pt3 exchange partition p0 with table nt3", + true, + }, + { + "create table pt4 (id int not null) partition by hash(id) partitions 1;", + "create table nt4 (id1 int not null);", + "alter table pt4 exchange partition p0 with table nt4;", + true, + }, + { + "create table pt5 (id int not null, primary key (id)) partition by hash(id) partitions 1;", + "create table nt5 (id int not null);", + "alter table pt5 exchange partition p0 with table nt5;", + true, + }, + { + "create table pt6 (id int not null, salary decimal, index idx (id, salary)) partition by hash(id) partitions 1;", + "create table nt6 (id int not null, salary decimal, index idx (salary, id));", + "alter table pt6 exchange partition p0 with table nt6;", + true, + }, + { + "create table pt7 (id int not null, index idx (id) invisible) partition by hash(id) partitions 1;", + "create table nt7 (id int not null, index idx (id));", + "alter table pt7 exchange partition p0 with table nt7;", false, }, + { + "create table pt8 (id int not null, index idx (id)) partition by hash(id) partitions 1;", + "create table nt8 (id int not null, index id_idx (id));", + "alter table pt8 exchange partition p0 with table nt8;", + true, + }, } tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") for i, t := range cases { - tk.MustExec(t.pt) - tk.MustExec(t.nt) - if t.ErrTablesDifferentMetadata { - _, err := tk.Exec(t.exchange) + tk.MustExec(t.ptSQL) + tk.MustExec(t.ntSQL) + if t.err { + _, err := tk.Exec(t.exchangeSQL) c.Assert(ddl.ErrTablesDifferentMetadata.Equal(err), IsTrue, Commentf( "case %d fail, sql = `%s`\nexpected error = `%v`\n actual error = `%v`", - i, t.exchange, ddl.ErrTablesDifferentMetadata, err, + i, t.exchangeSQL, ddl.ErrTablesDifferentMetadata, err, )) } else { - tk.MustExec(t.exchange) + tk.MustExec(t.exchangeSQL) } } diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index b81fca648b848..2f4dd5e2b4511 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -2746,7 +2746,7 @@ func checkFielTypeCompatible(ft *types.FieldType, other *types.FieldType) bool { ft.Decimal == other.Decimal && ft.Charset == other.Charset && ft.Collate == other.Collate && - (ft.Flen == other.Flen || (ft.StorageLength() == other.StorageLength() && ft.StorageLength() != types.VarStorageLen && other.StorageLength() != types.VarStorageLen)) && + (ft.Flen == other.Flen || ft.StorageLength() != types.VarStorageLen) && mysql.HasUnsignedFlag(ft.Flag) == mysql.HasUnsignedFlag(other.Flag) if !partialEqual || len(ft.Elems) != len(other.Elems) { return false @@ -2790,8 +2790,7 @@ func checkTableDefCompatible(source *model.TableInfo, target *model.TableInfo) e // Index type is not compatiable if sourceIdx.Tp != compatIdx.Tp || sourceIdx.Unique != compatIdx.Unique || - sourceIdx.Primary != compatIdx.Primary || - sourceIdx.Invisible != compatIdx.Invisible { + sourceIdx.Primary != compatIdx.Primary { return err } // The index column From 32c62854dc962236a8a48ea0690f8f51b0e07c88 Mon Sep 17 00:00:00 2001 From: zhaox1n Date: Thu, 14 May 2020 02:20:45 +0800 Subject: [PATCH 16/57] add more test and set go mod --- ddl/db_partition_test.go | 98 +++++++++++++++++++++++++++++++++------- ddl/ddl_api.go | 42 ++++++++++++----- ddl/error.go | 8 +++- ddl/generated_column.go | 6 +-- ddl/index.go | 2 +- go.mod | 8 ++-- go.sum | 13 +++--- 7 files changed, 130 insertions(+), 47 deletions(-) diff --git a/ddl/db_partition_test.go b/ddl/db_partition_test.go index 5569290c48fbd..d13e2486e221d 100644 --- a/ddl/db_partition_test.go +++ b/ddl/db_partition_test.go @@ -797,6 +797,9 @@ func (s *testIntegrationSuite5) TestAlterTableExchangePartition(c *C) { tk.MustExec(`INSERT INTO e3 VALUES (1),(5)`) tk.MustExec("ALTER TABLE e3 EXCHANGE PARTITION p1 WITH TABLE e2;") + tk.MustQuery("select * from e3 partition(p0)").Check(testkit.Rows()) + tk.MustQuery("select * from e2").Check(testkit.Rows("1", "5")) + tk.MustGetErrCode("ALTER TABLE e3 EXCHANGE PARTITION p0 WITH TABLE e2", tmysql.ErrRowDoesNotMatchPartition) tk.MustGetErrCode("ALTER TABLE e3 EXCHANGE PARTITION p2 WITH TABLE e2", tmysql.ErrRowDoesNotMatchPartition) tk.MustGetErrCode("ALTER TABLE e3 EXCHANGE PARTITION p3 WITH TABLE e2", tmysql.ErrRowDoesNotMatchPartition) @@ -813,75 +816,136 @@ func (s *testIntegrationSuite4) TestExchangePartitionTableCompatiable(c *C) { ptSQL string ntSQL string exchangeSQL string - err bool + err *terror.Error } cases := []testCase{ { "create table pt (id int not null) partition by hash (id) partitions 4;", "create table nt (id int(1) not null);", "alter table pt exchange partition p0 with table nt;", - false, + nil, }, { "create table pt1 (id int not null, fname varchar(3)) partition by hash (id) partitions 4;", "create table nt1 (id int not null, fname varchar(4));", "alter table pt1 exchange partition p0 with table nt1;", - true, + ddl.ErrTablesDifferentMetadata, }, { "create table pt2 (id int not null, salary decimal) partition by hash(id) partitions 4;", "create table nt2 (id int not null, salary decimal(3,2));", "alter table pt2 exchange partition p0 with table nt2;", - true, + ddl.ErrTablesDifferentMetadata, }, { "create table pt3 (id int not null, salary decimal) partition by hash(id) partitions 1;", "create table nt3 (id int not null, salary decimal(10, 1));", "alter table pt3 exchange partition p0 with table nt3", - true, + ddl.ErrTablesDifferentMetadata, }, { "create table pt4 (id int not null) partition by hash(id) partitions 1;", "create table nt4 (id1 int not null);", "alter table pt4 exchange partition p0 with table nt4;", - true, + ddl.ErrTablesDifferentMetadata, }, { "create table pt5 (id int not null, primary key (id)) partition by hash(id) partitions 1;", "create table nt5 (id int not null);", "alter table pt5 exchange partition p0 with table nt5;", - true, + ddl.ErrTablesDifferentMetadata, }, { "create table pt6 (id int not null, salary decimal, index idx (id, salary)) partition by hash(id) partitions 1;", "create table nt6 (id int not null, salary decimal, index idx (salary, id));", "alter table pt6 exchange partition p0 with table nt6;", - true, + ddl.ErrTablesDifferentMetadata, }, { "create table pt7 (id int not null, index idx (id) invisible) partition by hash(id) partitions 1;", "create table nt7 (id int not null, index idx (id));", "alter table pt7 exchange partition p0 with table nt7;", - false, + nil, }, { "create table pt8 (id int not null, index idx (id)) partition by hash(id) partitions 1;", "create table nt8 (id int not null, index id_idx (id));", "alter table pt8 exchange partition p0 with table nt8;", - true, + ddl.ErrTablesDifferentMetadata, + }, + { + // foreign key test + // Partition table is yet not supports to add foreign keys in mysql + "create table pt9 (id int not null primary key auto_increment,t_id int not null) partition by hash(id) partitions 1;", + "create table nt9 (id int not null primary key auto_increment, t_id int not null,foreign key fk_id (t_id) references pt5(id));", + "alter table pt9 exchange partition p0 with table pt9;", + ddl.ErrPartitionExchangeForeignKey, + }, + { + // Generated column (virtual) + "create table pt10 (id int not null, lname varchar(30), fname varchar(100) generated always as (concat(lname,' ')) virtual) partition by hash(id) partitions 1;", + "create table nt10 (id int not null, lname varchar(30), fname varchar(100));", + "alter table pt10 exchange partition p0 with table nt10;", + ddl.ErrUnsupportedOnGeneratedColumn, + }, + { + "create table pt11 (id int not null, lname varchar(30), fname varchar(100)) partition by hash(id) partitions 1;", + "create table nt11 (id int not null, lname varchar(30), fname varchar(100) generated always as (concat(lname, ' ')) virtual);", + "alter table pt11 exchange partition p0 with table nt11;", + ddl.ErrUnsupportedOnGeneratedColumn, + }, + { + + "create table pt12 (id int not null, lname varchar(30), fname varchar(100) generated always as (concat(lname,' ')) stored) partition by hash(id) partitions 1;", + "create table nt12 (id int not null, lname varchar(30), fname varchar(100));", + "alter table pt12 exchange partition p0 with table nt12;", + nil, + }, + { + "create table pt13 (id int not null, lname varchar(30), fname varchar(100)) partition by hash(id) partitions 1;", + "create table nt13 (id int not null, lname varchar(30), fname varchar(100) generated always as (concat(lname, ' ')) stored);", + "alter table pt13 exchange partition p0 with table nt13;", + nil, + }, + { + "create table pt14 (id int not null, lname varchar(30), fname varchar(100) generated always as (concat(lname, ' ')) virtual) partition by hash(id) partitions 1;", + "create table nt14 (id int not null, lname varchar(30), fname varchar(100) generated always as (concat(lname, ' ')) virtual);", + "alter table pt14 exchange partition p0 with table nt14;", + nil, + }, + { + "create table pt15 (id int not null, unique index uk_id (id)) partition by hash(id) partitions 1;", + "create table nt15 (id int not null, index uk_id (id));", + "alter table pt15 exchange partition p0 with table nt15", + ddl.ErrTablesDifferentMetadata, + }, + { + "create table pt16 (id int not null primary key auto_increment) partition by hash(id) partitions 1;", + "create table nt16 (id int not null primary key);", + "alter table pt16 exchange partition p0 with table nt16;", + ddl.ErrTablesDifferentMetadata, + }, + { + "create table pt17 (id int not null default 1) partition by hash(id) partitions 1;", + "create table nt17 (id int not null);", + "alter table pt17 exchange partition p0 with table nt17;", + nil, + }, + { + "create table pt18 (id int not null) partition by hash(id) partitions 1;", + "create view nt18 as select id from nt17;", + "alter table pt18 exchange partition p0 with table nt18", + ddl.ErrCheckNoSuchTable, }, } + tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") - for i, t := range cases { + for _, t := range cases { tk.MustExec(t.ptSQL) tk.MustExec(t.ntSQL) - if t.err { - _, err := tk.Exec(t.exchangeSQL) - c.Assert(ddl.ErrTablesDifferentMetadata.Equal(err), IsTrue, Commentf( - "case %d fail, sql = `%s`\nexpected error = `%v`\n actual error = `%v`", - i, t.exchangeSQL, ddl.ErrTablesDifferentMetadata, err, - )) + if t.err != nil { + tk.MustGetErrCode(t.exchangeSQL, t.err.Code()) } else { tk.MustExec(t.exchangeSQL) } diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 2f4dd5e2b4511..a274c3ed9b8ab 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -2391,7 +2391,7 @@ func checkAndCreateNewColumn(ctx sessionctx.Context, ti ast.Ident, schema *model } if option.Stored { - return nil, errUnsupportedOnGeneratedColumn.GenWithStackByArgs("Adding generated stored column through ALTER TABLE") + return nil, ErrUnsupportedOnGeneratedColumn.GenWithStackByArgs("Adding generated stored column through ALTER TABLE") } _, dependColNames := findDependedColumnNames(specNewColumn) @@ -2767,8 +2767,10 @@ func checkTableDefCompatible(source *model.TableInfo, target *model.TableInfo) e // Col compatible check for i, sourceCol := range source.Cols() { targetCol := target.Cols()[i] - if sourceCol.IsGenerated() != targetCol.IsGenerated() || - sourceCol.Name.L != targetCol.Name.L || + if sourceCol.GeneratedStored != targetCol.GeneratedStored { + return ErrUnsupportedOnGeneratedColumn.GenWithStackByArgs("Exchanging partitions for non-generated columns") + } + if sourceCol.Name.L != targetCol.Name.L || !checkFielTypeCompatible(&sourceCol.FieldType, &targetCol.FieldType) { return err } @@ -2809,6 +2811,26 @@ func checkTableDefCompatible(source *model.TableInfo, target *model.TableInfo) e return nil } +func checkExchangePartition(pt *model.TableInfo, nt *model.TableInfo) error { + + if nt.IsView() || nt.IsSequence() { + return errors.Trace(ErrCheckNoSuchTable) + } + if pt.GetPartitionInfo() == nil { + return errors.Trace(ErrPartitionMgmtOnNonpartitioned) + } + if nt.GetPartitionInfo() != nil { + return errors.Trace(ErrPartitionExchangePartTable.GenWithStackByArgs(nt.Name)) + } + + if nt.ForeignKeys != nil { + return errors.Trace(ErrPartitionExchangeForeignKey.GenWithStackByArgs(nt.Name)) + } + + // NOTE: if nt is temporary table, it should be checked + return nil +} + func (d *ddl) ExchangeTableParition(ctx sessionctx.Context, ident ast.Ident, spec *ast.AlterTableSpec) error { is := d.infoHandle.Get() ptSchema, ok := is.SchemaByName(ident.Schema) @@ -2836,24 +2858,20 @@ func (d *ddl) ExchangeTableParition(ctx sessionctx.Context, ident ast.Ident, spe ntMeta := nt.Meta() - if ptMeta.GetPartitionInfo() == nil { - return errors.Trace(ErrPartitionMgmtOnNonpartitioned) - } - if ntMeta.GetPartitionInfo() != nil { - return errors.Trace(ErrPartitionExchangePartTable.GenWithStackByArgs(ntIdent.Name)) + err = checkExchangePartition(ptMeta, ntMeta) + if err != nil { + return errors.Trace(err) } - // NOTE: if nt is temporary table, it should be checked - partName := spec.PartitionNames[0].L + // NOTE: if pt is subPartitioned, it should be checked + _, err = tables.FindPartitionByName(ptMeta, partName) if err != nil { return errors.Trace(err) } - // NOTE: if pt is subPartitioned, it should be checked - err = checkTableDefCompatible(ptMeta, ntMeta) if err != nil { return errors.Trace(err) diff --git a/ddl/error.go b/ddl/error.go index 50404cb4409fe..63f2a184d1720 100644 --- a/ddl/error.go +++ b/ddl/error.go @@ -65,7 +65,7 @@ var ( // errWrongFKOptionForGeneratedColumn is for wrong foreign key reference option on generated columns. errWrongFKOptionForGeneratedColumn = terror.ClassDDL.New(mysql.ErrWrongFKOptionForGeneratedColumn, mysql.MySQLErrName[mysql.ErrWrongFKOptionForGeneratedColumn]) // errUnsupportedOnGeneratedColumn is for unsupported actions on generated columns. - errUnsupportedOnGeneratedColumn = terror.ClassDDL.New(mysql.ErrUnsupportedOnGeneratedColumn, mysql.MySQLErrName[mysql.ErrUnsupportedOnGeneratedColumn]) + ErrUnsupportedOnGeneratedColumn = terror.ClassDDL.New(mysql.ErrUnsupportedOnGeneratedColumn, mysql.MySQLErrName[mysql.ErrUnsupportedOnGeneratedColumn]) // errGeneratedColumnNonPrior forbids to refer generated column non prior to it. errGeneratedColumnNonPrior = terror.ClassDDL.New(mysql.ErrGeneratedColumnNonPrior, mysql.MySQLErrName[mysql.ErrGeneratedColumnNonPrior]) // errDependentByGeneratedColumn forbids to delete columns which are dependent by generated columns. @@ -206,6 +206,10 @@ var ( ErrPartitionExchangePartTable = terror.ClassDDL.New(mysql.ErrPartitionExchangePartTable, mysql.MySQLErrName[mysql.ErrPartitionExchangePartTable]) // ErrTableDifferentMetadata is returned when exchanges tables is not compatible ErrTablesDifferentMetadata = terror.ClassDDL.New(mysql.ErrTablesDifferentMetadata, mysql.MySQLErrName[mysql.ErrTablesDifferentMetadata]) - // ErrRowDoesNotMatchPartition is return when the row record of exchange table does not match the partition rule + // ErrRowDoesNotMatchPartition is returned when the row record of exchange table does not match the partition rule ErrRowDoesNotMatchPartition = terror.ClassDDL.New(mysql.ErrRowDoesNotMatchPartition, mysql.MySQLErrName[mysql.ErrRowDoesNotMatchPartition]) + // ErrPartitionExchangeForeignKey is returned when exchanged non-partition table has foreign keys + ErrPartitionExchangeForeignKey = terror.ClassDDL.New(mysql.ErrPartitionExchangeForeignKey, mysql.MySQLErrName[mysql.ErrPartitionExchangeForeignKey]) + // ErrCheckNoSunchTable is returned when exchaned non-partition table is view or sequence + ErrCheckNoSuchTable = terror.ClassDDL.New(mysql.ErrCheckNoSuchTable, mysql.MySQLErrName[mysql.ErrCheckNoSuchTable]) ) diff --git a/ddl/generated_column.go b/ddl/generated_column.go index 0ea41d8691c2e..1e6b4480068a5 100644 --- a/ddl/generated_column.go +++ b/ddl/generated_column.go @@ -165,7 +165,7 @@ func checkModifyGeneratedColumn(tbl table.Table, oldCol, newCol *table.Column, n oldColIsStored := !oldCol.IsGenerated() || oldCol.GeneratedStored newColIsStored := !newCol.IsGenerated() || newCol.GeneratedStored if oldColIsStored != newColIsStored { - return errUnsupportedOnGeneratedColumn.GenWithStackByArgs("Changing the STORED status") + return ErrUnsupportedOnGeneratedColumn.GenWithStackByArgs("Changing the STORED status") } // rule 2. @@ -285,13 +285,13 @@ func checkIndexOrStored(tbl table.Table, oldCol, newCol *table.Column) error { } if newCol.GeneratedStored { - return errUnsupportedOnGeneratedColumn.GenWithStackByArgs("modifying a stored column") + return ErrUnsupportedOnGeneratedColumn.GenWithStackByArgs("modifying a stored column") } for _, idx := range tbl.Indices() { for _, col := range idx.Meta().Columns { if col.Name.L == newCol.Name.L { - return errUnsupportedOnGeneratedColumn.GenWithStackByArgs("modifying an indexed column") + return ErrUnsupportedOnGeneratedColumn.GenWithStackByArgs("modifying an indexed column") } } } diff --git a/ddl/index.go b/ddl/index.go index 1fa3c1c5f1923..f57a25781b942 100755 --- a/ddl/index.go +++ b/ddl/index.go @@ -101,7 +101,7 @@ func checkPKOnGeneratedColumn(tblInfo *model.TableInfo, indexPartSpecifications } // Virtual columns cannot be used in primary key. if lastCol.IsGenerated() && !lastCol.GeneratedStored { - return nil, errUnsupportedOnGeneratedColumn.GenWithStackByArgs("Defining a virtual generated column as primary key") + return nil, ErrUnsupportedOnGeneratedColumn.GenWithStackByArgs("Defining a virtual generated column as primary key") } } diff --git a/go.mod b/go.mod index a28a541f8b96f..0c82848fccec7 100644 --- a/go.mod +++ b/go.mod @@ -8,7 +8,7 @@ require ( github.com/danjacques/gofslock v0.0.0-20191023191349-0a45f885bc37 github.com/dgraph-io/ristretto v0.0.1 github.com/dgryski/go-farm v0.0.0-20190423205320-6a90982ecee2 - github.com/go-sql-driver/mysql v1.4.1 + github.com/go-sql-driver/mysql v1.5.0 github.com/gogo/protobuf v1.3.1 github.com/golang/protobuf v1.3.4 github.com/golang/snappy v0.0.1 @@ -29,8 +29,8 @@ require ( github.com/pingcap/fn v0.0.0-20191016082858-07623b84a47d github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989 github.com/pingcap/kvproto v0.0.0-20200424032552-6650270c39c3 - github.com/pingcap/log v0.0.0-20200117041106-d28c14d3b1cd - github.com/pingcap/parser v3.1.1+incompatible + github.com/pingcap/log v0.0.0-20200511115504-543df19646ad + github.com/pingcap/parser v0.0.0-20200513120358-b5cbbc04cce2 github.com/pingcap/pd/v4 v4.0.0-rc.1.0.20200422143320-428acd53eba2 github.com/pingcap/sysutil v0.0.0-20200408114249-ed3bd6f7fdb1 github.com/pingcap/tidb-tools v4.0.0-rc.1.0.20200421113014-507d2bb3a15e+incompatible @@ -62,5 +62,3 @@ require ( ) go 1.13 - -replace github.com/pingcap/parser => github.com/zhaox1n/parser v0.0.0-20200510040745-d20d92603e29 diff --git a/go.sum b/go.sum index 661667a2cc097..dfbcf51e22db9 100644 --- a/go.sum +++ b/go.sum @@ -153,6 +153,8 @@ github.com/go-playground/universal-translator v0.16.0/go.mod h1:1AnU7NaIRDWWzGEK github.com/go-playground/universal-translator v0.17.0/go.mod h1:UkSxE5sNxxRwHyU+Scu5vgOQjsIJAF8j9muTVoKLVtA= github.com/go-sql-driver/mysql v1.4.1 h1:g24URVg0OFbNUTx9qqY1IRZ9D9z3iPyi5zKhQZpNwpA= github.com/go-sql-driver/mysql v1.4.1/go.mod h1:zAC/RDZ24gD3HViQzih4MyKcchzm+sOG5ZlKdlhCg5w= +github.com/go-sql-driver/mysql v1.5.0 h1:ozyZYNQW3x3HtqT1jira07DN2PArx2v7/mN66gGcHOs= +github.com/go-sql-driver/mysql v1.5.0/go.mod h1:DCzpHaOWr8IXmIStZouvnhqoel9Qv2LBy8hT2VhHyBg= github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= github.com/gogo/protobuf v0.0.0-20180717141946-636bf0302bc9/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= github.com/gogo/protobuf v1.1.1/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= @@ -369,8 +371,11 @@ github.com/pingcap/log v0.0.0-20200117041106-d28c14d3b1cd h1:CV3VsP3Z02MVtdpTMfE github.com/pingcap/log v0.0.0-20200117041106-d28c14d3b1cd h1:CV3VsP3Z02MVtdpTMfEgRJ4T9NGgGTxdHpJerent7rM= github.com/pingcap/log v0.0.0-20200117041106-d28c14d3b1cd/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= github.com/pingcap/log v0.0.0-20200117041106-d28c14d3b1cd/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= +github.com/pingcap/log v0.0.0-20200511115504-543df19646ad h1:SveG82rmu/GFxYanffxsSF503SiQV+2JLnWEiGiF+Tc= +github.com/pingcap/log v0.0.0-20200511115504-543df19646ad/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= github.com/pingcap/parser v0.0.0-20200424075042-8222d8b724a4/go.mod h1:9v0Edh8IbgjGYW2ArJr19E+bvL8zKahsFp+ixWeId+4= -github.com/pingcap/parser v3.1.1+incompatible/go.mod h1:1FNvfp9+J0wvc4kl8eGNh7Rqrxveg15jJoWo/a0uHwA= +github.com/pingcap/parser v0.0.0-20200513120358-b5cbbc04cce2 h1:bgoZ3ctjzugt5brdM8HBeZDK1rHiCE9Vom+kY1qmdIw= +github.com/pingcap/parser v0.0.0-20200513120358-b5cbbc04cce2/go.mod h1:vQdbJqobJAgFyiRNNtXahpMoGWwPEuWciVEK5A20NS0= github.com/pingcap/pd/v4 v4.0.0-rc.1.0.20200422143320-428acd53eba2 h1:JTzYYukREvxVSKW/ncrzNjFitd8snoQ/Xz32pw8i+s8= github.com/pingcap/pd/v4 v4.0.0-rc.1.0.20200422143320-428acd53eba2/go.mod h1:s+utZtXDznOiL24VK0qGmtoHjjXNsscJx3m1n8cC56s= github.com/pingcap/sysutil v0.0.0-20200206130906-2bfa6dc40bcd/go.mod h1:EB/852NMQ+aRKioCpToQ94Wl7fktV+FNnxf3CX/TTXI= @@ -501,12 +506,6 @@ github.com/xiang90/probing v0.0.0-20190116061207-43a291ad63a2/go.mod h1:UETIi67q github.com/xordataexchange/crypt v0.0.3-0.20170626215501-b2862e3d0a77/go.mod h1:aYKd//L2LvnjZzWKhF00oedf4jCCReLcmhLdhm1A27Q= github.com/yookoala/realpath v1.0.0/go.mod h1:gJJMA9wuX7AcqLy1+ffPatSCySA1FQ2S8Ya9AIoYBpE= github.com/yuin/goldmark v1.1.25/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= -github.com/zhaox1n/parser v0.0.0-20200430160105-6fcbbbfa6158 h1:3EhundhszzvUGiWSvUzwW1wiM5YUVnih3EgIhJZ8a8s= -github.com/zhaox1n/parser v0.0.0-20200430160105-6fcbbbfa6158/go.mod h1:9v0Edh8IbgjGYW2ArJr19E+bvL8zKahsFp+ixWeId+4= -github.com/zhaox1n/parser v0.0.0-20200510040745-d20d92603e29 h1:uI3SH5eGCzeTSi6Knl2YxsZgdyK9yI+Od//OYbc4Bdc= -github.com/zhaox1n/parser v0.0.0-20200510040745-d20d92603e29/go.mod h1:9v0Edh8IbgjGYW2ArJr19E+bvL8zKahsFp+ixWeId+4= -github.com/zhaox1n/parser v3.0.12+incompatible h1:4HqB/EJuvIasw1Io39UX1UM2FutJ2UTsNm1wptuxeGw= -github.com/zhaox1n/parser v3.0.12+incompatible/go.mod h1:/xFSvLQ9LsJ9t+fZfxb6/FqVrulSjEb5glcqR4fmZao= go.etcd.io/bbolt v1.3.3 h1:MUGmc65QhB3pIlaQ5bB4LwqSj6GIonVJXpZiaKNyaKk= go.etcd.io/bbolt v1.3.3/go.mod h1:IbVyRI1SCnLcuJnV2u8VeU0CEYM7e686BmAb1XKL+uU= go.etcd.io/etcd v0.0.0-20191023171146-3cf2f69b5738/go.mod h1:dnLIgRNXwCJa5e+c6mIZCrds/GIG4ncV9HhK5PX7jPg= From f4f5b38359c00d31255913eafb7058def2df551a Mon Sep 17 00:00:00 2001 From: zhaox1n Date: Thu, 14 May 2020 04:51:49 +0800 Subject: [PATCH 17/57] fix the conflict --- ddl/db_partition_test.go | 23 ++++++++++++++++++----- ddl/ddl_api.go | 14 ++++++++++---- go.sum | 1 + 3 files changed, 29 insertions(+), 9 deletions(-) diff --git a/ddl/db_partition_test.go b/ddl/db_partition_test.go index 752c6f45589e8..5a4a053620d3a 100644 --- a/ddl/db_partition_test.go +++ b/ddl/db_partition_test.go @@ -879,7 +879,7 @@ func (s *testIntegrationSuite4) TestExchangePartitionTableCompatiable(c *C) { // Partition table is yet not supports to add foreign keys in mysql "create table pt9 (id int not null primary key auto_increment,t_id int not null) partition by hash(id) partitions 1;", "create table nt9 (id int not null primary key auto_increment, t_id int not null,foreign key fk_id (t_id) references pt5(id));", - "alter table pt9 exchange partition p0 with table pt9;", + "alter table pt9 exchange partition p0 with table nt9;", ddl.ErrPartitionExchangeForeignKey, }, { @@ -938,15 +938,30 @@ func (s *testIntegrationSuite4) TestExchangePartitionTableCompatiable(c *C) { "alter table pt18 exchange partition p0 with table nt18", ddl.ErrCheckNoSuchTable, }, + { + "create table pt19 (id int not null, lname varchar(30), fname varchar(100) generated always as (concat(lname, ' ')) stored) partition by hash(id) partitions 1;", + "create table nt19 (id int not null, lname varchar(30), fname varchar(100) generated always as (concat(lname, ' ')) virtual);", + "alter table pt19 exchange partition p0 with table nt19;", + ddl.ErrUnsupportedOnGeneratedColumn, + }, } tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") - for _, t := range cases { + for i, t := range cases { tk.MustExec(t.ptSQL) tk.MustExec(t.ntSQL) if t.err != nil { - tk.MustGetErrCode(t.exchangeSQL, t.err.Code()) + _, err := tk.Exec(t.exchangeSQL) + originErr := errors.Cause(err) + tErr, ok := originErr.(*terror.Error) + c.Assert(ok, IsTrue, Commentf("case %d fail, sql = `%s`\nexpect type 'terror.Error', but obtain '%T'", i, + t.exchangeSQL, originErr)) + sqlErr := tErr.ToSQLError() + c.Assert(int(t.err.Code()), Equals, int(sqlErr.Code), Commentf( + "case %d fail, sql = `%s`\nexpected error = `%v`\n actual error = `%v`", + i, t.exchangeSQL, t.err, err, + )) } else { tk.MustExec(t.exchangeSQL) } @@ -1872,8 +1887,6 @@ func (s *testIntegrationSuite3) TestPartitionErrorCode(c *C) { tk.MustGetErrCode("alter table t_part optimize partition p0,p1;", errno.ErrUnsupportedDDLOperation) tk.MustGetErrCode("alter table t_part rebuild partition p0,p1;", errno.ErrUnsupportedDDLOperation) tk.MustGetErrCode("alter table t_part remove partitioning;", errno.ErrUnsupportedDDLOperation) - tk.MustExec("create table t_part2 like t_part") - tk.MustGetErrCode("alter table t_part exchange partition p0 with table t_part2", errno.ErrUnsupportedDDLOperation) } func (s *testIntegrationSuite5) TestConstAndTimezoneDepent(c *C) { diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index e2d753a2d54aa..ba44276c5fef7 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -2176,8 +2176,6 @@ func (d *ddl) AlterTable(ctx sessionctx.Context, ident ast.Ident, specs []*ast.A err = errors.Trace(errUnsupportedOptimizePartition) case ast.AlterTableRemovePartitioning: err = errors.Trace(errUnsupportedRemovePartition) - case ast.AlterTableExchangePartition: - err = errors.Trace(errUnsupportedExchangePartition) case ast.AlterTableDropColumn: err = d.DropColumn(ctx, ident, spec) case ast.AlterTableDropIndex: @@ -2769,7 +2767,8 @@ func checkFielTypeCompatible(ft *types.FieldType, other *types.FieldType) bool { ft.Charset == other.Charset && ft.Collate == other.Collate && (ft.Flen == other.Flen || ft.StorageLength() != types.VarStorageLen) && - mysql.HasUnsignedFlag(ft.Flag) == mysql.HasUnsignedFlag(other.Flag) + mysql.HasUnsignedFlag(ft.Flag) == mysql.HasUnsignedFlag(other.Flag) && + mysql.HasAutoIncrementFlag(ft.Flag) == mysql.HasAutoIncrementFlag(other.Flag) if !partialEqual || len(ft.Elems) != len(other.Elems) { return false } @@ -2781,6 +2780,13 @@ func checkFielTypeCompatible(ft *types.FieldType, other *types.FieldType) bool { return true } +func checkColumnVirtual(col *model.ColumnInfo) bool { + if col.IsGenerated() && !col.GeneratedStored { + return true + } + return false +} + func checkTableDefCompatible(source *model.TableInfo, target *model.TableInfo) error { err := ErrTablesDifferentMetadata if len(source.Cols()) != len(target.Cols()) { @@ -2789,7 +2795,7 @@ func checkTableDefCompatible(source *model.TableInfo, target *model.TableInfo) e // Col compatible check for i, sourceCol := range source.Cols() { targetCol := target.Cols()[i] - if sourceCol.GeneratedStored != targetCol.GeneratedStored { + if checkColumnVirtual(sourceCol) != checkColumnVirtual(targetCol) { return ErrUnsupportedOnGeneratedColumn.GenWithStackByArgs("Exchanging partitions for non-generated columns") } if sourceCol.Name.L != targetCol.Name.L || diff --git a/go.sum b/go.sum index 67f9b8bc85c9c..d421dc589eb5e 100644 --- a/go.sum +++ b/go.sum @@ -413,6 +413,7 @@ github.com/rs/cors v1.7.0/go.mod h1:gFx+x8UowdsKA9AchylcLynDq+nNFfI8FkUZdN/jGCU= github.com/russross/blackfriday v1.5.2/go.mod h1:JO/DiYxRf+HjHt06OyowR9PTA263kcR/rfWxYHBV53g= github.com/russross/blackfriday/v2 v2.0.1/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM= github.com/satori/go.uuid v1.2.0/go.mod h1:dA0hQrYB0VpLJoorglMZABFdXlWrHn1NEOzdhQKdks0= +github.com/sergi/go-diff v1.0.1-0.20180205163309-da645544ed44 h1:tB9NOR21++IjLyVx3/PCPhWMwqGNCMQEH96A6dMZ/gc= github.com/sergi/go-diff v1.0.1-0.20180205163309-da645544ed44/go.mod h1:0CfEIISq7TuYL3j771MWULgwwjU+GofnZX9QAmXWZgo= github.com/shirou/gopsutil v2.19.10+incompatible h1:lA4Pi29JEVIQIgATSeftHSY0rMGI9CLrl2ZvDLiahto= github.com/shirou/gopsutil v2.19.10+incompatible/go.mod h1:5b4v6he4MtMOwMlS0TUMTu2PcXUg8+E1lC7eC3UO/RA= From 3cec6f25e7c9f7287cba28891f650278ba250505 Mon Sep 17 00:00:00 2001 From: zhaox1n Date: Fri, 15 May 2020 00:37:05 +0800 Subject: [PATCH 18/57] add more test of table compatible --- ddl/column.go | 7 ++++++ ddl/db_partition_test.go | 46 +++++++++++++++++++++++++++++++++++---- ddl/ddl_api.go | 14 +++++------- table/tables/partition.go | 2 +- 4 files changed, 55 insertions(+), 14 deletions(-) diff --git a/ddl/column.go b/ddl/column.go index 71f9994cd9eb0..0ef99330d9c74 100644 --- a/ddl/column.go +++ b/ddl/column.go @@ -891,3 +891,10 @@ func getColumnInfoByName(tbInfo *model.TableInfo, column string) *model.ColumnIn } return nil } + +func IsVirtualGeneratedColumn(col *model.ColumnInfo) bool { + if col.IsGenerated() && !col.GeneratedStored { + return true + } + return false +} diff --git a/ddl/db_partition_test.go b/ddl/db_partition_test.go index 230d6297c7ada..25de85fd7af0d 100644 --- a/ddl/db_partition_test.go +++ b/ddl/db_partition_test.go @@ -792,6 +792,7 @@ func (s *testIntegrationSuite5) TestAlterTableExchangePartition(c *C) { tk.MustExec("ALTER TABLE e EXCHANGE PARTITION p0 WITH TABLE e2") tk.MustQuery("select * from e2").Check(testkit.Rows("16")) tk.MustQuery("select * from e").Check(testkit.Rows("1669", "337", "2005")) + // validation check for range partition tk.MustGetErrCode("ALTER TABLE e EXCHANGE PARTITION p1 WITH TABLE e2", tmysql.ErrRowDoesNotMatchPartition) tk.MustGetErrCode("ALTER TABLE e EXCHANGE PARTITION p2 WITH TABLE e2", tmysql.ErrRowDoesNotMatchPartition) tk.MustGetErrCode("ALTER TABLE e EXCHANGE PARTITION p3 WITH TABLE e2", tmysql.ErrRowDoesNotMatchPartition) @@ -810,6 +811,7 @@ func (s *testIntegrationSuite5) TestAlterTableExchangePartition(c *C) { tk.MustQuery("select * from e3 partition(p0)").Check(testkit.Rows()) tk.MustQuery("select * from e2").Check(testkit.Rows("1", "5")) + // check validation fot hash partition tk.MustGetErrCode("ALTER TABLE e3 EXCHANGE PARTITION p0 WITH TABLE e2", tmysql.ErrRowDoesNotMatchPartition) tk.MustGetErrCode("ALTER TABLE e3 EXCHANGE PARTITION p2 WITH TABLE e2", tmysql.ErrRowDoesNotMatchPartition) tk.MustGetErrCode("ALTER TABLE e3 EXCHANGE PARTITION p3 WITH TABLE e2", tmysql.ErrRowDoesNotMatchPartition) @@ -924,24 +926,28 @@ func (s *testIntegrationSuite4) TestExchangePartitionTableCompatiable(c *C) { nil, }, { + // unique index "create table pt15 (id int not null, unique index uk_id (id)) partition by hash(id) partitions 1;", "create table nt15 (id int not null, index uk_id (id));", "alter table pt15 exchange partition p0 with table nt15", ddl.ErrTablesDifferentMetadata, }, { + // auto_increment "create table pt16 (id int not null primary key auto_increment) partition by hash(id) partitions 1;", "create table nt16 (id int not null primary key);", "alter table pt16 exchange partition p0 with table nt16;", ddl.ErrTablesDifferentMetadata, }, { + // default "create table pt17 (id int not null default 1) partition by hash(id) partitions 1;", "create table nt17 (id int not null);", "alter table pt17 exchange partition p0 with table nt17;", nil, }, { + // view test "create table pt18 (id int not null) partition by hash(id) partitions 1;", "create view nt18 as select id from nt17;", "alter table pt18 exchange partition p0 with table nt18", @@ -953,6 +959,38 @@ func (s *testIntegrationSuite4) TestExchangePartitionTableCompatiable(c *C) { "alter table pt19 exchange partition p0 with table nt19;", ddl.ErrUnsupportedOnGeneratedColumn, }, + { + "create table pt20 (id int not null) partition by hash(id) partitions 1;", + "create table nt20 (id int default null);", + "alter table pt20 exchange partition p0 with table nt20;", + ddl.ErrTablesDifferentMetadata, + }, + { + // unsigned + "create table pt21 (id int unsigned) partition by hash(id) partitions 1;", + "create table nt21 (id int);", + "alter table pt21 exchange partition p0 with table nt21;", + ddl.ErrTablesDifferentMetadata, + }, + { + // zerofill + "create table pt22 (id int) partition by hash(id) partitions 1;", + "create table nt22 (id int zerofill);", + "alter table pt22 exchange partition p0 with table nt22;", + ddl.ErrTablesDifferentMetadata, + }, + { + "create table pt23 (id int, lname varchar(10) charset binary) partition by hash(id) partitions 1;", + "create table nt23 (id int, lname varchar(10));", + "alter table pt23 exchange partition p0 with table nt23;", + ddl.ErrTablesDifferentMetadata, + }, + { + "create table pt25 (id int, a datetime on update current_timestamp) partition by hash(id) partitions 1;", + "create table nt25 (id int, a datetime);", + "alter table pt25 exchange partition p0 with table nt25;", + nil, + }, } tk := testkit.NewTestKit(c, s.store) @@ -1908,10 +1946,10 @@ func (s *testIntegrationSuite3) TestPartitionErrorCode(c *C) { tk.MustGetErrCode(`alter table t_part reorganize partition p0, p1 into ( partition p0 values less than (1980));`, tmysql.ErrUnsupportedDDLOperation) - tk.MustGetErrCode("alter table t_part check partition p0, p1;", errno.ErrUnsupportedDDLOperation) - tk.MustGetErrCode("alter table t_part optimize partition p0,p1;", errno.ErrUnsupportedDDLOperation) - tk.MustGetErrCode("alter table t_part rebuild partition p0,p1;", errno.ErrUnsupportedDDLOperation) - tk.MustGetErrCode("alter table t_part remove partitioning;", errno.ErrUnsupportedDDLOperation) + tk.MustGetErrCode("alter table t_part check partition p0, p1;", tmysql.ErrUnsupportedDDLOperation) + tk.MustGetErrCode("alter table t_part optimize partition p0,p1;", tmysql.ErrUnsupportedDDLOperation) + tk.MustGetErrCode("alter table t_part rebuild partition p0,p1;", tmysql.ErrUnsupportedDDLOperation) + tk.MustGetErrCode("alter table t_part remove partitioning;", tmysql.ErrUnsupportedDDLOperation) } func (s *testIntegrationSuite5) TestConstAndTimezoneDepent(c *C) { diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index ef715008faead..b65f684c75d24 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -2781,7 +2781,10 @@ func checkFielTypeCompatible(ft *types.FieldType, other *types.FieldType) bool { ft.Collate == other.Collate && (ft.Flen == other.Flen || ft.StorageLength() != types.VarStorageLen) && mysql.HasUnsignedFlag(ft.Flag) == mysql.HasUnsignedFlag(other.Flag) && - mysql.HasAutoIncrementFlag(ft.Flag) == mysql.HasAutoIncrementFlag(other.Flag) + mysql.HasAutoIncrementFlag(ft.Flag) == mysql.HasAutoIncrementFlag(other.Flag) && + mysql.HasNotNullFlag(ft.Flag) == mysql.HasNotNullFlag(other.Flag) && + mysql.HasZerofillFlag(ft.Flag) == mysql.HasZerofillFlag(other.Flag) && + mysql.HasBinaryFlag(ft.Flag) == mysql.HasBinaryFlag(other.Flag) if !partialEqual || len(ft.Elems) != len(other.Elems) { return false } @@ -2793,13 +2796,6 @@ func checkFielTypeCompatible(ft *types.FieldType, other *types.FieldType) bool { return true } -func checkColumnVirtual(col *model.ColumnInfo) bool { - if col.IsGenerated() && !col.GeneratedStored { - return true - } - return false -} - func checkTableDefCompatible(source *model.TableInfo, target *model.TableInfo) error { err := ErrTablesDifferentMetadata if len(source.Cols()) != len(target.Cols()) { @@ -2808,7 +2804,7 @@ func checkTableDefCompatible(source *model.TableInfo, target *model.TableInfo) e // Col compatible check for i, sourceCol := range source.Cols() { targetCol := target.Cols()[i] - if checkColumnVirtual(sourceCol) != checkColumnVirtual(targetCol) { + if IsVirtualGeneratedColumn(sourceCol) != IsVirtualGeneratedColumn(targetCol) { return ErrUnsupportedOnGeneratedColumn.GenWithStackByArgs("Exchanging partitions for non-generated columns") } if sourceCol.Name.L != targetCol.Name.L || diff --git a/table/tables/partition.go b/table/tables/partition.go index 45995e9204230..83ef70098c398 100644 --- a/table/tables/partition.go +++ b/table/tables/partition.go @@ -126,7 +126,7 @@ type ForRangePruning struct { Unsigned bool } -// dataForRangePruning extracts the less than parts from 'partition p0 less than xx ... partitoin p1 less than ...' +// DataForRangePruning extracts the less than parts from 'partition p0 less than xx ... partitoin p1 less than ...' func DataForRangePruning(pi *model.PartitionInfo) (*ForRangePruning, error) { var maxValue bool var unsigned bool From 53f4f73ba6122f3fc5086f97901bd0d621bdbaa9 Mon Sep 17 00:00:00 2001 From: zhaox1n Date: Fri, 15 May 2020 00:43:46 +0800 Subject: [PATCH 19/57] fix comment --- ddl/column.go | 1 + ddl/error.go | 6 +++--- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/ddl/column.go b/ddl/column.go index 0ef99330d9c74..f5b5f2021edb3 100644 --- a/ddl/column.go +++ b/ddl/column.go @@ -892,6 +892,7 @@ func getColumnInfoByName(tbInfo *model.TableInfo, column string) *model.ColumnIn return nil } +// check column if it is virtual func IsVirtualGeneratedColumn(col *model.ColumnInfo) bool { if col.IsGenerated() && !col.GeneratedStored { return true diff --git a/ddl/error.go b/ddl/error.go index a724191777146..d97bad6c96658 100644 --- a/ddl/error.go +++ b/ddl/error.go @@ -64,7 +64,7 @@ var ( errWrongKeyColumn = terror.ClassDDL.New(mysql.ErrWrongKeyColumn, mysql.MySQLErrName[mysql.ErrWrongKeyColumn]) // errWrongFKOptionForGeneratedColumn is for wrong foreign key reference option on generated columns. errWrongFKOptionForGeneratedColumn = terror.ClassDDL.New(mysql.ErrWrongFKOptionForGeneratedColumn, mysql.MySQLErrName[mysql.ErrWrongFKOptionForGeneratedColumn]) - // errUnsupportedOnGeneratedColumn is for unsupported actions on generated columns. + // ErrUnsupportedOnGeneratedColumn is for unsupported actions on generated columns. ErrUnsupportedOnGeneratedColumn = terror.ClassDDL.New(mysql.ErrUnsupportedOnGeneratedColumn, mysql.MySQLErrName[mysql.ErrUnsupportedOnGeneratedColumn]) // errGeneratedColumnNonPrior forbids to refer generated column non prior to it. errGeneratedColumnNonPrior = terror.ClassDDL.New(mysql.ErrGeneratedColumnNonPrior, mysql.MySQLErrName[mysql.ErrGeneratedColumnNonPrior]) @@ -212,12 +212,12 @@ var ( ErrUnsupportedExpressionIndex = terror.ClassDDL.New(mysql.ErrUnsupportedDDLOperation, fmt.Sprintf(mysql.MySQLErrName[mysql.ErrUnsupportedDDLOperation], "creating expression index without allow-expression-index in config")) // ErrPartitionExchangePartTable is returned when exchange table partition with another table is partitioned ErrPartitionExchangePartTable = terror.ClassDDL.New(mysql.ErrPartitionExchangePartTable, mysql.MySQLErrName[mysql.ErrPartitionExchangePartTable]) - // ErrTableDifferentMetadata is returned when exchanges tables is not compatible + // ErrTablesDifferentMetadata is returned when exchanges tables is not compatible ErrTablesDifferentMetadata = terror.ClassDDL.New(mysql.ErrTablesDifferentMetadata, mysql.MySQLErrName[mysql.ErrTablesDifferentMetadata]) // ErrRowDoesNotMatchPartition is returned when the row record of exchange table does not match the partition rule ErrRowDoesNotMatchPartition = terror.ClassDDL.New(mysql.ErrRowDoesNotMatchPartition, mysql.MySQLErrName[mysql.ErrRowDoesNotMatchPartition]) // ErrPartitionExchangeForeignKey is returned when exchanged non-partition table has foreign keys ErrPartitionExchangeForeignKey = terror.ClassDDL.New(mysql.ErrPartitionExchangeForeignKey, mysql.MySQLErrName[mysql.ErrPartitionExchangeForeignKey]) - // ErrCheckNoSunchTable is returned when exchaned non-partition table is view or sequence + // ErrCheckNoSuchTable is returned when exchaned non-partition table is view or sequence ErrCheckNoSuchTable = terror.ClassDDL.New(mysql.ErrCheckNoSuchTable, mysql.MySQLErrName[mysql.ErrCheckNoSuchTable]) ) From 101c673e26e44e1ad8486752f9bc53ada53280c4 Mon Sep 17 00:00:00 2001 From: zhaox1n Date: Fri, 15 May 2020 00:47:58 +0800 Subject: [PATCH 20/57] rename function --- ddl/column.go | 2 +- ddl/ddl_api.go | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/ddl/column.go b/ddl/column.go index f5b5f2021edb3..2448535592d24 100644 --- a/ddl/column.go +++ b/ddl/column.go @@ -893,7 +893,7 @@ func getColumnInfoByName(tbInfo *model.TableInfo, column string) *model.ColumnIn } // check column if it is virtual -func IsVirtualGeneratedColumn(col *model.ColumnInfo) bool { +func isVirtualGeneratedColumn(col *model.ColumnInfo) bool { if col.IsGenerated() && !col.GeneratedStored { return true } diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index b65f684c75d24..e7df96100e727 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -2804,7 +2804,7 @@ func checkTableDefCompatible(source *model.TableInfo, target *model.TableInfo) e // Col compatible check for i, sourceCol := range source.Cols() { targetCol := target.Cols()[i] - if IsVirtualGeneratedColumn(sourceCol) != IsVirtualGeneratedColumn(targetCol) { + if isVirtualGeneratedColumn(sourceCol) != isVirtualGeneratedColumn(targetCol) { return ErrUnsupportedOnGeneratedColumn.GenWithStackByArgs("Exchanging partitions for non-generated columns") } if sourceCol.Name.L != targetCol.Name.L || From a1ca360cbbfd16d9e7173a7ad30960766a9a18f9 Mon Sep 17 00:00:00 2001 From: zhaox1n Date: Fri, 15 May 2020 00:53:34 +0800 Subject: [PATCH 21/57] fix wrong english spell --- ddl/ddl_api.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index e7df96100e727..82f0adc4d7138 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -2826,7 +2826,7 @@ func checkTableDefCompatible(source *model.TableInfo, target *model.TableInfo) e if compatIdx == nil { return err } - // Index type is not compatiable + // Index type is not compatible if sourceIdx.Tp != compatIdx.Tp || sourceIdx.Unique != compatIdx.Unique || sourceIdx.Primary != compatIdx.Primary { From c559f1284b2ee4e03c3554aa7e737c2123789009 Mon Sep 17 00:00:00 2001 From: zhaox1n Date: Sat, 16 May 2020 19:17:32 +0800 Subject: [PATCH 22/57] add test for expression index --- ddl/db_partition_test.go | 43 ++++++++++++++++++++++++++++++++++++---- ddl/ddl_api.go | 1 + ddl/partition.go | 3 +++ 3 files changed, 43 insertions(+), 4 deletions(-) diff --git a/ddl/db_partition_test.go b/ddl/db_partition_test.go index 25de85fd7af0d..559447ce19554 100644 --- a/ddl/db_partition_test.go +++ b/ddl/db_partition_test.go @@ -771,7 +771,7 @@ func (s *testIntegrationSuite5) TestAlterTableDropPartition(c *C) { tk.MustGetErrCode("alter table t1 drop partition p2", tmysql.ErrOnlyOnRangeListPartition) } -func (s *testIntegrationSuite5) TestAlterTableExchangePartition(c *C) { +func (s *testIntegrationSuite4) TestAlterTableExchangePartition(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists e") @@ -792,7 +792,7 @@ func (s *testIntegrationSuite5) TestAlterTableExchangePartition(c *C) { tk.MustExec("ALTER TABLE e EXCHANGE PARTITION p0 WITH TABLE e2") tk.MustQuery("select * from e2").Check(testkit.Rows("16")) tk.MustQuery("select * from e").Check(testkit.Rows("1669", "337", "2005")) - // validation check for range partition + // validation test for range partition tk.MustGetErrCode("ALTER TABLE e EXCHANGE PARTITION p1 WITH TABLE e2", tmysql.ErrRowDoesNotMatchPartition) tk.MustGetErrCode("ALTER TABLE e EXCHANGE PARTITION p2 WITH TABLE e2", tmysql.ErrRowDoesNotMatchPartition) tk.MustGetErrCode("ALTER TABLE e EXCHANGE PARTITION p3 WITH TABLE e2", tmysql.ErrRowDoesNotMatchPartition) @@ -811,19 +811,22 @@ func (s *testIntegrationSuite5) TestAlterTableExchangePartition(c *C) { tk.MustQuery("select * from e3 partition(p0)").Check(testkit.Rows()) tk.MustQuery("select * from e2").Check(testkit.Rows("1", "5")) - // check validation fot hash partition + // validation test fot hash partition tk.MustGetErrCode("ALTER TABLE e3 EXCHANGE PARTITION p0 WITH TABLE e2", tmysql.ErrRowDoesNotMatchPartition) tk.MustGetErrCode("ALTER TABLE e3 EXCHANGE PARTITION p2 WITH TABLE e2", tmysql.ErrRowDoesNotMatchPartition) tk.MustGetErrCode("ALTER TABLE e3 EXCHANGE PARTITION p3 WITH TABLE e2", tmysql.ErrRowDoesNotMatchPartition) + + // without validation test tk.MustExec("ALTER TABLE e3 EXCHANGE PARTITION p0 with TABLE e2 WITHOUT VALIDATION") tk.MustQuery("select * from e3 partition(p0)").Check(testkit.Rows("1", "5")) tk.MustQuery("select * from e2").Check(testkit.Rows()) + // expression index test for hash partition + } func (s *testIntegrationSuite4) TestExchangePartitionTableCompatiable(c *C) { - type testCase struct { ptSQL string ntSQL string @@ -991,6 +994,12 @@ func (s *testIntegrationSuite4) TestExchangePartitionTableCompatiable(c *C) { "alter table pt25 exchange partition p0 with table nt25;", nil, }, + { + "create table pt26 (id int not null, lname varchar(30), fname varchar(100) generated always as (concat(lname, ' ')) virtual) partition by hash(id) partitions 1;", + "create table nt26 (id int not null, lname varchar(30), fname varchar(100) generated always as (concat(id, ' ')) virtual);", + "alter table pt26 exchange partition p0 with table nt26;", + nil, + }, } tk := testkit.NewTestKit(c, s.store) @@ -1016,6 +1025,32 @@ func (s *testIntegrationSuite4) TestExchangePartitionTableCompatiable(c *C) { } +func (s *testIntegrationSuite7) TestExchangePartitionExpressIndex(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists pt1;") + tk.MustExec("create table pt1(a int, b int, c int) PARTITION BY hash (a) partitions 1;") + tk.MustExec("alter table pt1 add index idx((a+c));") + + tk.MustExec("drop table if exists nt1;") + tk.MustExec("create table nt1(a int, b int, c int);") + tk.MustGetErrCode("alter table pt1 exchange partition p0 with table nt1;", tmysql.ErrTablesDifferentMetadata) + + tk.MustExec("alter table nt1 add column (`_V$_idx_0` bigint(20) generated always as (a+b) virtual);") + tk.MustGetErrCode("alter table pt1 exchange partition p0 with table nt1;", tmysql.ErrTablesDifferentMetadata) + + // test different expression index when expression return same field type + tk.MustExec("alter table nt1 drop column `_V$_idx_0`;") + tk.MustExec("alter table nt1 add index idx((b-c));") + tk.MustExec("alter table pt1 exchange partition p0 with table nt1;") + + // test different expression index when expression return different field type + tk.MustExec("alter table nt1 drop index idx;") + tk.MustExec("alter table nt1 add index idx((concat(a, b)));") + tk.MustGetErrCode("alter table pt1 exchange partition p0 with table nt1;", tmysql.ErrTablesDifferentMetadata) + +} + func (s *testIntegrationSuite4) TestAddPartitionTooManyPartitions(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 2b251ce7f32a2..c6437e9edebca 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -2829,6 +2829,7 @@ func checkTableDefCompatible(source *model.TableInfo, target *model.TableInfo) e return ErrUnsupportedOnGeneratedColumn.GenWithStackByArgs("Exchanging partitions for non-generated columns") } if sourceCol.Name.L != targetCol.Name.L || + sourceCol.Hidden != targetCol.Hidden || !checkFielTypeCompatible(&sourceCol.FieldType, &targetCol.FieldType) { return err } diff --git a/ddl/partition.go b/ddl/partition.go index 5812a88352acd..7ef9ae527c630 100644 --- a/ddl/partition.go +++ b/ddl/partition.go @@ -823,6 +823,9 @@ func checkExchangePartitionRecordValidation(w *worker, pi *model.PartitionInfo, switch pi.Type { case model.PartitionTypeHash: + if pi.Num == 1 { + return nil + } sql = fmt.Sprintf("select 1 from `%s`.`%s` where mod(%s, %d) != %d limit 1", schemaName.L, tableName.L, pi.Expr, pi.Num, index) break case model.PartitionTypeRange: From f9fdd0c8bbe73e6c7515eab3599825bac06cd09b Mon Sep 17 00:00:00 2001 From: zhaox1n Date: Sun, 17 May 2020 12:56:03 +0800 Subject: [PATCH 23/57] add check in ddl worker --- ddl/partition.go | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/ddl/partition.go b/ddl/partition.go index 7ef9ae527c630..bbe3d8604ad18 100644 --- a/ddl/partition.go +++ b/ddl/partition.go @@ -729,6 +729,12 @@ func (w *worker) onExchangeTablePartition(d *ddlCtx, t *meta.Meta, job *model.Jo return ver, errors.Trace(err) } + err = checkExchangePartition(pt, nt) + if err != nil { + job.State = model.JobStateCancelled + return ver, errors.Trace(err) + } + err = checkTableDefCompatible(pt, nt) if err != nil { job.State = model.JobStateCancelled @@ -743,12 +749,14 @@ func (w *worker) onExchangeTablePartition(d *ddlCtx, t *meta.Meta, job *model.Jo } } + // partition table base auto id ptBaseID, err := t.GetAutoTableID(ptSchemaID, pt.ID) if err != nil { job.State = model.JobStateCancelled return ver, errors.Trace(err) } + // non-partition table base auto id ntBaseID, err := t.GetAutoTableID(job.SchemaID, nt.ID) if err != nil { job.State = model.JobStateCancelled @@ -762,6 +770,7 @@ func (w *worker) onExchangeTablePartition(d *ddlCtx, t *meta.Meta, job *model.Jo } tempID := partDef.ID + // exchange table meta id partDef.ID = nt.ID err = t.UpdateTable(ptSchemaID, pt) @@ -770,6 +779,7 @@ func (w *worker) onExchangeTablePartition(d *ddlCtx, t *meta.Meta, job *model.Jo return ver, errors.Trace(err) } + // recreate non-partition table meta info err = t.DropTableOrView(job.SchemaID, nt.ID, true) if err != nil { job.State = model.JobStateCancelled From c979e69c980ee793502a7e91c1037cebfbad0344 Mon Sep 17 00:00:00 2001 From: zhaox1n Date: Mon, 18 May 2020 11:35:45 +0800 Subject: [PATCH 24/57] rename word --- ddl/db_partition_test.go | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/ddl/db_partition_test.go b/ddl/db_partition_test.go index 559447ce19554..af951a2ce33f0 100644 --- a/ddl/db_partition_test.go +++ b/ddl/db_partition_test.go @@ -822,8 +822,6 @@ func (s *testIntegrationSuite4) TestAlterTableExchangePartition(c *C) { tk.MustQuery("select * from e3 partition(p0)").Check(testkit.Rows("1", "5")) tk.MustQuery("select * from e2").Check(testkit.Rows()) - // expression index test for hash partition - } func (s *testIntegrationSuite4) TestExchangePartitionTableCompatiable(c *C) { @@ -890,7 +888,7 @@ func (s *testIntegrationSuite4) TestExchangePartitionTableCompatiable(c *C) { }, { // foreign key test - // Partition table is yet not supports to add foreign keys in mysql + // Partition table doesn't support to add foreign keys in mysql "create table pt9 (id int not null primary key auto_increment,t_id int not null) partition by hash(id) partitions 1;", "create table nt9 (id int not null primary key auto_increment, t_id int not null,foreign key fk_id (t_id) references pt5(id));", "alter table pt9 exchange partition p0 with table nt9;", @@ -1039,12 +1037,12 @@ func (s *testIntegrationSuite7) TestExchangePartitionExpressIndex(c *C) { tk.MustExec("alter table nt1 add column (`_V$_idx_0` bigint(20) generated always as (a+b) virtual);") tk.MustGetErrCode("alter table pt1 exchange partition p0 with table nt1;", tmysql.ErrTablesDifferentMetadata) - // test different expression index when expression return same field type + // test different expression index when expression returns same field type tk.MustExec("alter table nt1 drop column `_V$_idx_0`;") tk.MustExec("alter table nt1 add index idx((b-c));") tk.MustExec("alter table pt1 exchange partition p0 with table nt1;") - // test different expression index when expression return different field type + // test different expression index when expression returns different field type tk.MustExec("alter table nt1 drop index idx;") tk.MustExec("alter table nt1 add index idx((concat(a, b)));") tk.MustGetErrCode("alter table pt1 exchange partition p0 with table nt1;", tmysql.ErrTablesDifferentMetadata) From c096c8c1b3b52b2fa9ad8a5ae67d3265dc2acf19 Mon Sep 17 00:00:00 2001 From: zhaox1n Date: Mon, 18 May 2020 23:30:31 +0800 Subject: [PATCH 25/57] add global config of expression index --- ddl/db_partition_test.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/ddl/db_partition_test.go b/ddl/db_partition_test.go index af951a2ce33f0..759d256f3ab5e 100644 --- a/ddl/db_partition_test.go +++ b/ddl/db_partition_test.go @@ -27,6 +27,7 @@ import ( "github.com/pingcap/parser/ast" "github.com/pingcap/parser/model" "github.com/pingcap/parser/terror" + "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/ddl" "github.com/pingcap/tidb/ddl/testutil" "github.com/pingcap/tidb/domain" @@ -1024,6 +1025,7 @@ func (s *testIntegrationSuite4) TestExchangePartitionTableCompatiable(c *C) { } func (s *testIntegrationSuite7) TestExchangePartitionExpressIndex(c *C) { + config.GetGlobalConfig().Experimental.AllowsExpressionIndex = true tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists pt1;") From 488cb5f76b1ba313801708924095a23df379a361 Mon Sep 17 00:00:00 2001 From: zhaox1n Date: Fri, 22 May 2020 01:27:55 +0800 Subject: [PATCH 26/57] add auto_random test --- ddl/db_partition_test.go | 27 +++++++++++++++++++++------ ddl/ddl_api.go | 12 ++++++++---- ddl/ddl_worker_test.go | 2 +- ddl/partition.go | 10 +++------- 4 files changed, 33 insertions(+), 18 deletions(-) diff --git a/ddl/db_partition_test.go b/ddl/db_partition_test.go index c67075f870a35..13fb791e9e7c1 100644 --- a/ddl/db_partition_test.go +++ b/ddl/db_partition_test.go @@ -43,6 +43,7 @@ import ( "github.com/pingcap/tidb/util/admin" "github.com/pingcap/tidb/util/mock" "github.com/pingcap/tidb/util/testkit" + tutil "github.com/pingcap/tidb/util/testutil" ) func (s *testIntegrationSuite3) TestCreateTableWithPartition(c *C) { @@ -1019,12 +1020,7 @@ func (s *testIntegrationSuite4) TestExchangePartitionTableCompatiable(c *C) { tk.MustExec(t.ntSQL) if t.err != nil { _, err := tk.Exec(t.exchangeSQL) - originErr := errors.Cause(err) - tErr, ok := originErr.(*terror.Error) - c.Assert(ok, IsTrue, Commentf("case %d fail, sql = `%s`\nexpect type 'terror.Error', but obtain '%T'", i, - t.exchangeSQL, originErr)) - sqlErr := tErr.ToSQLError() - c.Assert(int(t.err.Code()), Equals, int(sqlErr.Code), Commentf( + c.Assert(terror.ErrorEqual(err, t.err), IsTrue, Commentf( "case %d fail, sql = `%s`\nexpected error = `%v`\n actual error = `%v`", i, t.exchangeSQL, t.err, err, )) @@ -1059,7 +1055,26 @@ func (s *testIntegrationSuite7) TestExchangePartitionExpressIndex(c *C) { tk.MustExec("alter table nt1 drop index idx;") tk.MustExec("alter table nt1 add index idx((concat(a, b)));") tk.MustGetErrCode("alter table pt1 exchange partition p0 with table nt1;", tmysql.ErrTablesDifferentMetadata) +} + +func (s *testIntegrationSuite7) TestExchangePartitionAutoRandom(c *C) { + tutil.ConfigTestUtils.SetupAutoRandomTestConfig() + defer tutil.ConfigTestUtils.RestoreAutoRandomTestConfig() + + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test;") + tk.MustExec("drop table if exists e1, e2, e3, e4;") + + tk.MustExec("create table e1 (a bigint primary key auto_random(3)) partition by hash(a) partitions 1;") + + tk.MustExec("create table e2 (a bigint primary key);") + tk.MustGetErrCode("alter table e1 exchange partition p0 with table e2;", tmysql.ErrTablesDifferentMetadata) + + tk.MustExec("create table e3 (a bigint primary key auto_random(2));") + tk.MustGetErrCode("alter table e1 exchange partition p0 with table e3;", tmysql.ErrTablesDifferentMetadata) + tk.MustExec("create table e4 (a bigint primary key auto_random(3));") + tk.MustExec("alter table e1 exchange partition p0 with table e4;") } func (s *testIntegrationSuite4) TestAddPartitionTooManyPartitions(c *C) { diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 11aae4a314823..1af9bb504242f 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -2210,7 +2210,7 @@ func (d *ddl) AlterTable(ctx sessionctx.Context, ident ast.Ident, specs []*ast.A case ast.AlterTableTruncatePartition: err = d.TruncateTablePartition(ctx, ident, spec) case ast.AlterTableExchangePartition: - err = d.ExchangeTableParition(ctx, ident, spec) + err = d.ExchangeTablePartition(ctx, ident, spec) case ast.AlterTableAddConstraint: constr := spec.Constraint switch spec.Constraint.Tp { @@ -2794,7 +2794,7 @@ func (d *ddl) DropTablePartition(ctx sessionctx.Context, ident ast.Ident, spec * return errors.Trace(err) } -func checkFielTypeCompatible(ft *types.FieldType, other *types.FieldType) bool { +func checkFieldTypeCompatible(ft *types.FieldType, other *types.FieldType) bool { // int(1) could match the type with int(8) partialEqual := ft.Tp == other.Tp && ft.Decimal == other.Decimal && @@ -2819,6 +2819,10 @@ func checkFielTypeCompatible(ft *types.FieldType, other *types.FieldType) bool { func checkTableDefCompatible(source *model.TableInfo, target *model.TableInfo) error { err := ErrTablesDifferentMetadata + // check auto_random + if source.AutoRandomBits != target.AutoRandomBits { + return err + } if len(source.Cols()) != len(target.Cols()) { return err } @@ -2830,7 +2834,7 @@ func checkTableDefCompatible(source *model.TableInfo, target *model.TableInfo) e } if sourceCol.Name.L != targetCol.Name.L || sourceCol.Hidden != targetCol.Hidden || - !checkFielTypeCompatible(&sourceCol.FieldType, &targetCol.FieldType) { + !checkFieldTypeCompatible(&sourceCol.FieldType, &targetCol.FieldType) { return err } } @@ -2890,7 +2894,7 @@ func checkExchangePartition(pt *model.TableInfo, nt *model.TableInfo) error { return nil } -func (d *ddl) ExchangeTableParition(ctx sessionctx.Context, ident ast.Ident, spec *ast.AlterTableSpec) error { +func (d *ddl) ExchangeTablePartition(ctx sessionctx.Context, ident ast.Ident, spec *ast.AlterTableSpec) error { is := d.infoHandle.Get() ptSchema, ok := is.SchemaByName(ident.Schema) if !ok { diff --git a/ddl/ddl_worker_test.go b/ddl/ddl_worker_test.go index 1d143b8aeec00..671349a8c72ab 100644 --- a/ddl/ddl_worker_test.go +++ b/ddl/ddl_worker_test.go @@ -990,7 +990,7 @@ func (s *testDDLSuite) TestCancelJob(c *C) { changedTable = testGetTable(c, d, dbInfo.ID, tblInfo.ID) c.Assert(checkIdxVisibility(changedTable, indexName, true), IsTrue) - // test exchange partition failed caused by canceled + // test exchange partition failed caused by canceled pt := testTableInfoWithPartition(c, d, "pt", 5) nt := testTableInfo(c, d, "nt", 5) testCreateTable(c, ctx, d, dbInfo, pt) diff --git a/ddl/partition.go b/ddl/partition.go index da8e2b7a106f4..3b676c532d074 100644 --- a/ddl/partition.go +++ b/ddl/partition.go @@ -820,6 +820,7 @@ func (w *worker) onExchangeTablePartition(d *ddlCtx, t *meta.Meta, job *model.Jo } func checkExchangePartitionRecordValidation(w *worker, pi *model.PartitionInfo, partName string, schemaName, tableName model.CIStr) error { + // make sure that pi contains the partName before this function runs var ( sql string index int @@ -830,9 +831,6 @@ func checkExchangePartitionRecordValidation(w *worker, pi *model.PartitionInfo, index = i break } - if len(pi.Definitions) == i { - return table.ErrUnknownPartition.GenWithStackByArgs(partName, tableName.O) - } } switch pi.Type { @@ -841,7 +839,6 @@ func checkExchangePartitionRecordValidation(w *worker, pi *model.PartitionInfo, return nil } sql = fmt.Sprintf("select 1 from `%s`.`%s` where mod(%s, %d) != %d limit 1", schemaName.L, tableName.L, pi.Expr, pi.Num, index) - break case model.PartitionTypeRange: rangeRrun, err := tables.DataForRangePruning(pi) if err != nil { @@ -858,9 +855,8 @@ func checkExchangePartitionRecordValidation(w *worker, pi *model.PartitionInfo, } else { sql = fmt.Sprintf("select 1 from `%s`.`%s` where %s < %d or %s >= %d limit 1", schemaName.L, tableName.L, pi.Expr, rangeRrun.LessThan[index-1], pi.Expr, rangeRrun.LessThan[index]) } - break default: - panic("cannot reach here") + return errors.Trace(errors.Errorf("unsupported partition type for checkExchangePartitionRecordValidation")) } var ctx sessionctx.Context @@ -876,7 +872,7 @@ func checkExchangePartitionRecordValidation(w *worker, pi *model.PartitionInfo, } rowCount := len(rows) if rowCount != 0 { - return ErrRowDoesNotMatchPartition + return errors.Trace(ErrRowDoesNotMatchPartition) } return nil } From b5cfdd32b7db199a55ee61f6efdfb918b6840615 Mon Sep 17 00:00:00 2001 From: zhaox1n Date: Tue, 26 May 2020 23:11:49 +0800 Subject: [PATCH 27/57] comment fix --- ddl/db_partition_test.go | 2 +- ddl/ddl_api.go | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/ddl/db_partition_test.go b/ddl/db_partition_test.go index c67075f870a35..fcedd5fceef42 100644 --- a/ddl/db_partition_test.go +++ b/ddl/db_partition_test.go @@ -823,7 +823,7 @@ func (s *testIntegrationSuite4) TestAlterTableExchangePartition(c *C) { tk.MustQuery("select * from e3 partition(p0)").Check(testkit.Rows()) tk.MustQuery("select * from e2").Check(testkit.Rows("1", "5")) - // validation test fot hash partition + // validation test for hash partition tk.MustGetErrCode("ALTER TABLE e3 EXCHANGE PARTITION p0 WITH TABLE e2", tmysql.ErrRowDoesNotMatchPartition) tk.MustGetErrCode("ALTER TABLE e3 EXCHANGE PARTITION p2 WITH TABLE e2", tmysql.ErrRowDoesNotMatchPartition) tk.MustGetErrCode("ALTER TABLE e3 EXCHANGE PARTITION p3 WITH TABLE e2", tmysql.ErrRowDoesNotMatchPartition) diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 11aae4a314823..58dd8fed7d92d 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -2818,7 +2818,7 @@ func checkFielTypeCompatible(ft *types.FieldType, other *types.FieldType) bool { } func checkTableDefCompatible(source *model.TableInfo, target *model.TableInfo) error { - err := ErrTablesDifferentMetadata + err := errors.Trace(ErrTablesDifferentMetadata) if len(source.Cols()) != len(target.Cols()) { return err } @@ -2860,8 +2860,8 @@ func checkTableDefCompatible(source *model.TableInfo, target *model.TableInfo) e } for i, sourceIdxCol := range sourceIdx.Columns { compatIdxCol := compatIdx.Columns[i] - if sourceIdxCol.Name.L != compatIdxCol.Name.L || - sourceIdxCol.Length != compatIdxCol.Length { + if sourceIdxCol.Length != compatIdxCol.Length || + sourceIdxCol.Name.L != compatIdxCol.Name.L { return err } } From be92bb33667815cc2f92d62fa7f1cc873e6f27e3 Mon Sep 17 00:00:00 2001 From: zhaox1n Date: Wed, 27 May 2020 02:01:04 +0800 Subject: [PATCH 28/57] more test added --- ddl/db_partition_test.go | 42 ++++++++++++++++++++++++++++++++++++++++ ddl/partition.go | 2 +- 2 files changed, 43 insertions(+), 1 deletion(-) diff --git a/ddl/db_partition_test.go b/ddl/db_partition_test.go index c2962ad947ee9..c1e5a474827b4 100644 --- a/ddl/db_partition_test.go +++ b/ddl/db_partition_test.go @@ -835,6 +835,48 @@ func (s *testIntegrationSuite4) TestAlterTableExchangePartition(c *C) { tk.MustQuery("select * from e3 partition(p0)").Check(testkit.Rows("1", "5")) tk.MustQuery("select * from e2").Check(testkit.Rows()) + // more boundary test of range partition + // for partition p0 + tk.MustExec(`create table e4 (a int) partition by range(a) ( + partition p0 values less than (3), + partition p1 values less than (6), + PARTITION p2 VALUES LESS THAN (9), + PARTITION p3 VALUES LESS THAN (MAXVALUE) + );`) + tk.MustExec(`create table e5(a int);`) + + tk.MustExec("insert into e5 values (1)") + + tk.MustGetErrCode("ALTER TABLE e4 EXCHANGE PARTITION p1 WITH TABLE e5", tmysql.ErrRowDoesNotMatchPartition) + tk.MustGetErrCode("ALTER TABLE e4 EXCHANGE PARTITION p2 WITH TABLE e5", tmysql.ErrRowDoesNotMatchPartition) + tk.MustGetErrCode("ALTER TABLE e4 EXCHANGE PARTITION p3 WITH TABLE e5", tmysql.ErrRowDoesNotMatchPartition) + tk.MustExec("ALTER TABLE e4 EXCHANGE PARTITION p0 with TABLE e5") + tk.MustQuery("select * from e4 partition(p0)").Check(testkit.Rows("1")) + + // for partition p1 + tk.MustExec("insert into e5 values (3)") + tk.MustGetErrCode("ALTER TABLE e4 EXCHANGE PARTITION p0 WITH TABLE e5", tmysql.ErrRowDoesNotMatchPartition) + tk.MustGetErrCode("ALTER TABLE e4 EXCHANGE PARTITION p2 WITH TABLE e5", tmysql.ErrRowDoesNotMatchPartition) + tk.MustGetErrCode("ALTER TABLE e4 EXCHANGE PARTITION p3 WITH TABLE e5", tmysql.ErrRowDoesNotMatchPartition) + tk.MustExec("ALTER TABLE e4 EXCHANGE PARTITION p1 with TABLE e5") + tk.MustQuery("select * from e4 partition(p1)").Check(testkit.Rows("3")) + + // for partition p2 + tk.MustExec("insert into e5 values (6)") + tk.MustGetErrCode("ALTER TABLE e4 EXCHANGE PARTITION p0 WITH TABLE e5", tmysql.ErrRowDoesNotMatchPartition) + tk.MustGetErrCode("ALTER TABLE e4 EXCHANGE PARTITION p1 WITH TABLE e5", tmysql.ErrRowDoesNotMatchPartition) + tk.MustGetErrCode("ALTER TABLE e4 EXCHANGE PARTITION p3 WITH TABLE e5", tmysql.ErrRowDoesNotMatchPartition) + tk.MustExec("ALTER TABLE e4 EXCHANGE PARTITION p2 with TABLE e5") + tk.MustQuery("select * from e4 partition(p2)").Check(testkit.Rows("6")) + + // for partition p3 + tk.MustExec("insert into e5 values (9)") + tk.MustGetErrCode("ALTER TABLE e4 EXCHANGE PARTITION p0 WITH TABLE e5", tmysql.ErrRowDoesNotMatchPartition) + tk.MustGetErrCode("ALTER TABLE e4 EXCHANGE PARTITION p1 WITH TABLE e5", tmysql.ErrRowDoesNotMatchPartition) + tk.MustGetErrCode("ALTER TABLE e4 EXCHANGE PARTITION p2 WITH TABLE e5", tmysql.ErrRowDoesNotMatchPartition) + tk.MustExec("ALTER TABLE e4 EXCHANGE PARTITION p3 with TABLE e5") + tk.MustQuery("select * from e4 partition(p3)").Check(testkit.Rows("9")) + } func (s *testIntegrationSuite4) TestExchangePartitionTableCompatiable(c *C) { diff --git a/ddl/partition.go b/ddl/partition.go index 3b676c532d074..0e17ba6c34aba 100644 --- a/ddl/partition.go +++ b/ddl/partition.go @@ -851,7 +851,7 @@ func checkExchangePartitionRecordValidation(w *worker, pi *model.PartitionInfo, if index == 0 { sql = fmt.Sprintf("select 1 from `%s`.`%s` where %s >= %d limit 1", schemaName.L, tableName.L, pi.Expr, rangeRrun.LessThan[index]) } else if index == len(pi.Definitions)-1 && rangeRrun.MaxValue { - sql = fmt.Sprintf("select 1 from `%s`.`%s` where %s <= %d limit 1", schemaName.L, tableName.L, pi.Expr, rangeRrun.LessThan[index-1]) + sql = fmt.Sprintf("select 1 from `%s`.`%s` where %s < %d limit 1", schemaName.L, tableName.L, pi.Expr, rangeRrun.LessThan[index-1]) } else { sql = fmt.Sprintf("select 1 from `%s`.`%s` where %s < %d or %s >= %d limit 1", schemaName.L, tableName.L, pi.Expr, rangeRrun.LessThan[index-1], pi.Expr, rangeRrun.LessThan[index]) } From d9bc6faac59f90efec3c1fe40cdb7870ffd0ac04 Mon Sep 17 00:00:00 2001 From: zhaox1n Date: Wed, 27 May 2020 03:02:38 +0800 Subject: [PATCH 29/57] add failed point --- ddl/failtest/fail_db_test.go | 51 ++++++++++++++++++++++++++++++++++++ ddl/partition.go | 31 +++++++++++++--------- 2 files changed, 70 insertions(+), 12 deletions(-) diff --git a/ddl/failtest/fail_db_test.go b/ddl/failtest/fail_db_test.go index 83154f20fd8cf..9bcf165a63bab 100644 --- a/ddl/failtest/fail_db_test.go +++ b/ddl/failtest/fail_db_test.go @@ -165,6 +165,57 @@ func (s *testFailDBSuite) TestHalfwayCancelOperations(c *C) { _, err = s.se.Execute(context.Background(), "select * from tx") c.Assert(err, IsNil) + // test for exchanging partition + c.Assert(failpoint.Enable("github.com/pingcap/tidb/ddl/exchangePartitionErr", `return(true)`), IsNil) + defer func() { + c.Assert(failpoint.Disable("github.com/pingcap/tidb/ddl/exchangePartitionErr"), IsNil) + }() + _, err = s.se.Execute(context.Background(), "create table pt(a int) partition by hash (a) partitions 2") + c.Assert(err, IsNil) + _, err = s.se.Execute(context.Background(), "insert into pt values(1), (3), (5)") + c.Assert(err, IsNil) + _, err = s.se.Execute(context.Background(), "create table nt(a int)") + c.Assert(err, IsNil) + _, err = s.se.Execute(context.Background(), "insert into nt values(7)") + c.Assert(err, IsNil) + _, err = s.se.Execute(context.Background(), "alter table pt exchange partition p1 with table nt") + c.Assert(err, NotNil) + + rs, err = s.se.Execute(context.Background(), "select count(*) from pt") + c.Assert(err, IsNil) + req = rs[0].NewChunk() + err = rs[0].Next(context.Background(), req) + c.Assert(err, IsNil) + c.Assert(req.NumRows() == 0, IsFalse) + row = req.GetRow(0) + c.Assert(row.Len(), Equals, 1) + c.Assert(row.GetInt64(0), DeepEquals, int64(3)) + c.Assert(rs[0].Close(), IsNil) + + rs, err = s.se.Execute(context.Background(), "select count(*) from nt") + c.Assert(err, IsNil) + req = rs[0].NewChunk() + err = rs[0].Next(context.Background(), req) + c.Assert(err, IsNil) + c.Assert(req.NumRows() == 0, IsFalse) + row = req.GetRow(0) + c.Assert(row.Len(), Equals, 1) + c.Assert(row.GetInt64(0), DeepEquals, int64(1)) + c.Assert(rs[0].Close(), IsNil) + + // Execute ddl statement reload schema. + _, err = s.se.Execute(context.Background(), "alter table pt comment 'pt'") + c.Assert(err, IsNil) + err = s.dom.DDL().GetHook().OnChanged(nil) + c.Assert(err, IsNil) + s.se, err = session.CreateSession4Test(s.store) + c.Assert(err, IsNil) + _, err = s.se.Execute(context.Background(), "use cancel_job_db") + c.Assert(err, IsNil) + // Test schema is correct. + _, err = s.se.Execute(context.Background(), "select * from pt") + c.Assert(err, IsNil) + // clean up _, err = s.se.Execute(context.Background(), "drop database cancel_job_db") c.Assert(err, IsNil) diff --git a/ddl/partition.go b/ddl/partition.go index 0e17ba6c34aba..31c90f5dab6f3 100644 --- a/ddl/partition.go +++ b/ddl/partition.go @@ -20,6 +20,7 @@ import ( "strings" "github.com/pingcap/errors" + "github.com/pingcap/failpoint" "github.com/pingcap/parser" "github.com/pingcap/parser/ast" "github.com/pingcap/parser/format" @@ -600,14 +601,14 @@ func removePartitionInfo(tblInfo *model.TableInfo, partName string) int64 { return pid } -func getPartitionDef(tblInfo *model.TableInfo, partName string) (*model.PartitionDefinition, error) { +func getPartitionDef(tblInfo *model.TableInfo, partName string) (index int, def *model.PartitionDefinition, _ error) { defs := tblInfo.Partition.Definitions for i := 0; i < len(defs); i++ { if strings.EqualFold(defs[i].Name.L, strings.ToLower(partName)) { - return &(defs[i]), nil + return i, &(defs[i]), nil } } - return nil, table.ErrUnknownPartition.GenWithStackByArgs(partName, tblInfo.Name.O) + return index, nil, table.ErrUnknownPartition.GenWithStackByArgs(partName, tblInfo.Name.O) } // onDropTablePartition deletes old partition meta. @@ -746,7 +747,7 @@ func (w *worker) onExchangeTablePartition(d *ddlCtx, t *meta.Meta, job *model.Jo } if withValidation { - err = checkExchangePartitionRecordValidation(w, pt.Partition, partName, ntDbInfo.Name, nt.Name) + err = checkExchangePartitionRecordValidation(w, pt, partName, ntDbInfo.Name, nt.Name) if err != nil { job.State = model.JobStateCancelled return ver, errors.Trace(err) @@ -767,7 +768,7 @@ func (w *worker) onExchangeTablePartition(d *ddlCtx, t *meta.Meta, job *model.Jo return ver, errors.Trace(err) } - partDef, err := getPartitionDef(pt, partName) + _, partDef, err := getPartitionDef(pt, partName) if err != nil { job.State = model.JobStateCancelled return ver, errors.Trace(err) @@ -783,6 +784,13 @@ func (w *worker) onExchangeTablePartition(d *ddlCtx, t *meta.Meta, job *model.Jo return ver, errors.Trace(err) } + failpoint.Inject("exchangePartitionErr", func(val failpoint.Value) { + if val.(bool) { + job.State = model.JobStateCancelled + failpoint.Return(ver, errors.New("occur an error after updating partition id")) + } + }) + // recreate non-partition table meta info err = t.DropTableOrView(job.SchemaID, nt.ID, true) if err != nil { @@ -819,18 +827,17 @@ func (w *worker) onExchangeTablePartition(d *ddlCtx, t *meta.Meta, job *model.Jo return ver, nil } -func checkExchangePartitionRecordValidation(w *worker, pi *model.PartitionInfo, partName string, schemaName, tableName model.CIStr) error { +func checkExchangePartitionRecordValidation(w *worker, pt *model.TableInfo, partName string, schemaName, tableName model.CIStr) error { // make sure that pi contains the partName before this function runs var ( sql string index int ) - for i, def := range pi.Definitions { - if strings.EqualFold(def.Name.L, strings.ToLower(partName)) { - index = i - break - } + pi := pt.Partition + index, _, err := getPartitionDef(pt, partName) + if err != nil { + return errors.Trace(err) } switch pi.Type { @@ -860,7 +867,7 @@ func checkExchangePartitionRecordValidation(w *worker, pi *model.PartitionInfo, } var ctx sessionctx.Context - ctx, err := w.sessPool.get() + ctx, err = w.sessPool.get() if err != nil { return errors.Trace(err) } From ea173bf6b134580ffa5278d4d99e0b3373297e32 Mon Sep 17 00:00:00 2001 From: zhaox1n Date: Wed, 27 May 2020 03:08:03 +0800 Subject: [PATCH 30/57] remove variable --- ddl/partition.go | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/ddl/partition.go b/ddl/partition.go index 31c90f5dab6f3..1bec724e3e5ff 100644 --- a/ddl/partition.go +++ b/ddl/partition.go @@ -829,10 +829,7 @@ func (w *worker) onExchangeTablePartition(d *ddlCtx, t *meta.Meta, job *model.Jo func checkExchangePartitionRecordValidation(w *worker, pt *model.TableInfo, partName string, schemaName, tableName model.CIStr) error { // make sure that pi contains the partName before this function runs - var ( - sql string - index int - ) + var sql string pi := pt.Partition index, _, err := getPartitionDef(pt, partName) From 57aeb95853ac1b3a81548f15789ffbc706fc98be Mon Sep 17 00:00:00 2001 From: Zhao Xin Date: Wed, 27 May 2020 12:00:08 +0800 Subject: [PATCH 31/57] Update ddl/column.go Co-authored-by: Lynn --- ddl/column.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ddl/column.go b/ddl/column.go index 2448535592d24..9b27e035912e4 100644 --- a/ddl/column.go +++ b/ddl/column.go @@ -892,7 +892,7 @@ func getColumnInfoByName(tbInfo *model.TableInfo, column string) *model.ColumnIn return nil } -// check column if it is virtual +// isVirtualGeneratedColumn checks the column if it is virtual. func isVirtualGeneratedColumn(col *model.ColumnInfo) bool { if col.IsGenerated() && !col.GeneratedStored { return true From 5a29bdd7973a3423c0f163adadcd26752c616fd7 Mon Sep 17 00:00:00 2001 From: Zhao Xin Date: Wed, 27 May 2020 12:00:21 +0800 Subject: [PATCH 32/57] Update ddl/ddl_api.go Co-authored-by: Lynn --- ddl/ddl_api.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 55bc356e3efc3..c05466f6982c9 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -2844,7 +2844,7 @@ func checkTableDefCompatible(source *model.TableInfo, target *model.TableInfo) e for _, sourceIdx := range source.Indices { var compatIdx *model.IndexInfo for _, targetIdx := range target.Indices { - if strings.EqualFold(sourceIdx.Name.O, targetIdx.Name.O) { + if strings.EqualFold(sourceIdx.Name.L, targetIdx.Name.L) { compatIdx = targetIdx } } From aa8da8eb9fb1a2e094e0323c28c291edd3337b58 Mon Sep 17 00:00:00 2001 From: zhaox1n Date: Wed, 27 May 2020 23:49:37 +0800 Subject: [PATCH 33/57] comment fix --- ddl/db_partition_test.go | 21 --------------------- ddl/ddl_api.go | 37 ++++++++++++------------------------- ddl/partition.go | 14 +++++++++----- ddl/serial_test.go | 14 ++++++++++++++ 4 files changed, 35 insertions(+), 51 deletions(-) diff --git a/ddl/db_partition_test.go b/ddl/db_partition_test.go index c1e5a474827b4..f33ce759fde53 100644 --- a/ddl/db_partition_test.go +++ b/ddl/db_partition_test.go @@ -43,7 +43,6 @@ import ( "github.com/pingcap/tidb/util/admin" "github.com/pingcap/tidb/util/mock" "github.com/pingcap/tidb/util/testkit" - tutil "github.com/pingcap/tidb/util/testutil" ) func (s *testIntegrationSuite3) TestCreateTableWithPartition(c *C) { @@ -1099,26 +1098,6 @@ func (s *testIntegrationSuite7) TestExchangePartitionExpressIndex(c *C) { tk.MustGetErrCode("alter table pt1 exchange partition p0 with table nt1;", tmysql.ErrTablesDifferentMetadata) } -func (s *testIntegrationSuite7) TestExchangePartitionAutoRandom(c *C) { - tutil.ConfigTestUtils.SetupAutoRandomTestConfig() - defer tutil.ConfigTestUtils.RestoreAutoRandomTestConfig() - - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test;") - tk.MustExec("drop table if exists e1, e2, e3, e4;") - - tk.MustExec("create table e1 (a bigint primary key auto_random(3)) partition by hash(a) partitions 1;") - - tk.MustExec("create table e2 (a bigint primary key);") - tk.MustGetErrCode("alter table e1 exchange partition p0 with table e2;", tmysql.ErrTablesDifferentMetadata) - - tk.MustExec("create table e3 (a bigint primary key auto_random(2));") - tk.MustGetErrCode("alter table e1 exchange partition p0 with table e3;", tmysql.ErrTablesDifferentMetadata) - - tk.MustExec("create table e4 (a bigint primary key auto_random(3));") - tk.MustExec("alter table e1 exchange partition p0 with table e4;") -} - func (s *testIntegrationSuite4) TestAddPartitionTooManyPartitions(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index c05466f6982c9..c4149819533a5 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -2818,13 +2818,12 @@ func checkFieldTypeCompatible(ft *types.FieldType, other *types.FieldType) bool } func checkTableDefCompatible(source *model.TableInfo, target *model.TableInfo) error { - err := errors.Trace(ErrTablesDifferentMetadata) // check auto_random if source.AutoRandomBits != target.AutoRandomBits { - return err + return errors.Trace(ErrTablesDifferentMetadata) } if len(source.Cols()) != len(target.Cols()) { - return err + return errors.Trace(ErrTablesDifferentMetadata) } // Col compatible check for i, sourceCol := range source.Cols() { @@ -2835,11 +2834,11 @@ func checkTableDefCompatible(source *model.TableInfo, target *model.TableInfo) e if sourceCol.Name.L != targetCol.Name.L || sourceCol.Hidden != targetCol.Hidden || !checkFieldTypeCompatible(&sourceCol.FieldType, &targetCol.FieldType) { - return err + return errors.Trace(ErrTablesDifferentMetadata) } } if len(source.Indices) != len(target.Indices) { - return err + return errors.Trace(ErrTablesDifferentMetadata) } for _, sourceIdx := range source.Indices { var compatIdx *model.IndexInfo @@ -2850,23 +2849,23 @@ func checkTableDefCompatible(source *model.TableInfo, target *model.TableInfo) e } // No match index if compatIdx == nil { - return err + return errors.Trace(ErrTablesDifferentMetadata) } // Index type is not compatible if sourceIdx.Tp != compatIdx.Tp || sourceIdx.Unique != compatIdx.Unique || sourceIdx.Primary != compatIdx.Primary { - return err + return errors.Trace(ErrTablesDifferentMetadata) } // The index column if len(sourceIdx.Columns) != len(compatIdx.Columns) { - return err + return errors.Trace(ErrTablesDifferentMetadata) } for i, sourceIdxCol := range sourceIdx.Columns { compatIdxCol := compatIdx.Columns[i] if sourceIdxCol.Length != compatIdxCol.Length || sourceIdxCol.Name.L != compatIdxCol.Name.L { - return err + return errors.Trace(ErrTablesDifferentMetadata) } } } @@ -2875,7 +2874,6 @@ func checkTableDefCompatible(source *model.TableInfo, target *model.TableInfo) e } func checkExchangePartition(pt *model.TableInfo, nt *model.TableInfo) error { - if nt.IsView() || nt.IsSequence() { return errors.Trace(ErrCheckNoSuchTable) } @@ -2895,28 +2893,17 @@ func checkExchangePartition(pt *model.TableInfo, nt *model.TableInfo) error { } func (d *ddl) ExchangeTablePartition(ctx sessionctx.Context, ident ast.Ident, spec *ast.AlterTableSpec) error { - is := d.infoHandle.Get() - ptSchema, ok := is.SchemaByName(ident.Schema) - if !ok { - return errors.Trace(infoschema.ErrDatabaseNotExists.GenWithStackByArgs(ptSchema)) - } - pt, err := is.TableByName(ident.Schema, ident.Name) + ptSchema, pt, err := d.getSchemaAndTableByIdent(ctx, ident) if err != nil { - return errors.Trace(infoschema.ErrTableNotExists.GenWithStackByArgs(ident.Schema, ident.Name)) + return errors.Trace(err) } ptMeta := pt.Meta() ntIdent := ast.Ident{Schema: spec.NewTable.Schema, Name: spec.NewTable.Name} - - ntSchema, ok := is.SchemaByName(ntIdent.Schema) - if !ok { - return errors.Trace(infoschema.ErrDatabaseNotExists.GenWithStackByArgs(ntSchema)) - } - - nt, err := is.TableByName(ntIdent.Schema, ntIdent.Name) + ntSchema, nt, err := d.getSchemaAndTableByIdent(ctx, ntIdent) if err != nil { - return errors.Trace(infoschema.ErrTableNotExists.GenWithStackByArgs(ntIdent.Schema, ntIdent.Name)) + return errors.Trace(err) } ntMeta := nt.Meta() diff --git a/ddl/partition.go b/ddl/partition.go index 1bec724e3e5ff..6fd0d59b28215 100644 --- a/ddl/partition.go +++ b/ddl/partition.go @@ -19,6 +19,7 @@ import ( "strconv" "strings" + "github.com/cznic/mathutil" "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/parser" @@ -806,13 +807,16 @@ func (w *worker) onExchangeTablePartition(d *ddlCtx, t *meta.Meta, job *model.Jo return ver, errors.Trace(err) } - _, err = t.GenAutoTableID(ptSchemaID, pt.ID, ntBaseID) - if err != nil { - job.State = model.JobStateCancelled - return ver, errors.Trace(err) + // both pt and nt set the maximum auto_id between ntBaseID and ptBaseID + if ntBaseID > ptBaseID { + _, err = t.GenAutoTableID(ptSchemaID, pt.ID, ntBaseID-ptBaseID) + if err != nil { + job.State = model.JobStateCancelled + return ver, errors.Trace(err) + } } - _, err = t.GenAutoTableID(job.SchemaID, nt.ID, ptBaseID) + _, err = t.GenAutoTableID(job.SchemaID, nt.ID, mathutil.MaxInt64(ptBaseID, ntBaseID)) if err != nil { job.State = model.JobStateCancelled return ver, errors.Trace(err) diff --git a/ddl/serial_test.go b/ddl/serial_test.go index ffdd07206bd80..95a8dace32923 100644 --- a/ddl/serial_test.go +++ b/ddl/serial_test.go @@ -957,6 +957,20 @@ func (s *testSerialSuite) TestAutoRandom(c *C) { tk.MustExec("insert into t values()") }) + // Test exchange partition + tk.MustExec("drop table if exists e1, e2, e3, e4;") + + tk.MustExec("create table e1 (a bigint primary key auto_random(3)) partition by hash(a) partitions 1;") + + tk.MustExec("create table e2 (a bigint primary key);") + tk.MustGetErrCode("alter table e1 exchange partition p0 with table e2;", errno.ErrTablesDifferentMetadata) + + tk.MustExec("create table e3 (a bigint primary key auto_random(2));") + tk.MustGetErrCode("alter table e1 exchange partition p0 with table e3;", errno.ErrTablesDifferentMetadata) + + tk.MustExec("create table e4 (a bigint primary key auto_random(3));") + tk.MustExec("alter table e1 exchange partition p0 with table e4;") + // Disallow using it when allow-auto-random is not enabled. config.GetGlobalConfig().Experimental.AllowAutoRandom = false assertExperimentDisabled("create table auto_random_table (a int primary key auto_random(3))") From ebce34137d95705e69bf00a2e6cbf25e2b555c55 Mon Sep 17 00:00:00 2001 From: zhaox1n Date: Thu, 28 May 2020 00:21:19 +0800 Subject: [PATCH 34/57] add test --- ddl/db_partition_test.go | 35 +++++++++++++++++++++++++++++++++++ 1 file changed, 35 insertions(+) diff --git a/ddl/db_partition_test.go b/ddl/db_partition_test.go index f33ce759fde53..c536e77148d43 100644 --- a/ddl/db_partition_test.go +++ b/ddl/db_partition_test.go @@ -2204,4 +2204,39 @@ func (s *testIntegrationSuite3) TestCommitWhenSchemaChange(c *C) { // That bug will cause data and index inconsistency! tk.MustExec("admin check table schema_change") tk.MustQuery("select * from schema_change").Check(testkit.Rows()) + + // Check iconsistency when exchanging partition + tk.MustExec(`drop table if exists pt, nt;`) + tk.MustExec(`create table pt (a int) partition by hash(a) partitions 2;`) + tk.MustExec(`create table nt (a int);`) + + tk.MustExec("begin") + tk.MustExec("insert into nt values (1), (3), (5);") + tk2.MustExec("alter table pt exchange partition p1 with table nt;") + tk.MustExec("insert into nt values (7), (9);") + + atomic.StoreUint32(&session.SchemaChangedWithoutRetry, 1) + _, err = tk.Se.Execute(context.Background(), "commit") + atomic.StoreUint32(&session.SchemaChangedWithoutRetry, 0) + c.Assert(domain.ErrInfoSchemaChanged.Equal(err), IsTrue) + + tk.MustExec("admin check table pt") + tk.MustQuery("select * from pt").Check(testkit.Rows()) + tk.MustExec("admin check table nt") + tk.MustQuery("select * from nt").Check(testkit.Rows()) + + tk.MustExec("begin") + tk.MustExec("insert into pt values (1), (3), (5);") + tk2.MustExec("alter table pt exchange partition p1 with table nt;") + tk.MustExec("insert into pt values (7), (9);") + atomic.StoreUint32(&session.SchemaChangedWithoutRetry, 1) + _, err = tk.Se.Execute(context.Background(), "commit") + atomic.StoreUint32(&session.SchemaChangedWithoutRetry, 0) + c.Assert(domain.ErrInfoSchemaChanged.Equal(err), IsTrue) + + tk.MustExec("admin check table pt") + tk.MustQuery("select * from pt").Check(testkit.Rows()) + tk.MustExec("admin check table nt") + tk.MustQuery("select * from nt").Check(testkit.Rows()) + } From 65f4c64b6f57cf58441a0d61cfe16ce341d578de Mon Sep 17 00:00:00 2001 From: zhaox1n Date: Thu, 28 May 2020 00:33:30 +0800 Subject: [PATCH 35/57] add test --- ddl/db_partition_test.go | 1 - 1 file changed, 1 deletion(-) diff --git a/ddl/db_partition_test.go b/ddl/db_partition_test.go index c536e77148d43..3cae3340072e8 100644 --- a/ddl/db_partition_test.go +++ b/ddl/db_partition_test.go @@ -2238,5 +2238,4 @@ func (s *testIntegrationSuite3) TestCommitWhenSchemaChange(c *C) { tk.MustQuery("select * from pt").Check(testkit.Rows()) tk.MustExec("admin check table nt") tk.MustQuery("select * from nt").Check(testkit.Rows()) - } From 5a46b18329776ed2201b60076404b6c4e0f38f1f Mon Sep 17 00:00:00 2001 From: zhaox1n Date: Thu, 28 May 2020 01:32:50 +0800 Subject: [PATCH 36/57] exchange auto_random --- ddl/partition.go | 28 ++++++++++++++++++++++++++++ 1 file changed, 28 insertions(+) diff --git a/ddl/partition.go b/ddl/partition.go index 6fd0d59b28215..feae130f76dcf 100644 --- a/ddl/partition.go +++ b/ddl/partition.go @@ -762,6 +762,12 @@ func (w *worker) onExchangeTablePartition(d *ddlCtx, t *meta.Meta, job *model.Jo return ver, errors.Trace(err) } + ptRandID, err := t.GetAutoRandomID(ptSchemaID, pt.ID) + if err != nil { + job.State = model.JobStateCancelled + return ver, errors.Trace(err) + } + // non-partition table base auto id ntBaseID, err := t.GetAutoTableID(job.SchemaID, nt.ID) if err != nil { @@ -769,6 +775,12 @@ func (w *worker) onExchangeTablePartition(d *ddlCtx, t *meta.Meta, job *model.Jo return ver, errors.Trace(err) } + ntRandID, err := t.GetAutoRandomID(job.SchemaID, nt.ID) + if err != nil { + job.State = model.JobStateCancelled + return ver, errors.Trace(err) + } + _, partDef, err := getPartitionDef(pt, partName) if err != nil { job.State = model.JobStateCancelled @@ -822,6 +834,22 @@ func (w *worker) onExchangeTablePartition(d *ddlCtx, t *meta.Meta, job *model.Jo return ver, errors.Trace(err) } + if ntRandID != 0 && ptRandID != 0 { + if ntRandID > ptRandID { + _, err = t.GenAutoRandomID(ptSchemaID, pt.ID, ntRandID-ptRandID) + if err != nil { + job.State = model.JobStateCancelled + return ver, errors.Trace(err) + } + } + + _, err = t.GenAutoRandomID(job.SchemaID, nt.ID, mathutil.MaxInt64(ptRandID, ntRandID)) + if err != nil { + job.State = model.JobStateCancelled + return ver, errors.Trace(err) + } + } + ver, err = updateSchemaVersion(t, job) if err != nil { return ver, errors.Trace(err) From ec51f5e73e75523803aed197e8a95fb477ebe678 Mon Sep 17 00:00:00 2001 From: zhaox1n Date: Fri, 29 May 2020 01:41:45 +0800 Subject: [PATCH 37/57] fix range columns partition --- ddl/db_partition_test.go | 15 ++++++++++++++- ddl/partition.go | 33 ++++++++++++++++++++++++++------- 2 files changed, 40 insertions(+), 8 deletions(-) diff --git a/ddl/db_partition_test.go b/ddl/db_partition_test.go index 3cae3340072e8..d3c66e5116dc8 100644 --- a/ddl/db_partition_test.go +++ b/ddl/db_partition_test.go @@ -872,10 +872,23 @@ func (s *testIntegrationSuite4) TestAlterTableExchangePartition(c *C) { tk.MustExec("insert into e5 values (9)") tk.MustGetErrCode("ALTER TABLE e4 EXCHANGE PARTITION p0 WITH TABLE e5", tmysql.ErrRowDoesNotMatchPartition) tk.MustGetErrCode("ALTER TABLE e4 EXCHANGE PARTITION p1 WITH TABLE e5", tmysql.ErrRowDoesNotMatchPartition) - tk.MustGetErrCode("ALTER TABLE e4 EXCHANGE PARTITION p2 WITH TABLE e5", tmysql.ErrRowDoesNotMatchPartition) + tk.MustGetErrCode("alter table e4 exchange partition p2 with table e5", tmysql.ErrRowDoesNotMatchPartition) tk.MustExec("ALTER TABLE e4 EXCHANGE PARTITION p3 with TABLE e5") tk.MustQuery("select * from e4 partition(p3)").Check(testkit.Rows("9")) + // for columns range partition + tk.MustExec(`create table e6 (a int) partition by range columns (a) ( + partition p0 values less than (3), + partition p1 values less than (6) + );`) + tk.MustExec(`create table e7 (a int);`) + tk.MustExec(`insert into e6 values (1);`) + tk.MustExec(`insert into e7 values (2);`) + tk.MustExec("alter table e6 exchange partition p0 with table e7") + + tk.MustQuery("select * from e6 partition(p0)").Check(testkit.Rows("2")) + tk.MustQuery("select * from e7").Check(testkit.Rows("1")) + tk.MustGetErrCode("alter table e6 exchange partition p1 with table e7", tmysql.ErrRowDoesNotMatchPartition) } func (s *testIntegrationSuite4) TestExchangePartitionTableCompatiable(c *C) { diff --git a/ddl/partition.go b/ddl/partition.go index feae130f76dcf..d0a29c5c1b1f0 100644 --- a/ddl/partition.go +++ b/ddl/partition.go @@ -860,7 +860,6 @@ func (w *worker) onExchangeTablePartition(d *ddlCtx, t *meta.Meta, job *model.Jo } func checkExchangePartitionRecordValidation(w *worker, pt *model.TableInfo, partName string, schemaName, tableName model.CIStr) error { - // make sure that pi contains the partName before this function runs var sql string pi := pt.Partition @@ -884,12 +883,11 @@ func checkExchangePartitionRecordValidation(w *worker, pt *model.TableInfo, part if len(pi.Definitions) == 1 && rangeRrun.MaxValue { return nil } - if index == 0 { - sql = fmt.Sprintf("select 1 from `%s`.`%s` where %s >= %d limit 1", schemaName.L, tableName.L, pi.Expr, rangeRrun.LessThan[index]) - } else if index == len(pi.Definitions)-1 && rangeRrun.MaxValue { - sql = fmt.Sprintf("select 1 from `%s`.`%s` where %s < %d limit 1", schemaName.L, tableName.L, pi.Expr, rangeRrun.LessThan[index-1]) - } else { - sql = fmt.Sprintf("select 1 from `%s`.`%s` where %s < %d or %s >= %d limit 1", schemaName.L, tableName.L, pi.Expr, rangeRrun.LessThan[index-1], pi.Expr, rangeRrun.LessThan[index]) + // For range expression and range columns + if len(pi.Columns) == 0 { + sql = buildCheckSQLForRangeExprPartition(pi, rangeRrun, index, schemaName, tableName) + } else if len(pi.Columns) == 1 { + sql = buildCheckSQLForRangeColumnsPartition(pi, rangeRrun, index, schemaName, tableName) } default: return errors.Trace(errors.Errorf("unsupported partition type for checkExchangePartitionRecordValidation")) @@ -913,6 +911,27 @@ func checkExchangePartitionRecordValidation(w *worker, pt *model.TableInfo, part return nil } +func buildCheckSQLForRangeExprPartition(pi *model.PartitionInfo, rangeRrun *tables.ForRangePruning, index int, schemaName, tableName model.CIStr) string { + if index == 0 { + return fmt.Sprintf("select 1 from `%s`.`%s` where %s >= %d limit 1", schemaName.L, tableName.L, pi.Expr, rangeRrun.LessThan[index]) + } else if index == len(pi.Definitions)-1 && rangeRrun.MaxValue { + return fmt.Sprintf("select 1 from `%s`.`%s` where %s < %d limit 1", schemaName.L, tableName.L, pi.Expr, rangeRrun.LessThan[index-1]) + } else { + return fmt.Sprintf("select 1 from `%s`.`%s` where %s < %d or %s >= %d limit 1", schemaName.L, tableName.L, pi.Expr, rangeRrun.LessThan[index-1], pi.Expr, rangeRrun.LessThan[index]) + } +} + +func buildCheckSQLForRangeColumnsPartition(pi *model.PartitionInfo, rangeRrun *tables.ForRangePruning, index int, schemaName, tableName model.CIStr) string { + colName := pi.Columns[0].L + if index == 0 { + return fmt.Sprintf("select 1 from `%s`.`%s` where `%s` >= %d limit 1", schemaName.L, tableName.L, colName, rangeRrun.LessThan[index]) + } else if index == len(pi.Definitions)-1 && rangeRrun.MaxValue { + return fmt.Sprintf("select 1 from `%s`.`%s` where `%s` < %d limit 1", schemaName.L, tableName.L, colName, rangeRrun.LessThan[index-1]) + } else { + return fmt.Sprintf("select 1 from `%s`.`%s` where `%s` < %d or `%s` >= %d limit 1", schemaName.L, tableName.L, colName, rangeRrun.LessThan[index-1], colName, rangeRrun.LessThan[index]) + } +} + func checkAddPartitionTooManyPartitions(piDefs uint64) error { if piDefs > uint64(PartitionCountLimit) { return errors.Trace(ErrTooManyPartitions) From 95b25549a96e99ed2ba15032dabb13a777d985f3 Mon Sep 17 00:00:00 2001 From: zhaox1n Date: Fri, 29 May 2020 14:21:26 +0800 Subject: [PATCH 38/57] fix comment --- ddl/db_partition_test.go | 12 ++++++------ ddl/partition.go | 28 +++++++++++++--------------- table/tables/partition.go | 6 +++--- 3 files changed, 22 insertions(+), 24 deletions(-) diff --git a/ddl/db_partition_test.go b/ddl/db_partition_test.go index 245fa27d2543e..2f9af16de4f7e 100644 --- a/ddl/db_partition_test.go +++ b/ddl/db_partition_test.go @@ -884,13 +884,13 @@ func (s *testIntegrationSuite4) TestAlterTableExchangePartition(c *C) { tk.MustQuery("select * from e4 partition(p3)").Check(testkit.Rows("9")) // for columns range partition - tk.MustExec(`create table e6 (a int) partition by range columns (a) ( - partition p0 values less than (3), - partition p1 values less than (6) + tk.MustExec(`create table e6 (a varchar(3)) partition by range columns (a) ( + partition p0 values less than ('3'), + partition p1 values less than ('6') );`) - tk.MustExec(`create table e7 (a int);`) - tk.MustExec(`insert into e6 values (1);`) - tk.MustExec(`insert into e7 values (2);`) + tk.MustExec(`create table e7 (a varchar(3));`) + tk.MustExec(`insert into e6 values ('1');`) + tk.MustExec(`insert into e7 values ('2');`) tk.MustExec("alter table e6 exchange partition p0 with table e7") tk.MustQuery("select * from e6 partition(p0)").Check(testkit.Rows("2")) diff --git a/ddl/partition.go b/ddl/partition.go index 70ff2ebb8290b..6bccc44f31d33 100644 --- a/ddl/partition.go +++ b/ddl/partition.go @@ -34,7 +34,6 @@ import ( "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/table" - "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/sqlexec" @@ -882,19 +881,18 @@ func checkExchangePartitionRecordValidation(w *worker, pt *model.TableInfo, part } sql = fmt.Sprintf("select 1 from `%s`.`%s` where mod(%s, %d) != %d limit 1", schemaName.L, tableName.L, pi.Expr, pi.Num, index) case model.PartitionTypeRange: - rangeRrun, err := tables.DataForRangePruning(pi) if err != nil { return errors.Trace(err) } // Table has only one partition and has the maximum value - if len(pi.Definitions) == 1 && rangeRrun.MaxValue { + if len(pi.Definitions) == 1 && strings.EqualFold(pi.Definitions[index].LessThan[0], "MAXVALUE") { return nil } // For range expression and range columns if len(pi.Columns) == 0 { - sql = buildCheckSQLForRangeExprPartition(pi, rangeRrun, index, schemaName, tableName) + sql = buildCheckSQLForRangeExprPartition(pi, index, schemaName, tableName) } else if len(pi.Columns) == 1 { - sql = buildCheckSQLForRangeColumnsPartition(pi, rangeRrun, index, schemaName, tableName) + sql = buildCheckSQLForRangeColumnsPartition(pi, index, schemaName, tableName) } default: return errors.Trace(errors.Errorf("unsupported partition type for checkExchangePartitionRecordValidation")) @@ -918,24 +916,24 @@ func checkExchangePartitionRecordValidation(w *worker, pt *model.TableInfo, part return nil } -func buildCheckSQLForRangeExprPartition(pi *model.PartitionInfo, rangeRrun *tables.ForRangePruning, index int, schemaName, tableName model.CIStr) string { +func buildCheckSQLForRangeExprPartition(pi *model.PartitionInfo, index int, schemaName, tableName model.CIStr) string { if index == 0 { - return fmt.Sprintf("select 1 from `%s`.`%s` where %s >= %d limit 1", schemaName.L, tableName.L, pi.Expr, rangeRrun.LessThan[index]) - } else if index == len(pi.Definitions)-1 && rangeRrun.MaxValue { - return fmt.Sprintf("select 1 from `%s`.`%s` where %s < %d limit 1", schemaName.L, tableName.L, pi.Expr, rangeRrun.LessThan[index-1]) + return fmt.Sprintf("select 1 from `%s`.`%s` where %s >= %s limit 1", schemaName.L, tableName.L, pi.Expr, pi.Definitions[index].LessThan[0]) + } else if index == len(pi.Definitions)-1 && strings.EqualFold(pi.Definitions[index].LessThan[0], "MAXVALUE") { + return fmt.Sprintf("select 1 from `%s`.`%s` where %s < %s limit 1", schemaName.L, tableName.L, pi.Expr, pi.Definitions[index-1].LessThan[0]) } else { - return fmt.Sprintf("select 1 from `%s`.`%s` where %s < %d or %s >= %d limit 1", schemaName.L, tableName.L, pi.Expr, rangeRrun.LessThan[index-1], pi.Expr, rangeRrun.LessThan[index]) + return fmt.Sprintf("select 1 from `%s`.`%s` where %s < %s or %s >= %s limit 1", schemaName.L, tableName.L, pi.Expr, pi.Definitions[index-1].LessThan[0], pi.Expr, pi.Definitions[index].LessThan[0]) } } -func buildCheckSQLForRangeColumnsPartition(pi *model.PartitionInfo, rangeRrun *tables.ForRangePruning, index int, schemaName, tableName model.CIStr) string { +func buildCheckSQLForRangeColumnsPartition(pi *model.PartitionInfo, index int, schemaName, tableName model.CIStr) string { colName := pi.Columns[0].L if index == 0 { - return fmt.Sprintf("select 1 from `%s`.`%s` where `%s` >= %d limit 1", schemaName.L, tableName.L, colName, rangeRrun.LessThan[index]) - } else if index == len(pi.Definitions)-1 && rangeRrun.MaxValue { - return fmt.Sprintf("select 1 from `%s`.`%s` where `%s` < %d limit 1", schemaName.L, tableName.L, colName, rangeRrun.LessThan[index-1]) + return fmt.Sprintf("select 1 from `%s`.`%s` where `%s` >= %s limit 1", schemaName.L, tableName.L, colName, pi.Definitions[index].LessThan[0]) + } else if index == len(pi.Definitions)-1 && strings.EqualFold(pi.Definitions[index].LessThan[0], "MAXVALUE") { + return fmt.Sprintf("select 1 from `%s`.`%s` where `%s` < %s limit 1", schemaName.L, tableName.L, colName, pi.Definitions[index-1].LessThan[0]) } else { - return fmt.Sprintf("select 1 from `%s`.`%s` where `%s` < %d or `%s` >= %d limit 1", schemaName.L, tableName.L, colName, rangeRrun.LessThan[index-1], colName, rangeRrun.LessThan[index]) + return fmt.Sprintf("select 1 from `%s`.`%s` where `%s` < %s or `%s` >= %s limit 1", schemaName.L, tableName.L, colName, pi.Definitions[index-1].LessThan[0], colName, pi.Definitions[index].LessThan[0]) } } diff --git a/table/tables/partition.go b/table/tables/partition.go index 453d0fd366836..ac3108aa58a1d 100644 --- a/table/tables/partition.go +++ b/table/tables/partition.go @@ -162,8 +162,8 @@ type ForRangePruning struct { Unsigned bool } -// DataForRangePruning extracts the less than parts from 'partition p0 less than xx ... partitoin p1 less than ...' -func DataForRangePruning(pi *model.PartitionInfo) (*ForRangePruning, error) { +// dataForRangePruning extracts the less than parts from 'partition p0 less than xx ... partitoin p1 less than ...' +func dataForRangePruning(pi *model.PartitionInfo) (*ForRangePruning, error) { var maxValue bool var unsigned bool lessThan := make([]int64, len(pi.Definitions)) @@ -240,7 +240,7 @@ func generateRangePartitionExpr(ctx sessionctx.Context, pi *model.PartitionInfo, switch len(pi.Columns) { case 0: - tmp, err := DataForRangePruning(pi) + tmp, err := dataForRangePruning(pi) if err != nil { return nil, errors.Trace(err) } From 134c55d9e5b86b35eea81a73d8fc7c0df6646ad7 Mon Sep 17 00:00:00 2001 From: zhaox1n Date: Fri, 29 May 2020 14:56:51 +0800 Subject: [PATCH 39/57] fix comment --- ddl/ddl_worker_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ddl/ddl_worker_test.go b/ddl/ddl_worker_test.go index 671349a8c72ab..116d1689e39c1 100644 --- a/ddl/ddl_worker_test.go +++ b/ddl/ddl_worker_test.go @@ -1005,7 +1005,7 @@ func (s *testDDLSuite) TestCancelJob(c *C) { c.Assert(changedNtTable.Meta().ID == nt.ID, IsTrue) c.Assert(changedPtTable.Meta().Partition.Definitions[0].ID == pt.Partition.Definitions[0].ID, IsTrue) - // canel exchange partition successfully + // cancel exchange partition successfully updateTest(&tests[44]) doDDLJobSuccess(ctx, d, c, dbInfo.ID, nt.ID, test.act, exchangeTablePartition) c.Check(checkErr, IsNil) From d7248239f3dfd13147febd070c856eac2f3a6527 Mon Sep 17 00:00:00 2001 From: zhaox1n Date: Sat, 30 May 2020 01:06:06 +0800 Subject: [PATCH 40/57] fix comment --- ddl/failtest/fail_db_test.go | 131 ++++++++++------------------------- 1 file changed, 36 insertions(+), 95 deletions(-) diff --git a/ddl/failtest/fail_db_test.go b/ddl/failtest/fail_db_test.go index 9bcf165a63bab..6ae1f72aaa25f 100644 --- a/ddl/failtest/fail_db_test.go +++ b/ddl/failtest/fail_db_test.go @@ -94,131 +94,72 @@ func (s *testFailDBSuite) TestHalfwayCancelOperations(c *C) { defer func() { c.Assert(failpoint.Disable("github.com/pingcap/tidb/ddl/truncateTableErr"), IsNil) }() + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("create database cancel_job_db") + tk.MustExec("use cancel_job_db") + // test for truncating table - _, err := s.se.Execute(context.Background(), "create database cancel_job_db") - c.Assert(err, IsNil) - _, err = s.se.Execute(context.Background(), "use cancel_job_db") - c.Assert(err, IsNil) - _, err = s.se.Execute(context.Background(), "create table t(a int)") - c.Assert(err, IsNil) - _, err = s.se.Execute(context.Background(), "insert into t values(1)") - c.Assert(err, IsNil) - _, err = s.se.Execute(context.Background(), "truncate table t") + tk.MustExec("create table t(a int)") + tk.MustExec("insert into t values(1)") + _, err := tk.Exec("truncate table t") c.Assert(err, NotNil) + // Make sure that the table's data has not been deleted. - rs, err := s.se.Execute(context.Background(), "select count(*) from t") - c.Assert(err, IsNil) - req := rs[0].NewChunk() - err = rs[0].Next(context.Background(), req) - c.Assert(err, IsNil) - c.Assert(req.NumRows() == 0, IsFalse) - row := req.GetRow(0) - c.Assert(row.Len(), Equals, 1) - c.Assert(row.GetInt64(0), DeepEquals, int64(1)) - c.Assert(rs[0].Close(), IsNil) - // Execute ddl statement reload schema. - _, err = s.se.Execute(context.Background(), "alter table t comment 'test1'") - c.Assert(err, IsNil) + tk.MustQuery("select * from t").Check(testkit.Rows("1")) + // Execute ddl statement reload schema + tk.MustExec("alter table t comment 'test1'") err = s.dom.DDL().GetHook().OnChanged(nil) c.Assert(err, IsNil) - s.se, err = session.CreateSession4Test(s.store) - c.Assert(err, IsNil) - _, err = s.se.Execute(context.Background(), "use cancel_job_db") - c.Assert(err, IsNil) - // Test schema is correct. - _, err = s.se.Execute(context.Background(), "select * from t") - c.Assert(err, IsNil) + tk = testkit.NewTestKit(c, s.store) + tk.MustExec("use cancel_job_db") + // Test schema is correct. + tk.MustExec("select * from t") // test for renaming table c.Assert(failpoint.Enable("github.com/pingcap/tidb/ddl/renameTableErr", `return(true)`), IsNil) defer func() { c.Assert(failpoint.Disable("github.com/pingcap/tidb/ddl/renameTableErr"), IsNil) }() - - _, err = s.se.Execute(context.Background(), "create table tx(a int)") - c.Assert(err, IsNil) - _, err = s.se.Execute(context.Background(), "insert into tx values(1)") - c.Assert(err, IsNil) - _, err = s.se.Execute(context.Background(), "rename table tx to ty") + tk.MustExec("create table tx(a int)") + tk.MustExec("insert into tx values(1)") + _, err = tk.Exec("rename table tx to ty") c.Assert(err, NotNil) // Make sure that the table's data has not been deleted. - rs, err = s.se.Execute(context.Background(), "select count(*) from tx") - c.Assert(err, IsNil) - req = rs[0].NewChunk() - err = rs[0].Next(context.Background(), req) - c.Assert(err, IsNil) - c.Assert(req.NumRows() == 0, IsFalse) - row = req.GetRow(0) - c.Assert(row.Len(), Equals, 1) - c.Assert(row.GetInt64(0), DeepEquals, int64(1)) - c.Assert(rs[0].Close(), IsNil) + tk.MustQuery("select * from tx").Check(testkit.Rows("1")) // Execute ddl statement reload schema. - _, err = s.se.Execute(context.Background(), "alter table tx comment 'tx'") - c.Assert(err, IsNil) + tk.MustExec("alter table tx comment 'tx'") err = s.dom.DDL().GetHook().OnChanged(nil) c.Assert(err, IsNil) - s.se, err = session.CreateSession4Test(s.store) - c.Assert(err, IsNil) - _, err = s.se.Execute(context.Background(), "use cancel_job_db") - c.Assert(err, IsNil) - // Test schema is correct. - _, err = s.se.Execute(context.Background(), "select * from tx") - c.Assert(err, IsNil) + tk = testkit.NewTestKit(c, s.store) + tk.MustExec("use cancel_job_db") + tk.MustExec("select * from tx") // test for exchanging partition c.Assert(failpoint.Enable("github.com/pingcap/tidb/ddl/exchangePartitionErr", `return(true)`), IsNil) defer func() { c.Assert(failpoint.Disable("github.com/pingcap/tidb/ddl/exchangePartitionErr"), IsNil) }() - _, err = s.se.Execute(context.Background(), "create table pt(a int) partition by hash (a) partitions 2") - c.Assert(err, IsNil) - _, err = s.se.Execute(context.Background(), "insert into pt values(1), (3), (5)") - c.Assert(err, IsNil) - _, err = s.se.Execute(context.Background(), "create table nt(a int)") - c.Assert(err, IsNil) - _, err = s.se.Execute(context.Background(), "insert into nt values(7)") - c.Assert(err, IsNil) - _, err = s.se.Execute(context.Background(), "alter table pt exchange partition p1 with table nt") + tk.MustExec("create table pt(a int) partition by hash (a) partitions 2") + tk.MustExec("insert into pt values(1), (3), (5)") + tk.MustExec("create table nt(a int)") + tk.MustExec("insert into nt values(7)") + _, err = tk.Exec("alter table pt exchange partition p1 with table nt") c.Assert(err, NotNil) - rs, err = s.se.Execute(context.Background(), "select count(*) from pt") - c.Assert(err, IsNil) - req = rs[0].NewChunk() - err = rs[0].Next(context.Background(), req) - c.Assert(err, IsNil) - c.Assert(req.NumRows() == 0, IsFalse) - row = req.GetRow(0) - c.Assert(row.Len(), Equals, 1) - c.Assert(row.GetInt64(0), DeepEquals, int64(3)) - c.Assert(rs[0].Close(), IsNil) - - rs, err = s.se.Execute(context.Background(), "select count(*) from nt") - c.Assert(err, IsNil) - req = rs[0].NewChunk() - err = rs[0].Next(context.Background(), req) - c.Assert(err, IsNil) - c.Assert(req.NumRows() == 0, IsFalse) - row = req.GetRow(0) - c.Assert(row.Len(), Equals, 1) - c.Assert(row.GetInt64(0), DeepEquals, int64(1)) - c.Assert(rs[0].Close(), IsNil) - + tk.MustQuery("select * from pt").Check(testkit.Rows("1", "3", "5")) + tk.MustQuery("select * from nt").Check(testkit.Rows("7")) // Execute ddl statement reload schema. - _, err = s.se.Execute(context.Background(), "alter table pt comment 'pt'") - c.Assert(err, IsNil) + tk.MustExec("alter table pt comment 'pt'") err = s.dom.DDL().GetHook().OnChanged(nil) c.Assert(err, IsNil) - s.se, err = session.CreateSession4Test(s.store) - c.Assert(err, IsNil) - _, err = s.se.Execute(context.Background(), "use cancel_job_db") - c.Assert(err, IsNil) + + tk = testkit.NewTestKit(c, s.store) + tk.MustExec("use cancel_job_db") // Test schema is correct. - _, err = s.se.Execute(context.Background(), "select * from pt") - c.Assert(err, IsNil) + tk.MustExec("select * from pt") // clean up - _, err = s.se.Execute(context.Background(), "drop database cancel_job_db") - c.Assert(err, IsNil) + tk.MustExec("drop database cancel_job_db") } // TestInitializeOffsetAndState tests the case that the column's offset and state don't be initialized in the file of ddl_api.go when From 894b7e581d765cbca710bd8c2282b02c662d89fd Mon Sep 17 00:00:00 2001 From: zhaox1n Date: Sat, 30 May 2020 11:17:21 +0800 Subject: [PATCH 41/57] fix bug --- ddl/partition.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ddl/partition.go b/ddl/partition.go index 6bccc44f31d33..d66c37deed75e 100644 --- a/ddl/partition.go +++ b/ddl/partition.go @@ -840,7 +840,7 @@ func (w *worker) onExchangeTablePartition(d *ddlCtx, t *meta.Meta, job *model.Jo return ver, errors.Trace(err) } - if ntRandID != 0 && ptRandID != 0 { + if ntRandID != 0 || ptRandID != 0 { if ntRandID > ptRandID { _, err = t.GenAutoRandomID(ptSchemaID, pt.ID, ntRandID-ptRandID) if err != nil { From 40729a169177754db62460f16433c53ac03b51c5 Mon Sep 17 00:00:00 2001 From: zhaox1n Date: Mon, 1 Jun 2020 21:47:06 +0800 Subject: [PATCH 42/57] fix comment --- ddl/ddl_worker_test.go | 1 - ddl/error.go | 13 +++++++------ ddl/partition.go | 3 ++- 3 files changed, 9 insertions(+), 8 deletions(-) diff --git a/ddl/ddl_worker_test.go b/ddl/ddl_worker_test.go index 116d1689e39c1..bc70da6eb6ddf 100644 --- a/ddl/ddl_worker_test.go +++ b/ddl/ddl_worker_test.go @@ -1013,7 +1013,6 @@ func (s *testDDLSuite) TestCancelJob(c *C) { changedPtTable = testGetTable(c, d, dbInfo.ID, pt.ID) c.Assert(changedNtTable.Meta().ID == nt.ID, IsFalse) c.Assert(changedPtTable.Meta().Partition.Definitions[0].ID == nt.ID, IsTrue) - } func (s *testDDLSuite) TestIgnorableSpec(c *C) { diff --git a/ddl/error.go b/ddl/error.go index d97bad6c96658..a9d0d2ced1269 100644 --- a/ddl/error.go +++ b/ddl/error.go @@ -210,14 +210,15 @@ var ( ErrAddColumnWithSequenceAsDefault = terror.ClassDDL.New(mysql.ErrAddColumnWithSequenceAsDefault, mysql.MySQLErrName[mysql.ErrAddColumnWithSequenceAsDefault]) // ErrUnsupportedExpressionIndex is returned when create an expression index without allow-expression-index. ErrUnsupportedExpressionIndex = terror.ClassDDL.New(mysql.ErrUnsupportedDDLOperation, fmt.Sprintf(mysql.MySQLErrName[mysql.ErrUnsupportedDDLOperation], "creating expression index without allow-expression-index in config")) - // ErrPartitionExchangePartTable is returned when exchange table partition with another table is partitioned + // ErrPartitionExchangePartTable is returned when exchange table partition with another table is partitioned. ErrPartitionExchangePartTable = terror.ClassDDL.New(mysql.ErrPartitionExchangePartTable, mysql.MySQLErrName[mysql.ErrPartitionExchangePartTable]) - // ErrTablesDifferentMetadata is returned when exchanges tables is not compatible + // ErrTablesDifferentMetadata is returned when exchanges tables is not compatible. ErrTablesDifferentMetadata = terror.ClassDDL.New(mysql.ErrTablesDifferentMetadata, mysql.MySQLErrName[mysql.ErrTablesDifferentMetadata]) - // ErrRowDoesNotMatchPartition is returned when the row record of exchange table does not match the partition rule + // ErrRowDoesNotMatchPartition is returned when the row record of exchange table does not match the partition rule. ErrRowDoesNotMatchPartition = terror.ClassDDL.New(mysql.ErrRowDoesNotMatchPartition, mysql.MySQLErrName[mysql.ErrRowDoesNotMatchPartition]) - // ErrPartitionExchangeForeignKey is returned when exchanged non-partition table has foreign keys + // ErrPartitionExchangeForeignKey is returned when exchanged non-partition table has foreign keys. ErrPartitionExchangeForeignKey = terror.ClassDDL.New(mysql.ErrPartitionExchangeForeignKey, mysql.MySQLErrName[mysql.ErrPartitionExchangeForeignKey]) - // ErrCheckNoSuchTable is returned when exchaned non-partition table is view or sequence - ErrCheckNoSuchTable = terror.ClassDDL.New(mysql.ErrCheckNoSuchTable, mysql.MySQLErrName[mysql.ErrCheckNoSuchTable]) + // ErrCheckNoSuchTable is returned when exchaned non-partition table is view or sequence. + ErrCheckNoSuchTable = terror.ClassDDL.New(mysql.ErrCheckNoSuchTable, mysql.MySQLErrName[mysql.ErrCheckNoSuchTable]) + errUnsupportedPartitionType = terror.ClassDDL.New(mysql.ErrUnsupportedDDLOperation, fmt.Sprintf(mysql.MySQLErrName[mysql.ErrUnsupportedDDLOperation], "partition type of table: %s")) ) diff --git a/ddl/partition.go b/ddl/partition.go index d66c37deed75e..7ab1a91313103 100644 --- a/ddl/partition.go +++ b/ddl/partition.go @@ -725,6 +725,7 @@ func (w *worker) onExchangeTablePartition(d *ddlCtx, t *meta.Meta, job *model.Jo ntDbInfo, err := t.GetDatabase(job.SchemaID) if err != nil { + job.State = model.JobStateCancelled return ver, errors.Trace(err) } @@ -895,7 +896,7 @@ func checkExchangePartitionRecordValidation(w *worker, pt *model.TableInfo, part sql = buildCheckSQLForRangeColumnsPartition(pi, index, schemaName, tableName) } default: - return errors.Trace(errors.Errorf("unsupported partition type for checkExchangePartitionRecordValidation")) + return errUnsupportedPartitionType.GenWithStackByArgs(pt.Name.O) } var ctx sessionctx.Context From 6ea633d3adf1ad2d4d6e56e0ebf7df92f5cb4071 Mon Sep 17 00:00:00 2001 From: zhaox1n Date: Tue, 2 Jun 2020 00:01:07 +0800 Subject: [PATCH 43/57] fix comment --- ddl/db_partition_test.go | 27 +++++++++++++++++++++++++++ ddl/error.go | 2 +- ddl/partition.go | 6 +++--- 3 files changed, 31 insertions(+), 4 deletions(-) diff --git a/ddl/db_partition_test.go b/ddl/db_partition_test.go index 2f9af16de4f7e..973d39a624ebc 100644 --- a/ddl/db_partition_test.go +++ b/ddl/db_partition_test.go @@ -896,6 +896,33 @@ func (s *testIntegrationSuite4) TestAlterTableExchangePartition(c *C) { tk.MustQuery("select * from e6 partition(p0)").Check(testkit.Rows("2")) tk.MustQuery("select * from e7").Check(testkit.Rows("1")) tk.MustGetErrCode("alter table e6 exchange partition p1 with table e7", tmysql.ErrRowDoesNotMatchPartition) + + // test exchange partition from different databases + tk.MustExec("create table e8 (a int) partition by hash(a) partitions 2;") + tk.MustExec("create database if not exists exchange_partition") + tk.MustExec("insert into e8 values (1), (3), (5)") + tk.MustExec("use exchange_partition;") + tk.MustExec("create table e9 (a int);") + tk.MustExec("insert into e9 values (7), (9)") + tk.MustExec("alter table test.e8 exchange partition p1 with table e9") + + tk.MustExec("insert into e9 values (11)") + tk.MustQuery("select * from e9").Check(testkit.Rows("1", "3", "5", "11")) + tk.MustExec("insert into test.e8 values (11)") + tk.MustQuery("select * from test.e8").Check(testkit.Rows("7", "9", "11")) + + tk.MustExec("use test") + tk.MustExec("create table e10 (a int) partition by hash(a) partitions 2") + tk.MustExec("insert into e10 values (0), (2), (4)") + tk.MustExec("create table e11 (a int)") + tk.MustExec("insert into e11 values (1), (3)") + tk.MustExec("alter table e10 exchange partition p1 with table e11") + tk.MustExec("insert into e11 values (5)") + tk.MustQuery("select * from e11").Check(testkit.Rows("5")) + tk.MustExec("insert into e10 values (5), (6)") + tk.MustQuery("select * from e10 partition(p0)").Check(testkit.Rows("0", "2", "4", "6")) + tk.MustQuery("select * from e10 partition(p1)").Check(testkit.Rows("1", "3", "5")) + } func (s *testIntegrationSuite4) TestExchangePartitionTableCompatiable(c *C) { diff --git a/ddl/error.go b/ddl/error.go index a9d0d2ced1269..30c7ee025463f 100644 --- a/ddl/error.go +++ b/ddl/error.go @@ -220,5 +220,5 @@ var ( ErrPartitionExchangeForeignKey = terror.ClassDDL.New(mysql.ErrPartitionExchangeForeignKey, mysql.MySQLErrName[mysql.ErrPartitionExchangeForeignKey]) // ErrCheckNoSuchTable is returned when exchaned non-partition table is view or sequence. ErrCheckNoSuchTable = terror.ClassDDL.New(mysql.ErrCheckNoSuchTable, mysql.MySQLErrName[mysql.ErrCheckNoSuchTable]) - errUnsupportedPartitionType = terror.ClassDDL.New(mysql.ErrUnsupportedDDLOperation, fmt.Sprintf(mysql.MySQLErrName[mysql.ErrUnsupportedDDLOperation], "partition type of table: %s")) + errUnsupportedPartitionType = terror.ClassDDL.New(mysql.ErrUnsupportedDDLOperation, fmt.Sprintf(mysql.MySQLErrName[mysql.ErrUnsupportedDDLOperation], "partition type of table %s when exchanging partition")) ) diff --git a/ddl/partition.go b/ddl/partition.go index 7ab1a91313103..c70727c4fcffc 100644 --- a/ddl/partition.go +++ b/ddl/partition.go @@ -886,7 +886,7 @@ func checkExchangePartitionRecordValidation(w *worker, pt *model.TableInfo, part return errors.Trace(err) } // Table has only one partition and has the maximum value - if len(pi.Definitions) == 1 && strings.EqualFold(pi.Definitions[index].LessThan[0], "MAXVALUE") { + if len(pi.Definitions) == 1 && strings.EqualFold(pi.Definitions[index].LessThan[0], partitionMaxValue) { return nil } // For range expression and range columns @@ -920,7 +920,7 @@ func checkExchangePartitionRecordValidation(w *worker, pt *model.TableInfo, part func buildCheckSQLForRangeExprPartition(pi *model.PartitionInfo, index int, schemaName, tableName model.CIStr) string { if index == 0 { return fmt.Sprintf("select 1 from `%s`.`%s` where %s >= %s limit 1", schemaName.L, tableName.L, pi.Expr, pi.Definitions[index].LessThan[0]) - } else if index == len(pi.Definitions)-1 && strings.EqualFold(pi.Definitions[index].LessThan[0], "MAXVALUE") { + } else if index == len(pi.Definitions)-1 && strings.EqualFold(pi.Definitions[index].LessThan[0], partitionMaxValue) { return fmt.Sprintf("select 1 from `%s`.`%s` where %s < %s limit 1", schemaName.L, tableName.L, pi.Expr, pi.Definitions[index-1].LessThan[0]) } else { return fmt.Sprintf("select 1 from `%s`.`%s` where %s < %s or %s >= %s limit 1", schemaName.L, tableName.L, pi.Expr, pi.Definitions[index-1].LessThan[0], pi.Expr, pi.Definitions[index].LessThan[0]) @@ -931,7 +931,7 @@ func buildCheckSQLForRangeColumnsPartition(pi *model.PartitionInfo, index int, s colName := pi.Columns[0].L if index == 0 { return fmt.Sprintf("select 1 from `%s`.`%s` where `%s` >= %s limit 1", schemaName.L, tableName.L, colName, pi.Definitions[index].LessThan[0]) - } else if index == len(pi.Definitions)-1 && strings.EqualFold(pi.Definitions[index].LessThan[0], "MAXVALUE") { + } else if index == len(pi.Definitions)-1 && strings.EqualFold(pi.Definitions[index].LessThan[0], partitionMaxValue) { return fmt.Sprintf("select 1 from `%s`.`%s` where `%s` < %s limit 1", schemaName.L, tableName.L, colName, pi.Definitions[index-1].LessThan[0]) } else { return fmt.Sprintf("select 1 from `%s`.`%s` where `%s` < %s or `%s` >= %s limit 1", schemaName.L, tableName.L, colName, pi.Definitions[index-1].LessThan[0], colName, pi.Definitions[index].LessThan[0]) From d9b98c0de1cbbc42cfdb42ce120c364cf126b168 Mon Sep 17 00:00:00 2001 From: zhaox1n Date: Tue, 2 Jun 2020 03:05:54 +0800 Subject: [PATCH 44/57] fix comment --- ddl/ddl_api.go | 4 ++-- ddl/ddl_worker.go | 6 ++++++ ddl/ddl_worker_test.go | 3 ++- ddl/partition.go | 22 +++++++++++----------- go.mod | 2 ++ go.sum | 2 ++ infoschema/builder.go | 22 ++++++++++++++++++++-- 7 files changed, 45 insertions(+), 16 deletions(-) diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index c4149819533a5..e62d33ef0846c 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -2917,7 +2917,7 @@ func (d *ddl) ExchangeTablePartition(ctx sessionctx.Context, ident ast.Ident, sp // NOTE: if pt is subPartitioned, it should be checked - _, err = tables.FindPartitionByName(ptMeta, partName) + defID, err := tables.FindPartitionByName(ptMeta, partName) if err != nil { return errors.Trace(err) } @@ -2933,7 +2933,7 @@ func (d *ddl) ExchangeTablePartition(ctx sessionctx.Context, ident ast.Ident, sp SchemaName: ntSchema.Name.L, Type: model.ActionExchangeTablePartition, BinlogInfo: &model.HistoryInfo{}, - Args: []interface{}{ptSchema.ID, ptMeta.ID, partName, spec.WithValidation}, + Args: []interface{}{defID, ptSchema.ID, ptMeta.ID, partName, spec.WithValidation}, } err = d.doDDLJob(ctx, job) diff --git a/ddl/ddl_worker.go b/ddl/ddl_worker.go index dfc569efb4197..331434590371b 100644 --- a/ddl/ddl_worker.go +++ b/ddl/ddl_worker.go @@ -835,6 +835,12 @@ func updateSchemaVersion(t *meta.Meta, job *model.Job) (int64, error) { return 0, errors.Trace(err) } diff.TableID = job.TableID + case model.ActionExchangeTablePartition: + err = job.DecodeArgs(&diff.TableID, &diff.PtSchemaID, &diff.PtTableID) + if err != nil { + return 0, errors.Trace(err) + } + diff.OldTableID = job.TableID default: diff.TableID = job.TableID } diff --git a/ddl/ddl_worker_test.go b/ddl/ddl_worker_test.go index bc70da6eb6ddf..cb2ddef475437 100644 --- a/ddl/ddl_worker_test.go +++ b/ddl/ddl_worker_test.go @@ -997,7 +997,8 @@ func (s *testDDLSuite) TestCancelJob(c *C) { testCreateTable(c, ctx, d, dbInfo, nt) updateTest(&tests[43]) - exchangeTablePartition := []interface{}{dbInfo.ID, pt.ID, "p0", true} + defID := pt.Partition.Definitions[0].ID + exchangeTablePartition := []interface{}{defID, dbInfo.ID, pt.ID, "p0", true} doDDLJobErrWithSchemaState(ctx, d, c, dbInfo.ID, nt.ID, test.act, exchangeTablePartition, &test.cancelState) c.Check(checkErr, IsNil) changedNtTable := testGetTable(c, d, dbInfo.ID, nt.ID) diff --git a/ddl/partition.go b/ddl/partition.go index c70727c4fcffc..271bb3a7f6539 100644 --- a/ddl/partition.go +++ b/ddl/partition.go @@ -712,13 +712,15 @@ func onTruncateTablePartition(d *ddlCtx, t *meta.Meta, job *model.Job) (int64, e // onExchangeTablePartition exchange partition data func (w *worker) onExchangeTablePartition(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ error) { var ( + //defID only for updateVersion + defID int64 ptSchemaID int64 ptID int64 partName string withValidation bool ) - if err := job.DecodeArgs(&ptSchemaID, &ptID, &partName, &withValidation); err != nil { + if err := job.DecodeArgs(&defID, &ptSchemaID, &ptID, &partName, &withValidation); err != nil { job.State = model.JobStateCancelled return ver, errors.Trace(err) } @@ -754,8 +756,13 @@ func (w *worker) onExchangeTablePartition(d *ddlCtx, t *meta.Meta, job *model.Jo return ver, errors.Trace(err) } + index, _, err := getPartitionDef(pt, partName) + if err != nil { + return ver, errors.Trace(err) + } + if withValidation { - err = checkExchangePartitionRecordValidation(w, pt, partName, ntDbInfo.Name, nt.Name) + err = checkExchangePartitionRecordValidation(w, pt, index, ntDbInfo.Name, nt.Name) if err != nil { job.State = model.JobStateCancelled return ver, errors.Trace(err) @@ -866,14 +873,10 @@ func (w *worker) onExchangeTablePartition(d *ddlCtx, t *meta.Meta, job *model.Jo return ver, nil } -func checkExchangePartitionRecordValidation(w *worker, pt *model.TableInfo, partName string, schemaName, tableName model.CIStr) error { +func checkExchangePartitionRecordValidation(w *worker, pt *model.TableInfo, index int, schemaName, tableName model.CIStr) error { var sql string pi := pt.Partition - index, _, err := getPartitionDef(pt, partName) - if err != nil { - return errors.Trace(err) - } switch pi.Type { case model.PartitionTypeHash: @@ -882,9 +885,6 @@ func checkExchangePartitionRecordValidation(w *worker, pt *model.TableInfo, part } sql = fmt.Sprintf("select 1 from `%s`.`%s` where mod(%s, %d) != %d limit 1", schemaName.L, tableName.L, pi.Expr, pi.Num, index) case model.PartitionTypeRange: - if err != nil { - return errors.Trace(err) - } // Table has only one partition and has the maximum value if len(pi.Definitions) == 1 && strings.EqualFold(pi.Definitions[index].LessThan[0], partitionMaxValue) { return nil @@ -900,7 +900,7 @@ func checkExchangePartitionRecordValidation(w *worker, pt *model.TableInfo, part } var ctx sessionctx.Context - ctx, err = w.sessPool.get() + ctx, err := w.sessPool.get() if err != nil { return errors.Trace(err) } diff --git a/go.mod b/go.mod index 4dac0fa4186b1..9f79625ea5eaf 100644 --- a/go.mod +++ b/go.mod @@ -63,3 +63,5 @@ require ( ) go 1.13 + +replace github.com/pingcap/parser => github.com/zhaox1n/parser v0.0.0-20200601184100-acd1a1774f3b diff --git a/go.sum b/go.sum index 2a6ae406f7bdd..26cda74def8d8 100644 --- a/go.sum +++ b/go.sum @@ -601,6 +601,8 @@ github.com/yookoala/realpath v1.0.0/go.mod h1:gJJMA9wuX7AcqLy1+ffPatSCySA1FQ2S8Y github.com/yuin/goldmark v1.1.25/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/zhangjinpeng1987/raft v0.0.0-20190624145930-deeb32d6553d h1:rGkexfPDxNuTCObUwTbsRUlti+evR/Ksb4dKy6esXW0= github.com/zhangjinpeng1987/raft v0.0.0-20190624145930-deeb32d6553d/go.mod h1:1KDQ09J8MRHEtHze4at7BJZDW/doUAgkJ8w9KjEUhSo= +github.com/zhaox1n/parser v0.0.0-20200601184100-acd1a1774f3b h1:ztkWO3u6NFlSkV3vr3vOdeOlh8owOpHx9t5qZvFO5IM= +github.com/zhaox1n/parser v0.0.0-20200601184100-acd1a1774f3b/go.mod h1:vQdbJqobJAgFyiRNNtXahpMoGWwPEuWciVEK5A20NS0= go.etcd.io/bbolt v1.3.2/go.mod h1:IbVyRI1SCnLcuJnV2u8VeU0CEYM7e686BmAb1XKL+uU= go.etcd.io/bbolt v1.3.3 h1:MUGmc65QhB3pIlaQ5bB4LwqSj6GIonVJXpZiaKNyaKk= go.etcd.io/bbolt v1.3.3/go.mod h1:IbVyRI1SCnLcuJnV2u8VeU0CEYM7e686BmAb1XKL+uU= diff --git a/infoschema/builder.go b/infoschema/builder.go index c58c8499c907f..498a913921a1a 100644 --- a/infoschema/builder.go +++ b/infoschema/builder.go @@ -47,6 +47,8 @@ func (b *Builder) ApplyDiff(m *meta.Meta, diff *model.SchemaDiff) ([]int64, erro return tblIDs, nil } else if diff.Type == model.ActionModifySchemaCharsetAndCollate { return nil, b.applyModifySchemaCharsetAndCollate(m, diff) + } else if diff.Type == model.ActionExchangeTablePartition { + return nil, b.applyExchangePartition(m, diff) } roDBInfo, ok := b.is.SchemaByID(diff.SchemaID) if !ok { @@ -60,7 +62,7 @@ func (b *Builder) ApplyDiff(m *meta.Meta, diff *model.SchemaDiff) ([]int64, erro newTableID = diff.TableID case model.ActionDropTable, model.ActionDropView, model.ActionDropSequence: oldTableID = diff.TableID - case model.ActionTruncateTable, model.ActionCreateView: + case model.ActionTruncateTable, model.ActionCreateView, model.ActionExchangeTablePartition: oldTableID = diff.OldTableID newTableID = diff.TableID default: @@ -74,7 +76,8 @@ func (b *Builder) ApplyDiff(m *meta.Meta, diff *model.SchemaDiff) ([]int64, erro // We try to reuse the old allocator, so the cached auto ID can be reused. var allocs autoid.Allocators if tableIDIsValid(oldTableID) { - if oldTableID == newTableID && diff.Type != model.ActionRenameTable { + if oldTableID == newTableID && diff.Type != model.ActionRenameTable && + diff.Type != model.ActionExchangeTablePartition { oldAllocs, _ := b.is.AllocByID(oldTableID) allocs = filterAllocators(diff, oldAllocs) } @@ -155,6 +158,21 @@ func (b *Builder) copySortedTables(oldTableID, newTableID int64) { } } +func (b *Builder) applyExchangePartition(m *meta.Meta, diff *model.SchemaDiff) error { + ptDi, err := m.GetDatabase(diff.PtSchemaID) + if err != nil { + return errors.Trace(err) + } + tblIDs := make([]int64, 0, 2) + tblIDs = b.applyDropTable(ptDi, diff.PtTableID, tblIDs) + var allocs autoid.Allocators + tblIDs, err = b.applyCreateTable(m, ptDi, diff.PtTableID, allocs, diff.Type, tblIDs) + if err != nil { + return errors.Trace(err) + } + return nil +} + func (b *Builder) applyCreateSchema(m *meta.Meta, diff *model.SchemaDiff) error { di, err := m.GetDatabase(diff.SchemaID) if err != nil { From 780738069a01f733837eee43efe200915211d61a Mon Sep 17 00:00:00 2001 From: zhaox1n Date: Tue, 2 Jun 2020 03:27:47 +0800 Subject: [PATCH 45/57] fix comment --- go.mod | 2 +- go.sum | 2 ++ 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/go.mod b/go.mod index 8f75954709df6..1e58283d1fc5e 100644 --- a/go.mod +++ b/go.mod @@ -64,4 +64,4 @@ require ( go 1.13 -replace github.com/pingcap/parser => github.com/zhaox1n/parser v0.0.0-20200601184100-acd1a1774f3b +replace github.com/pingcap/parser => github.com/zhaox1n/parser v0.0.0-20200601191909-048d6ec9eef1 diff --git a/go.sum b/go.sum index d194756650ae6..f6d4a8c51edd5 100644 --- a/go.sum +++ b/go.sum @@ -603,6 +603,8 @@ github.com/zhangjinpeng1987/raft v0.0.0-20190624145930-deeb32d6553d h1:rGkexfPDx github.com/zhangjinpeng1987/raft v0.0.0-20190624145930-deeb32d6553d/go.mod h1:1KDQ09J8MRHEtHze4at7BJZDW/doUAgkJ8w9KjEUhSo= github.com/zhaox1n/parser v0.0.0-20200601184100-acd1a1774f3b h1:ztkWO3u6NFlSkV3vr3vOdeOlh8owOpHx9t5qZvFO5IM= github.com/zhaox1n/parser v0.0.0-20200601184100-acd1a1774f3b/go.mod h1:vQdbJqobJAgFyiRNNtXahpMoGWwPEuWciVEK5A20NS0= +github.com/zhaox1n/parser v0.0.0-20200601191909-048d6ec9eef1 h1:f4Qcit4fJibpvNmFSYMDS/O6KHOxQTnwFnfD1gvZ/O4= +github.com/zhaox1n/parser v0.0.0-20200601191909-048d6ec9eef1/go.mod h1:vQdbJqobJAgFyiRNNtXahpMoGWwPEuWciVEK5A20NS0= go.etcd.io/bbolt v1.3.2/go.mod h1:IbVyRI1SCnLcuJnV2u8VeU0CEYM7e686BmAb1XKL+uU= go.etcd.io/bbolt v1.3.3 h1:MUGmc65QhB3pIlaQ5bB4LwqSj6GIonVJXpZiaKNyaKk= go.etcd.io/bbolt v1.3.3/go.mod h1:IbVyRI1SCnLcuJnV2u8VeU0CEYM7e686BmAb1XKL+uU= From 4769d3bcc1147cde13358d727ddbe8529b17a1d9 Mon Sep 17 00:00:00 2001 From: zhaox1n Date: Tue, 2 Jun 2020 13:39:22 +0800 Subject: [PATCH 46/57] bug fix --- go.mod | 2 +- infoschema/builder.go | 22 +++++++++++++--------- 2 files changed, 14 insertions(+), 10 deletions(-) diff --git a/go.mod b/go.mod index 1e58283d1fc5e..8c65cbd4c50af 100644 --- a/go.mod +++ b/go.mod @@ -31,7 +31,7 @@ require ( github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989 github.com/pingcap/kvproto v0.0.0-20200518112156-d4aeb467de29 github.com/pingcap/log v0.0.0-20200511115504-543df19646ad - github.com/pingcap/parser v0.0.0-20200522094936-3b720a0512a6 + github.com/pingcap/parser v0.0.0-20200601053017-87d29a820b2d github.com/pingcap/pd/v4 v4.0.0-rc.2.0.20200520083007-2c251bd8f181 github.com/pingcap/sysutil v0.0.0-20200408114249-ed3bd6f7fdb1 github.com/pingcap/tidb-tools v4.0.0-rc.1.0.20200514040632-f76b3e428e19+incompatible diff --git a/infoschema/builder.go b/infoschema/builder.go index 498a913921a1a..fe4c1e3d75b2b 100644 --- a/infoschema/builder.go +++ b/infoschema/builder.go @@ -47,8 +47,6 @@ func (b *Builder) ApplyDiff(m *meta.Meta, diff *model.SchemaDiff) ([]int64, erro return tblIDs, nil } else if diff.Type == model.ActionModifySchemaCharsetAndCollate { return nil, b.applyModifySchemaCharsetAndCollate(m, diff) - } else if diff.Type == model.ActionExchangeTablePartition { - return nil, b.applyExchangePartition(m, diff) } roDBInfo, ok := b.is.SchemaByID(diff.SchemaID) if !ok { @@ -109,6 +107,13 @@ func (b *Builder) ApplyDiff(m *meta.Meta, diff *model.SchemaDiff) ([]int64, erro return nil, errors.Trace(err) } } + if diff.Type == model.ActionExchangeTablePartition { + var err error + tblIDs, err = b.applyExchangePartition(m, diff, tblIDs) + if err != nil { + return nil, errors.Trace(err) + } + } return tblIDs, nil } @@ -158,19 +163,18 @@ func (b *Builder) copySortedTables(oldTableID, newTableID int64) { } } -func (b *Builder) applyExchangePartition(m *meta.Meta, diff *model.SchemaDiff) error { +func (b *Builder) applyExchangePartition(m *meta.Meta, diff *model.SchemaDiff, affected []int64) ([]int64, error) { ptDi, err := m.GetDatabase(diff.PtSchemaID) if err != nil { - return errors.Trace(err) + return affected, errors.Trace(err) } - tblIDs := make([]int64, 0, 2) - tblIDs = b.applyDropTable(ptDi, diff.PtTableID, tblIDs) + affected = b.applyDropTable(ptDi, diff.PtTableID, affected) var allocs autoid.Allocators - tblIDs, err = b.applyCreateTable(m, ptDi, diff.PtTableID, allocs, diff.Type, tblIDs) + affected, err = b.applyCreateTable(m, ptDi, diff.PtTableID, allocs, diff.Type, affected) if err != nil { - return errors.Trace(err) + return affected, errors.Trace(err) } - return nil + return affected, nil } func (b *Builder) applyCreateSchema(m *meta.Meta, diff *model.SchemaDiff) error { From 325d59696ff1471becb11a92cd52aedc776fba65 Mon Sep 17 00:00:00 2001 From: zhaox1n Date: Tue, 2 Jun 2020 13:48:03 +0800 Subject: [PATCH 47/57] address comment --- ddl/partition.go | 4 ++-- infoschema/builder.go | 12 +++++++----- 2 files changed, 9 insertions(+), 7 deletions(-) diff --git a/ddl/partition.go b/ddl/partition.go index 271bb3a7f6539..e4e8cffe1dca7 100644 --- a/ddl/partition.go +++ b/ddl/partition.go @@ -712,7 +712,7 @@ func onTruncateTablePartition(d *ddlCtx, t *meta.Meta, job *model.Job) (int64, e // onExchangeTablePartition exchange partition data func (w *worker) onExchangeTablePartition(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ error) { var ( - //defID only for updateVersion + //defID only for updateSchemaVersion defID int64 ptSchemaID int64 ptID int64 @@ -725,7 +725,7 @@ func (w *worker) onExchangeTablePartition(d *ddlCtx, t *meta.Meta, job *model.Jo return ver, errors.Trace(err) } - ntDbInfo, err := t.GetDatabase(job.SchemaID) + ntDbInfo, err := checkSchemaExistAndCancelNotExistJob(t, job) if err != nil { job.State = model.JobStateCancelled return ver, errors.Trace(err) diff --git a/infoschema/builder.go b/infoschema/builder.go index fe4c1e3d75b2b..c75390eed360d 100644 --- a/infoschema/builder.go +++ b/infoschema/builder.go @@ -168,11 +168,13 @@ func (b *Builder) applyExchangePartition(m *meta.Meta, diff *model.SchemaDiff, a if err != nil { return affected, errors.Trace(err) } - affected = b.applyDropTable(ptDi, diff.PtTableID, affected) - var allocs autoid.Allocators - affected, err = b.applyCreateTable(m, ptDi, diff.PtTableID, allocs, diff.Type, affected) - if err != nil { - return affected, errors.Trace(err) + if tableIDIsValid(diff.PtTableID) { + affected = b.applyDropTable(ptDi, diff.PtTableID, affected) + var allocs autoid.Allocators + affected, err = b.applyCreateTable(m, ptDi, diff.PtTableID, allocs, diff.Type, affected) + if err != nil { + return affected, errors.Trace(err) + } } return affected, nil } From 85d71c0f23c3a222f0a33c48088fcfde0665a9f6 Mon Sep 17 00:00:00 2001 From: zhaox1n Date: Fri, 5 Jun 2020 00:19:56 +0800 Subject: [PATCH 48/57] address comment --- ddl/ddl_api.go | 19 ++++++++++++++++++- ddl/error.go | 6 ++++-- ddl/partition.go | 5 +++++ 3 files changed, 27 insertions(+), 3 deletions(-) diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 89db6a76f4a52..f03756143ec01 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -2834,7 +2834,12 @@ func checkFieldTypeCompatible(ft *types.FieldType, other *types.FieldType) bool func checkTableDefCompatible(source *model.TableInfo, target *model.TableInfo) error { // check auto_random - if source.AutoRandomBits != target.AutoRandomBits { + if source.AutoRandomBits != target.AutoRandomBits || + source.Charset != target.Charset || + source.Collate != target.Collate || + source.ShardRowIDBits != target.ShardRowIDBits || + source.MaxShardRowIDBits != target.MaxShardRowIDBits || + source.TiFlashReplica != target.TiFlashReplica { return errors.Trace(ErrTablesDifferentMetadata) } if len(source.Cols()) != len(target.Cols()) { @@ -2851,6 +2856,15 @@ func checkTableDefCompatible(source *model.TableInfo, target *model.TableInfo) e !checkFieldTypeCompatible(&sourceCol.FieldType, &targetCol.FieldType) { return errors.Trace(ErrTablesDifferentMetadata) } + if sourceCol.State != model.StatePublic { + return errors.Trace(ErrTablesDifferentMetadata) + } + if targetCol.State != model.StatePublic { + return errors.Trace(ErrTablesDifferentMetadata) + } + if sourceCol.ID != targetCol.ID { + return ErrPartitionExchangeDifferentOption.GenWithStackByArgs(fmt.Sprintf("column: %s", sourceCol.Name)) + } } if len(source.Indices) != len(target.Indices) { return errors.Trace(ErrTablesDifferentMetadata) @@ -2883,6 +2897,9 @@ func checkTableDefCompatible(source *model.TableInfo, target *model.TableInfo) e return errors.Trace(ErrTablesDifferentMetadata) } } + if sourceIdx.ID != compatIdx.ID { + return ErrPartitionExchangeDifferentOption.GenWithStackByArgs(fmt.Sprintf("index: %s", sourceIdx.Name)) + } } return nil diff --git a/ddl/error.go b/ddl/error.go index 30c7ee025463f..780debc1c1207 100644 --- a/ddl/error.go +++ b/ddl/error.go @@ -216,9 +216,11 @@ var ( ErrTablesDifferentMetadata = terror.ClassDDL.New(mysql.ErrTablesDifferentMetadata, mysql.MySQLErrName[mysql.ErrTablesDifferentMetadata]) // ErrRowDoesNotMatchPartition is returned when the row record of exchange table does not match the partition rule. ErrRowDoesNotMatchPartition = terror.ClassDDL.New(mysql.ErrRowDoesNotMatchPartition, mysql.MySQLErrName[mysql.ErrRowDoesNotMatchPartition]) - // ErrPartitionExchangeForeignKey is returned when exchanged non-partition table has foreign keys. + // ErrPartitionExchangeForeignKey is returned when exchanged normal table has foreign keys. ErrPartitionExchangeForeignKey = terror.ClassDDL.New(mysql.ErrPartitionExchangeForeignKey, mysql.MySQLErrName[mysql.ErrPartitionExchangeForeignKey]) - // ErrCheckNoSuchTable is returned when exchaned non-partition table is view or sequence. + // ErrCheckNoSuchTable is returned when exchaned normal table is view or sequence. ErrCheckNoSuchTable = terror.ClassDDL.New(mysql.ErrCheckNoSuchTable, mysql.MySQLErrName[mysql.ErrCheckNoSuchTable]) errUnsupportedPartitionType = terror.ClassDDL.New(mysql.ErrUnsupportedDDLOperation, fmt.Sprintf(mysql.MySQLErrName[mysql.ErrUnsupportedDDLOperation], "partition type of table %s when exchanging partition")) + // ErrPartitionExchangeDifferentOption is returned when attribute doesnot match between partition table and normal table. + ErrPartitionExchangeDifferentOption = terror.ClassDDL.New(mysql.ErrPartitionExchangeDifferentOption, mysql.MySQLErrName[mysql.ErrPartitionExchangeDifferentOption]) ) diff --git a/ddl/partition.go b/ddl/partition.go index e4e8cffe1dca7..778fa54ab6347 100644 --- a/ddl/partition.go +++ b/ddl/partition.go @@ -744,6 +744,11 @@ func (w *worker) onExchangeTablePartition(d *ddlCtx, t *meta.Meta, job *model.Jo return ver, errors.Trace(err) } + if pt.State != model.StatePublic { + job.State = model.JobStateCancelled + return ver, ErrInvalidDDLState.GenWithStack("table %s is not in public, but %s", pt.Name, pt.State) + } + err = checkExchangePartition(pt, nt) if err != nil { job.State = model.JobStateCancelled From 6c1d63a4065e5892b197e643c6d52a4f6aa75ced Mon Sep 17 00:00:00 2001 From: zhaox1n Date: Sat, 6 Jun 2020 09:50:56 +0800 Subject: [PATCH 49/57] address comment --- ddl/db_partition_test.go | 28 +++++++++++++++++++++++----- ddl/ddl_api.go | 4 +++- ddl/ddl_worker.go | 12 +++++++++++- go.mod | 2 +- infoschema/builder.go | 22 ++++++++++++---------- 5 files changed, 50 insertions(+), 18 deletions(-) diff --git a/ddl/db_partition_test.go b/ddl/db_partition_test.go index c9a11ecf8d470..165b58b2c6d3b 100644 --- a/ddl/db_partition_test.go +++ b/ddl/db_partition_test.go @@ -923,6 +923,18 @@ func (s *testIntegrationSuite4) TestAlterTableExchangePartition(c *C) { tk.MustQuery("select * from e10 partition(p0)").Check(testkit.Rows("0", "2", "4", "6")) tk.MustQuery("select * from e10 partition(p1)").Check(testkit.Rows("1", "3", "5")) + // test for column id + tk.MustExec("create table e12 (a int(1), b int, index (a)) partition by hash(a) partitions3") + tk.MustExec("create table e13 (a int(8), b int, index (a));") + tk.MustExec("alter table e13 drop column b") + tk.MustExec("alter table e13 add column b int") + tk.MustGetErrCode("alter table e12 exchange partition p0 with table e13", tmysql.ErrPartitionExchangeDifferentOption) + // test for index id + tk.MustExec("create table e14 (a int, b int, index(a));") + tk.MustExec("alter table e12 drop index a") + tk.MustExec("alter table e12 add index (a);") + tk.MustGetErrCode("alter table e12 exchange partition p0 with table e14", tmysql.ErrPartitionExchangeDifferentOption) + } func (s *testIntegrationSuite4) TestExchangePartitionTableCompatiable(c *C) { @@ -1013,13 +1025,13 @@ func (s *testIntegrationSuite4) TestExchangePartitionTableCompatiable(c *C) { "create table pt12 (id int not null, lname varchar(30), fname varchar(100) generated always as (concat(lname,' ')) stored) partition by hash(id) partitions 1;", "create table nt12 (id int not null, lname varchar(30), fname varchar(100));", "alter table pt12 exchange partition p0 with table nt12;", - nil, + ddl.ErrTablesDifferentMetadata, }, { "create table pt13 (id int not null, lname varchar(30), fname varchar(100)) partition by hash(id) partitions 1;", "create table nt13 (id int not null, lname varchar(30), fname varchar(100) generated always as (concat(lname, ' ')) stored);", "alter table pt13 exchange partition p0 with table nt13;", - nil, + ddl.ErrTablesDifferentMetadata, }, { "create table pt14 (id int not null, lname varchar(30), fname varchar(100) generated always as (concat(lname, ' ')) virtual) partition by hash(id) partitions 1;", @@ -1097,7 +1109,7 @@ func (s *testIntegrationSuite4) TestExchangePartitionTableCompatiable(c *C) { "create table pt26 (id int not null, lname varchar(30), fname varchar(100) generated always as (concat(lname, ' ')) virtual) partition by hash(id) partitions 1;", "create table nt26 (id int not null, lname varchar(30), fname varchar(100) generated always as (concat(id, ' ')) virtual);", "alter table pt26 exchange partition p0 with table nt26;", - nil, + ddl.ErrTablesDifferentMetadata, }, } @@ -1137,12 +1149,18 @@ func (s *testIntegrationSuite7) TestExchangePartitionExpressIndex(c *C) { // test different expression index when expression returns same field type tk.MustExec("alter table nt1 drop column `_V$_idx_0`;") tk.MustExec("alter table nt1 add index idx((b-c));") - tk.MustExec("alter table pt1 exchange partition p0 with table nt1;") + tk.MustGetErrCode("alter table pt1 exchange partition p0 with table nt1;", tmysql.ErrTablesDifferentMetadata) // test different expression index when expression returns different field type tk.MustExec("alter table nt1 drop index idx;") tk.MustExec("alter table nt1 add index idx((concat(a, b)));") tk.MustGetErrCode("alter table pt1 exchange partition p0 with table nt1;", tmysql.ErrTablesDifferentMetadata) + + tk.MustExec("drop table if exists nt2;") + tk.MustExec("alter table nt2 (a int, b int, c int)") + tk.MustExec("alter table add index idx((a+c))") + tk.MustExec("alter table pt1 exchange partition p0 with table nt2") + } func (s *testIntegrationSuite4) TestAddPartitionTooManyPartitions(c *C) { @@ -2253,7 +2271,7 @@ func (s *testIntegrationSuite3) TestCommitWhenSchemaChange(c *C) { tk.MustExec("admin check table schema_change") tk.MustQuery("select * from schema_change").Check(testkit.Rows()) - // Check iconsistency when exchanging partition + // Check inconsistency when exchanging partition tk.MustExec(`drop table if exists pt, nt;`) tk.MustExec(`create table pt (a int) partition by hash(a) partitions 2;`) tk.MustExec(`create table nt (a int);`) diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 35e4aff869826..8eeca90c5301e 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -2851,9 +2851,11 @@ func checkTableDefCompatible(source *model.TableInfo, target *model.TableInfo) e if isVirtualGeneratedColumn(sourceCol) != isVirtualGeneratedColumn(targetCol) { return ErrUnsupportedOnGeneratedColumn.GenWithStackByArgs("Exchanging partitions for non-generated columns") } + // It should strictyle compare expressions for generated columns if sourceCol.Name.L != targetCol.Name.L || sourceCol.Hidden != targetCol.Hidden || - !checkFieldTypeCompatible(&sourceCol.FieldType, &targetCol.FieldType) { + !checkFieldTypeCompatible(&sourceCol.FieldType, &targetCol.FieldType) || + sourceCol.GeneratedExprString == targetCol.GeneratedExprString { return errors.Trace(ErrTablesDifferentMetadata) } if sourceCol.State != model.StatePublic { diff --git a/ddl/ddl_worker.go b/ddl/ddl_worker.go index de0f9a8aed9e0..d9aa32e05d2bb 100644 --- a/ddl/ddl_worker.go +++ b/ddl/ddl_worker.go @@ -844,11 +844,21 @@ func updateSchemaVersion(t *meta.Meta, job *model.Job) (int64, error) { } diff.TableID = job.TableID case model.ActionExchangeTablePartition: - err = job.DecodeArgs(&diff.TableID, &diff.PtSchemaID, &diff.PtTableID) + var ( + ptSchemaID int64 + ntTableID int64 + ) + err = job.DecodeArgs(&diff.TableID, &ptSchemaID, &ntTableID) if err != nil { return 0, errors.Trace(err) } diff.OldTableID = job.TableID + affects := make([]*model.AffectedOption, 1) + affects[0] = &model.AffectedOption{ + SchemaID: ptSchemaID, + TableID: ntTableID, + } + diff.AffectedOpts = affects default: diff.TableID = job.TableID } diff --git a/go.mod b/go.mod index cc6ab18e99d5b..119eb50a81b3a 100644 --- a/go.mod +++ b/go.mod @@ -64,4 +64,4 @@ require ( go 1.13 -replace github.com/pingcap/parser => github.com/zhaox1n/parser v0.0.0-20200601191909-048d6ec9eef1 +replace github.com/pingcap/parser => github.com/zhaox1n/parser affectedOption diff --git a/infoschema/builder.go b/infoschema/builder.go index c75390eed360d..5a251e7e4e039 100644 --- a/infoschema/builder.go +++ b/infoschema/builder.go @@ -107,11 +107,13 @@ func (b *Builder) ApplyDiff(m *meta.Meta, diff *model.SchemaDiff) ([]int64, erro return nil, errors.Trace(err) } } - if diff.Type == model.ActionExchangeTablePartition { - var err error - tblIDs, err = b.applyExchangePartition(m, diff, tblIDs) - if err != nil { - return nil, errors.Trace(err) + if diff.AffectedOpts != nil { + for _, opt := range diff.AffectedOpts { + var err error + tblIDs, err = b.applyAffectedOpt(m, diff.Type, opt, tblIDs) + if err != nil { + return nil, errors.Trace(err) + } } } return tblIDs, nil @@ -163,15 +165,15 @@ func (b *Builder) copySortedTables(oldTableID, newTableID int64) { } } -func (b *Builder) applyExchangePartition(m *meta.Meta, diff *model.SchemaDiff, affected []int64) ([]int64, error) { - ptDi, err := m.GetDatabase(diff.PtSchemaID) +func (b *Builder) applyAffectedOpt(m *meta.Meta, tp model.ActionType, diff *model.AffectedOption, affected []int64) ([]int64, error) { + ptDi, err := m.GetDatabase(diff.SchemaID) if err != nil { return affected, errors.Trace(err) } - if tableIDIsValid(diff.PtTableID) { - affected = b.applyDropTable(ptDi, diff.PtTableID, affected) + if tableIDIsValid(diff.TableID) { + affected = b.applyDropTable(ptDi, diff.TableID, affected) var allocs autoid.Allocators - affected, err = b.applyCreateTable(m, ptDi, diff.PtTableID, allocs, diff.Type, affected) + affected, err = b.applyCreateTable(m, ptDi, diff.TableID, allocs, tp, affected) if err != nil { return affected, errors.Trace(err) } From b5b68245f0aae5912c275934cfea21f7b5633051 Mon Sep 17 00:00:00 2001 From: zhaox1n Date: Sat, 6 Jun 2020 10:19:27 +0800 Subject: [PATCH 50/57] bug fix --- ddl/ddl_api.go | 2 +- go.mod | 2 +- go.sum | 2 ++ 3 files changed, 4 insertions(+), 2 deletions(-) diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 8eeca90c5301e..3096b739e550b 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -2855,7 +2855,7 @@ func checkTableDefCompatible(source *model.TableInfo, target *model.TableInfo) e if sourceCol.Name.L != targetCol.Name.L || sourceCol.Hidden != targetCol.Hidden || !checkFieldTypeCompatible(&sourceCol.FieldType, &targetCol.FieldType) || - sourceCol.GeneratedExprString == targetCol.GeneratedExprString { + sourceCol.GeneratedExprString != targetCol.GeneratedExprString { return errors.Trace(ErrTablesDifferentMetadata) } if sourceCol.State != model.StatePublic { diff --git a/go.mod b/go.mod index 119eb50a81b3a..f7f0888c502af 100644 --- a/go.mod +++ b/go.mod @@ -64,4 +64,4 @@ require ( go 1.13 -replace github.com/pingcap/parser => github.com/zhaox1n/parser affectedOption +replace github.com/pingcap/parser => github.com/zhaox1n/parser v0.0.0-20200605031011-ca6fb6810542 diff --git a/go.sum b/go.sum index 98e0cabfde64c..66fc3ebfd4d4e 100644 --- a/go.sum +++ b/go.sum @@ -605,6 +605,8 @@ github.com/zhaox1n/parser v0.0.0-20200601184100-acd1a1774f3b h1:ztkWO3u6NFlSkV3v github.com/zhaox1n/parser v0.0.0-20200601184100-acd1a1774f3b/go.mod h1:vQdbJqobJAgFyiRNNtXahpMoGWwPEuWciVEK5A20NS0= github.com/zhaox1n/parser v0.0.0-20200601191909-048d6ec9eef1 h1:f4Qcit4fJibpvNmFSYMDS/O6KHOxQTnwFnfD1gvZ/O4= github.com/zhaox1n/parser v0.0.0-20200601191909-048d6ec9eef1/go.mod h1:vQdbJqobJAgFyiRNNtXahpMoGWwPEuWciVEK5A20NS0= +github.com/zhaox1n/parser v0.0.0-20200605031011-ca6fb6810542 h1:kPMeLbDthIDqmryFgtQ1n2U/Ctj37fCFDWOrW54XJK8= +github.com/zhaox1n/parser v0.0.0-20200605031011-ca6fb6810542/go.mod h1:vQdbJqobJAgFyiRNNtXahpMoGWwPEuWciVEK5A20NS0= go.etcd.io/bbolt v1.3.2/go.mod h1:IbVyRI1SCnLcuJnV2u8VeU0CEYM7e686BmAb1XKL+uU= go.etcd.io/bbolt v1.3.3 h1:MUGmc65QhB3pIlaQ5bB4LwqSj6GIonVJXpZiaKNyaKk= go.etcd.io/bbolt v1.3.3/go.mod h1:IbVyRI1SCnLcuJnV2u8VeU0CEYM7e686BmAb1XKL+uU= From 999b0d92e0ae3408a0ecc81346f0f9957f46b164 Mon Sep 17 00:00:00 2001 From: zhaox1n Date: Sat, 6 Jun 2020 10:27:31 +0800 Subject: [PATCH 51/57] bug fix --- ddl/db_partition_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ddl/db_partition_test.go b/ddl/db_partition_test.go index 165b58b2c6d3b..ad74cd0a1cad6 100644 --- a/ddl/db_partition_test.go +++ b/ddl/db_partition_test.go @@ -1157,7 +1157,7 @@ func (s *testIntegrationSuite7) TestExchangePartitionExpressIndex(c *C) { tk.MustGetErrCode("alter table pt1 exchange partition p0 with table nt1;", tmysql.ErrTablesDifferentMetadata) tk.MustExec("drop table if exists nt2;") - tk.MustExec("alter table nt2 (a int, b int, c int)") + tk.MustExec("create table nt2 (a int, b int, c int)") tk.MustExec("alter table add index idx((a+c))") tk.MustExec("alter table pt1 exchange partition p0 with table nt2") From 2cf5d0368c9f84e95bd6cdc4732797b3d3c50315 Mon Sep 17 00:00:00 2001 From: zhaox1n Date: Sat, 6 Jun 2020 10:39:24 +0800 Subject: [PATCH 52/57] sql fix --- ddl/db_partition_test.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/ddl/db_partition_test.go b/ddl/db_partition_test.go index ad74cd0a1cad6..5fbb5a62828ad 100644 --- a/ddl/db_partition_test.go +++ b/ddl/db_partition_test.go @@ -924,7 +924,7 @@ func (s *testIntegrationSuite4) TestAlterTableExchangePartition(c *C) { tk.MustQuery("select * from e10 partition(p1)").Check(testkit.Rows("1", "3", "5")) // test for column id - tk.MustExec("create table e12 (a int(1), b int, index (a)) partition by hash(a) partitions3") + tk.MustExec("create table e12 (a int(1), b int, index (a)) partition by hash(a) partitions 3") tk.MustExec("create table e13 (a int(8), b int, index (a));") tk.MustExec("alter table e13 drop column b") tk.MustExec("alter table e13 add column b int") @@ -1158,7 +1158,7 @@ func (s *testIntegrationSuite7) TestExchangePartitionExpressIndex(c *C) { tk.MustExec("drop table if exists nt2;") tk.MustExec("create table nt2 (a int, b int, c int)") - tk.MustExec("alter table add index idx((a+c))") + tk.MustExec("alter table nt2 add index idx((a+c))") tk.MustExec("alter table pt1 exchange partition p0 with table nt2") } From 52676f8260f876da7b620dce3f21c80d6c5c159c Mon Sep 17 00:00:00 2001 From: zhaox1n Date: Sun, 7 Jun 2020 02:07:00 +0800 Subject: [PATCH 53/57] add test of auto_random --- ddl/ddl_api.go | 6 ++---- ddl/serial_test.go | 28 ++++++++++++++++++++++++---- 2 files changed, 26 insertions(+), 8 deletions(-) diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 3096b739e550b..ac62d2137dea6 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -2858,10 +2858,8 @@ func checkTableDefCompatible(source *model.TableInfo, target *model.TableInfo) e sourceCol.GeneratedExprString != targetCol.GeneratedExprString { return errors.Trace(ErrTablesDifferentMetadata) } - if sourceCol.State != model.StatePublic { - return errors.Trace(ErrTablesDifferentMetadata) - } - if targetCol.State != model.StatePublic { + if sourceCol.State != model.StatePublic || + targetCol.State != model.StatePublic { return errors.Trace(ErrTablesDifferentMetadata) } if sourceCol.ID != targetCol.ID { diff --git a/ddl/serial_test.go b/ddl/serial_test.go index 3fbed17bea77b..ee80e25b5c61f 100644 --- a/ddl/serial_test.go +++ b/ddl/serial_test.go @@ -978,7 +978,21 @@ func (s *testSerialSuite) TestAutoRandom(c *C) { tk.MustExec("insert into t values()") }) - // Test exchange partition + // Disallow using it when allow-auto-random is not enabled. + config.GetGlobalConfig().Experimental.AllowAutoRandom = false + assertExperimentDisabled("create table auto_random_table (a int primary key auto_random(3))") +} + +func (s *testSerialSuite) TestExchangePartitionForAutoRandom(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("create database if not exists auto_random_db") + defer tk.MustExec("drop database if exists auto_random_db") + + testutil.ConfigTestUtils.SetupAutoRandomTestConfig() + defer testutil.ConfigTestUtils.RestoreAutoRandomTestConfig() + + tk.MustExec("use auto_random_db") + tk.MustExec("drop table if exists e1, e2, e3, e4;") tk.MustExec("create table e1 (a bigint primary key auto_random(3)) partition by hash(a) partitions 1;") @@ -988,13 +1002,19 @@ func (s *testSerialSuite) TestAutoRandom(c *C) { tk.MustExec("create table e3 (a bigint primary key auto_random(2));") tk.MustGetErrCode("alter table e1 exchange partition p0 with table e3;", errno.ErrTablesDifferentMetadata) + tk.MustExec("insert into e1 values (), (), ()") tk.MustExec("create table e4 (a bigint primary key auto_random(3));") + tk.MustExec("insert into e4 values ()") tk.MustExec("alter table e1 exchange partition p0 with table e4;") - // Disallow using it when allow-auto-random is not enabled. - config.GetGlobalConfig().Experimental.AllowAutoRandom = false - assertExperimentDisabled("create table auto_random_table (a int primary key auto_random(3))") + tk.MustQuery("select count(*) from e1").Check(testkit.Rows("1")) + tk.MustExec("insert into e1 values ()") + tk.MustQuery("select count(*) from e1").Check(testkit.Rows("2")) + + tk.MustQuery("select count(*) from e4").Check(testkit.Rows("3")) + tk.MustExec("insert into e4 values ()") + tk.MustQuery("select count(*) from e4").Check(testkit.Rows("4")) } func (s *testSerialSuite) TestModifyingColumn4NewCollations(c *C) { From 6bc1a4a1473ca30eaadfab575a177cd701ec8163 Mon Sep 17 00:00:00 2001 From: zhaox1n Date: Sun, 7 Jun 2020 12:22:02 +0800 Subject: [PATCH 54/57] expand the implementation of ApplyDiff --- ddl/ddl_worker.go | 5 +++-- infoschema/builder.go | 27 ++++++++++----------------- 2 files changed, 13 insertions(+), 19 deletions(-) diff --git a/ddl/ddl_worker.go b/ddl/ddl_worker.go index d9aa32e05d2bb..637444f7c9c11 100644 --- a/ddl/ddl_worker.go +++ b/ddl/ddl_worker.go @@ -855,8 +855,9 @@ func updateSchemaVersion(t *meta.Meta, job *model.Job) (int64, error) { diff.OldTableID = job.TableID affects := make([]*model.AffectedOption, 1) affects[0] = &model.AffectedOption{ - SchemaID: ptSchemaID, - TableID: ntTableID, + SchemaID: ptSchemaID, + TableID: ntTableID, + OldTableID: ntTableID, } diff.AffectedOpts = affects default: diff --git a/infoschema/builder.go b/infoschema/builder.go index 5a251e7e4e039..01718c41cf2d3 100644 --- a/infoschema/builder.go +++ b/infoschema/builder.go @@ -110,10 +110,19 @@ func (b *Builder) ApplyDiff(m *meta.Meta, diff *model.SchemaDiff) ([]int64, erro if diff.AffectedOpts != nil { for _, opt := range diff.AffectedOpts { var err error - tblIDs, err = b.applyAffectedOpt(m, diff.Type, opt, tblIDs) + affectedDiff := &model.SchemaDiff{ + Version: diff.Version, + Type: diff.Type, + SchemaID: opt.SchemaID, + TableID: opt.TableID, + OldSchemaID: opt.OldSchemaID, + OldTableID: opt.OldTableID, + } + affectedIDs, err := b.ApplyDiff(m, affectedDiff) if err != nil { return nil, errors.Trace(err) } + tblIDs = append(tblIDs, affectedIDs...) } } return tblIDs, nil @@ -165,22 +174,6 @@ func (b *Builder) copySortedTables(oldTableID, newTableID int64) { } } -func (b *Builder) applyAffectedOpt(m *meta.Meta, tp model.ActionType, diff *model.AffectedOption, affected []int64) ([]int64, error) { - ptDi, err := m.GetDatabase(diff.SchemaID) - if err != nil { - return affected, errors.Trace(err) - } - if tableIDIsValid(diff.TableID) { - affected = b.applyDropTable(ptDi, diff.TableID, affected) - var allocs autoid.Allocators - affected, err = b.applyCreateTable(m, ptDi, diff.TableID, allocs, tp, affected) - if err != nil { - return affected, errors.Trace(err) - } - } - return affected, nil -} - func (b *Builder) applyCreateSchema(m *meta.Meta, diff *model.SchemaDiff) error { di, err := m.GetDatabase(diff.SchemaID) if err != nil { From 9cd7caabe6ae9da321bb8311f882e49ab234ecf6 Mon Sep 17 00:00:00 2001 From: zhaox1n Date: Sun, 7 Jun 2020 23:49:10 +0800 Subject: [PATCH 55/57] fix typo --- ddl/ddl_worker.go | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/ddl/ddl_worker.go b/ddl/ddl_worker.go index 637444f7c9c11..5590d8b57e60c 100644 --- a/ddl/ddl_worker.go +++ b/ddl/ddl_worker.go @@ -846,9 +846,9 @@ func updateSchemaVersion(t *meta.Meta, job *model.Job) (int64, error) { case model.ActionExchangeTablePartition: var ( ptSchemaID int64 - ntTableID int64 + ptTableID int64 ) - err = job.DecodeArgs(&diff.TableID, &ptSchemaID, &ntTableID) + err = job.DecodeArgs(&diff.TableID, &ptSchemaID, &ptTableID) if err != nil { return 0, errors.Trace(err) } @@ -856,8 +856,8 @@ func updateSchemaVersion(t *meta.Meta, job *model.Job) (int64, error) { affects := make([]*model.AffectedOption, 1) affects[0] = &model.AffectedOption{ SchemaID: ptSchemaID, - TableID: ntTableID, - OldTableID: ntTableID, + TableID: ptTableID, + OldTableID: ptTableID, } diff.AffectedOpts = affects default: From 771e4dde29d6c1ab3e0a3d3e8efe87c2d47126ff Mon Sep 17 00:00:00 2001 From: Zhao Xin Date: Mon, 8 Jun 2020 19:49:58 +0800 Subject: [PATCH 56/57] Update ddl/partition.go Co-authored-by: Lynn --- ddl/partition.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ddl/partition.go b/ddl/partition.go index 778fa54ab6347..34d5495aa37ea 100644 --- a/ddl/partition.go +++ b/ddl/partition.go @@ -874,7 +874,7 @@ func (w *worker) onExchangeTablePartition(d *ddlCtx, t *meta.Meta, job *model.Jo return ver, errors.Trace(err) } - job.FinishTableJob(model.JobStateDone, model.StateNone, ver, nt) + job.FinishTableJob(model.JobStateDone, model.StateNone, ver, pt) return ver, nil } From 44f6018bd5e9149c2eaaeda1ffcb54442e86aca1 Mon Sep 17 00:00:00 2001 From: zhaox1n Date: Tue, 9 Jun 2020 23:25:09 +0800 Subject: [PATCH 57/57] address comment --- ddl/db_partition_test.go | 6 ++++++ ddl/ddl_api.go | 3 ++- go.mod | 4 +--- go.sum | 10 ++-------- 4 files changed, 11 insertions(+), 12 deletions(-) diff --git a/ddl/db_partition_test.go b/ddl/db_partition_test.go index 5fbb5a62828ad..252d7516d2565 100644 --- a/ddl/db_partition_test.go +++ b/ddl/db_partition_test.go @@ -1111,6 +1111,12 @@ func (s *testIntegrationSuite4) TestExchangePartitionTableCompatiable(c *C) { "alter table pt26 exchange partition p0 with table nt26;", ddl.ErrTablesDifferentMetadata, }, + { + "create table pt27 (a int key, b int, index(a)) partition by hash(a) partitions 1;", + "create table nt27 (a int not null, b int, index(a));", + "alter table pt27 exchange partition p0 with table nt27;", + ddl.ErrTablesDifferentMetadata, + }, } tk := testkit.NewTestKit(c, s.store) diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 17acd3b277d65..e711b947ca523 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -2827,7 +2827,8 @@ func checkFieldTypeCompatible(ft *types.FieldType, other *types.FieldType) bool mysql.HasAutoIncrementFlag(ft.Flag) == mysql.HasAutoIncrementFlag(other.Flag) && mysql.HasNotNullFlag(ft.Flag) == mysql.HasNotNullFlag(other.Flag) && mysql.HasZerofillFlag(ft.Flag) == mysql.HasZerofillFlag(other.Flag) && - mysql.HasBinaryFlag(ft.Flag) == mysql.HasBinaryFlag(other.Flag) + mysql.HasBinaryFlag(ft.Flag) == mysql.HasBinaryFlag(other.Flag) && + mysql.HasPriKeyFlag(ft.Flag) == mysql.HasPriKeyFlag(other.Flag) if !partialEqual || len(ft.Elems) != len(other.Elems) { return false } diff --git a/go.mod b/go.mod index 617139b6132ae..62f33220d1737 100644 --- a/go.mod +++ b/go.mod @@ -33,7 +33,7 @@ require ( github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989 github.com/pingcap/kvproto v0.0.0-20200518112156-d4aeb467de29 github.com/pingcap/log v0.0.0-20200511115504-543df19646ad - github.com/pingcap/parser v0.0.0-20200601053017-87d29a820b2d + github.com/pingcap/parser v0.0.0-20200609083335-ebf3b6e5a0a2 github.com/pingcap/pd/v4 v4.0.0-rc.2.0.20200520083007-2c251bd8f181 github.com/pingcap/sysutil v0.0.0-20200408114249-ed3bd6f7fdb1 github.com/pingcap/tidb-tools v4.0.0-rc.1.0.20200514040632-f76b3e428e19+incompatible @@ -64,5 +64,3 @@ require ( ) go 1.13 - -replace github.com/pingcap/parser => github.com/zhaox1n/parser v0.0.0-20200605031011-ca6fb6810542 diff --git a/go.sum b/go.sum index 939baf2dcddbf..94ebb80ea7771 100644 --- a/go.sum +++ b/go.sum @@ -454,8 +454,8 @@ github.com/pingcap/log v0.0.0-20200511115504-543df19646ad/go.mod h1:4rbK1p9ILyIf github.com/pingcap/parser v0.0.0-20200424075042-8222d8b724a4/go.mod h1:9v0Edh8IbgjGYW2ArJr19E+bvL8zKahsFp+ixWeId+4= github.com/pingcap/parser v0.0.0-20200507022230-f3bf29096657/go.mod h1:9v0Edh8IbgjGYW2ArJr19E+bvL8zKahsFp+ixWeId+4= github.com/pingcap/parser v0.0.0-20200522094936-3b720a0512a6/go.mod h1:vQdbJqobJAgFyiRNNtXahpMoGWwPEuWciVEK5A20NS0= -github.com/pingcap/parser v0.0.0-20200601053017-87d29a820b2d h1:eXigbUyAJDOhAIX9Hx72SqwbfSmpMiohdVM+WV5M0QQ= -github.com/pingcap/parser v0.0.0-20200601053017-87d29a820b2d/go.mod h1:vQdbJqobJAgFyiRNNtXahpMoGWwPEuWciVEK5A20NS0= +github.com/pingcap/parser v0.0.0-20200609083335-ebf3b6e5a0a2 h1:UcyGqy3itmPNQkVhOo5YFr4/QRfw+KNwW2b02b1/S2A= +github.com/pingcap/parser v0.0.0-20200609083335-ebf3b6e5a0a2/go.mod h1:vQdbJqobJAgFyiRNNtXahpMoGWwPEuWciVEK5A20NS0= github.com/pingcap/pd/v4 v4.0.0-rc.1.0.20200422143320-428acd53eba2 h1:JTzYYukREvxVSKW/ncrzNjFitd8snoQ/Xz32pw8i+s8= github.com/pingcap/pd/v4 v4.0.0-rc.1.0.20200422143320-428acd53eba2/go.mod h1:s+utZtXDznOiL24VK0qGmtoHjjXNsscJx3m1n8cC56s= github.com/pingcap/pd/v4 v4.0.0-rc.2.0.20200520083007-2c251bd8f181 h1:FM+PzdoR3fmWAJx3ug+p5aOgs5aZYwFkoDL7Potdsz0= @@ -612,12 +612,6 @@ github.com/yookoala/realpath v1.0.0/go.mod h1:gJJMA9wuX7AcqLy1+ffPatSCySA1FQ2S8Y github.com/yuin/goldmark v1.1.25/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/zhangjinpeng1987/raft v0.0.0-20190624145930-deeb32d6553d h1:rGkexfPDxNuTCObUwTbsRUlti+evR/Ksb4dKy6esXW0= github.com/zhangjinpeng1987/raft v0.0.0-20190624145930-deeb32d6553d/go.mod h1:1KDQ09J8MRHEtHze4at7BJZDW/doUAgkJ8w9KjEUhSo= -github.com/zhaox1n/parser v0.0.0-20200601184100-acd1a1774f3b h1:ztkWO3u6NFlSkV3vr3vOdeOlh8owOpHx9t5qZvFO5IM= -github.com/zhaox1n/parser v0.0.0-20200601184100-acd1a1774f3b/go.mod h1:vQdbJqobJAgFyiRNNtXahpMoGWwPEuWciVEK5A20NS0= -github.com/zhaox1n/parser v0.0.0-20200601191909-048d6ec9eef1 h1:f4Qcit4fJibpvNmFSYMDS/O6KHOxQTnwFnfD1gvZ/O4= -github.com/zhaox1n/parser v0.0.0-20200601191909-048d6ec9eef1/go.mod h1:vQdbJqobJAgFyiRNNtXahpMoGWwPEuWciVEK5A20NS0= -github.com/zhaox1n/parser v0.0.0-20200605031011-ca6fb6810542 h1:kPMeLbDthIDqmryFgtQ1n2U/Ctj37fCFDWOrW54XJK8= -github.com/zhaox1n/parser v0.0.0-20200605031011-ca6fb6810542/go.mod h1:vQdbJqobJAgFyiRNNtXahpMoGWwPEuWciVEK5A20NS0= go.etcd.io/bbolt v1.3.2/go.mod h1:IbVyRI1SCnLcuJnV2u8VeU0CEYM7e686BmAb1XKL+uU= go.etcd.io/bbolt v1.3.3 h1:MUGmc65QhB3pIlaQ5bB4LwqSj6GIonVJXpZiaKNyaKk= go.etcd.io/bbolt v1.3.3/go.mod h1:IbVyRI1SCnLcuJnV2u8VeU0CEYM7e686BmAb1XKL+uU=