From 8e8da6d4e4d98a1ab3caa817e45e8e73efc1ffec Mon Sep 17 00:00:00 2001 From: Mattias Jonsson Date: Fri, 29 Nov 2024 01:31:23 +0100 Subject: [PATCH] This is an automated cherry-pick of #57114 Signed-off-by: ti-chi-bot --- pkg/ddl/delete_range.go | 30 + pkg/ddl/partition.go | 386 +++- pkg/ddl/sanity_check.go | 16 +- pkg/ddl/tests/partition/BUILD.bazel | 1 + pkg/ddl/tests/partition/db_partition_test.go | 112 +- pkg/ddl/tests/partition/multi_domain_test.go | 1445 ++++++++++++++ .../tests/partition/reorg_partition_test.go | 1022 ++++++++++ pkg/meta/model/job_args.go | 1719 +++++++++++++++++ pkg/table/tables/partition.go | 13 +- .../integrationtest/r/ddl/db_partition.result | 4 +- 10 files changed, 4681 insertions(+), 67 deletions(-) create mode 100644 pkg/ddl/tests/partition/multi_domain_test.go create mode 100644 pkg/ddl/tests/partition/reorg_partition_test.go create mode 100644 pkg/meta/model/job_args.go diff --git a/pkg/ddl/delete_range.go b/pkg/ddl/delete_range.go index c06ebe4a62432..3bf7dcee7ba7d 100644 --- a/pkg/ddl/delete_range.go +++ b/pkg/ddl/delete_range.go @@ -332,6 +332,36 @@ func insertJobIntoDeleteRangeTable(ctx context.Context, sctx sessionctx.Context, return errors.Trace(err) } } +<<<<<<< HEAD +======= + // always delete the table range, even when it's a partitioned table where + // it may contain global index regions. + return errors.Trace(doBatchDeleteTablesRange(ctx, wrapper, job.ID, []int64{tableID}, ea, "truncate table: table ID")) + case model.ActionDropTablePartition: + args, err := model.GetFinishedTablePartitionArgs(job) + if err != nil { + return errors.Trace(err) + } + return errors.Trace(doBatchDeleteTablesRange(ctx, wrapper, job.ID, args.OldPhysicalTblIDs, ea, "drop partition: physical table ID(s)")) + case model.ActionReorganizePartition, model.ActionRemovePartitioning, model.ActionAlterTablePartitioning: + // Delete dropped partitions, as well as replaced global indexes. + args, err := model.GetFinishedTablePartitionArgs(job) + if err != nil { + return errors.Trace(err) + } + for _, idx := range args.OldGlobalIndexes { + if err := doBatchDeleteIndiceRange(ctx, wrapper, job.ID, idx.TableID, []int64{idx.IndexID}, ea, "reorganize partition, replaced global indexes"); err != nil { + return errors.Trace(err) + } + } + return errors.Trace(doBatchDeleteTablesRange(ctx, wrapper, job.ID, args.OldPhysicalTblIDs, ea, "reorganize partition: physical table ID(s)")) + case model.ActionTruncateTablePartition: + args, err := model.GetTruncateTableArgs(job) + if err != nil { + return errors.Trace(err) + } + return errors.Trace(doBatchDeleteTablesRange(ctx, wrapper, job.ID, args.OldPartitionIDs, ea, "truncate partition: physical table ID(s)")) +>>>>>>> b6025b97877 (*: Reorg partition fix delete ranges and handling non-clustered tables with concurrent DML (#57114)) // ActionAddIndex, ActionAddPrimaryKey needs do it, because it needs to be rolled back when it's canceled. case model.ActionAddIndex, model.ActionAddPrimaryKey: allIndexIDs := make([]int64, 1) diff --git a/pkg/ddl/partition.go b/pkg/ddl/partition.go index f0f13b7e39236..e8c84339d4bfd 100644 --- a/pkg/ddl/partition.go +++ b/pkg/ddl/partition.go @@ -44,7 +44,11 @@ import ( "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/opcode" +<<<<<<< HEAD "github.com/pingcap/tidb/pkg/parser/terror" +======= + field_types "github.com/pingcap/tidb/pkg/parser/types" +>>>>>>> b6025b97877 (*: Reorg partition fix delete ranges and handling non-clustered tables with concurrent DML (#57114)) "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/table" @@ -1948,6 +1952,7 @@ func dropLabelRules(_ *ddlCtx, schemaName, tableName string, partNames []string) } // delete batch rules patch := label.NewRulePatch([]*label.Rule{}, deleteRules) +<<<<<<< HEAD return infosync.UpdateLabelRules(context.TODO(), patch) } @@ -1956,6 +1961,116 @@ func (w *worker) onDropTablePartition(d *ddlCtx, t *meta.Meta, job *model.Job) ( var partNames []string partInfo := model.PartitionInfo{} if err := job.DecodeArgs(&partNames, &partInfo); err != nil { +======= + return infosync.UpdateLabelRules(ctx, patch) +} + +// rollbackLikeDropPartition does rollback for Reorganize partition and Add partition. +// It will drop newly created partitions that has not yet been used, including cleaning +// up label rules and bundles as well as changed indexes due to global flag. +func (w *worker) rollbackLikeDropPartition(jobCtx *jobContext, job *model.Job) (ver int64, _ error) { + args := jobCtx.jobArgs.(*model.TablePartitionArgs) + partInfo := args.PartInfo + metaMut := jobCtx.metaMut + tblInfo, err := GetTableInfoAndCancelFaultJob(metaMut, job, job.SchemaID) + if err != nil { + return ver, errors.Trace(err) + } + tblInfo.Partition.DroppingDefinitions = nil + physicalTableIDs, pNames, rollbackBundles := rollbackAddingPartitionInfo(tblInfo) + err = infosync.PutRuleBundlesWithDefaultRetry(context.TODO(), rollbackBundles) + if err != nil { + job.State = model.JobStateCancelled + return ver, errors.Wrapf(err, "failed to notify PD the placement rules") + } + // TODO: Will this drop LabelRules for existing partitions, if the new partitions have the same name? + err = dropLabelRules(w.ctx, job.SchemaName, tblInfo.Name.L, pNames) + if err != nil { + job.State = model.JobStateCancelled + return ver, errors.Wrapf(err, "failed to notify PD the label rules") + } + + if _, err := alterTableLabelRule(job.SchemaName, tblInfo, getIDs([]*model.TableInfo{tblInfo})); err != nil { + job.State = model.JobStateCancelled + return ver, err + } + if partInfo.Type != pmodel.PartitionTypeNone { + // ALTER TABLE ... PARTITION BY + // Also remove anything with the new table id + physicalTableIDs = append(physicalTableIDs, partInfo.NewTableID) + // Reset if it was normal table before + if tblInfo.Partition.Type == pmodel.PartitionTypeNone || + tblInfo.Partition.DDLType == pmodel.PartitionTypeNone { + tblInfo.Partition = nil + } + } + + var dropIndices []*model.IndexInfo + for _, indexInfo := range tblInfo.Indices { + if indexInfo.State == model.StateWriteOnly { + dropIndices = append(dropIndices, indexInfo) + } + } + var deleteIndices []model.TableIDIndexID + for _, indexInfo := range dropIndices { + DropIndexColumnFlag(tblInfo, indexInfo) + RemoveDependentHiddenColumns(tblInfo, indexInfo) + removeIndexInfo(tblInfo, indexInfo) + if indexInfo.Global { + deleteIndices = append(deleteIndices, model.TableIDIndexID{TableID: tblInfo.ID, IndexID: indexInfo.ID}) + } + // All other indexes has only been applied to new partitions, that is deleted in whole, + // including indexes. + } + if tblInfo.Partition != nil { + tblInfo.Partition.ClearReorgIntermediateInfo() + } + + ver, err = updateVersionAndTableInfo(jobCtx, job, tblInfo, true) + if err != nil { + return ver, errors.Trace(err) + } + job.FinishTableJob(model.JobStateRollbackDone, model.StateNone, ver, tblInfo) + args.OldPhysicalTblIDs = physicalTableIDs + args.OldGlobalIndexes = deleteIndices + job.FillFinishedArgs(args) + return ver, nil +} + +// onDropTablePartition deletes old partition meta. +// States in reverse order: +// StateNone +// +// Old partitions are queued to be deleted (delete_range), global index up-to-date +// +// StateDeleteReorganization +// +// Old partitions are not accessible/used by any sessions. +// Inserts/updates of global index which still have entries pointing to old partitions +// will overwrite those entries +// In the background we are reading all old partitions and deleting their entries from +// the global indexes. +// +// StateDeleteOnly +// +// old partitions are no longer visible, but if there is inserts/updates to the global indexes, +// duplicate key errors will be given, even if the entries are from dropped partitions +// Note that overlapping ranges (i.e. a dropped partitions with 'less than (N)' will now .. ?!? +// +// StateWriteOnly +// +// old partitions are blocked for read and write. But for read we are allowing +// "overlapping" partition to be read instead. Which means that write can only +// happen in the 'overlapping' partitions original range, not into the extended +// range open by the dropped partitions. +// +// StatePublic +// +// Original state, unaware of DDL +func (w *worker) onDropTablePartition(jobCtx *jobContext, job *model.Job) (ver int64, _ error) { + args, err := model.GetTablePartitionArgs(job) + if err != nil { +>>>>>>> b6025b97877 (*: Reorg partition fix delete ranges and handling non-clustered tables with concurrent DML (#57114)) job.State = model.JobStateCancelled return ver, errors.Trace(err) } @@ -3012,6 +3127,34 @@ func (w *worker) onReorganizePartition(d *ddlCtx, t *meta.Meta, job *model.Job) tblInfo.Partition.AddingDefinitions = nil tblInfo.Partition.DDLState = model.StateNone +<<<<<<< HEAD +======= + var dropIndices []*model.IndexInfo + for _, indexInfo := range tblInfo.Indices { + if indexInfo.Unique && indexInfo.State == model.StateDeleteOnly { + // Drop the old unique (possible global) index, see onDropIndex + indexInfo.State = model.StateNone + DropIndexColumnFlag(tblInfo, indexInfo) + RemoveDependentHiddenColumns(tblInfo, indexInfo) + dropIndices = append(dropIndices, indexInfo) + } + } + // Local indexes is not an issue, since they will be gone with the dropped + // partitions, but replaced global indexes should be checked! + for _, indexInfo := range dropIndices { + removeIndexInfo(tblInfo, indexInfo) + if indexInfo.Global { + args.OldGlobalIndexes = append(args.OldGlobalIndexes, model.TableIDIndexID{TableID: tblInfo.ID, IndexID: indexInfo.ID}) + } + } + failpoint.Inject("reorgPartFail4", func(val failpoint.Value) { + if val.(bool) { + job.ErrorCount += variable.GetDDLErrorCountLimit() / 2 + failpoint.Return(ver, errors.New("Injected error by reorgPartFail4")) + } + }) + var oldTblID int64 +>>>>>>> b6025b97877 (*: Reorg partition fix delete ranges and handling non-clustered tables with concurrent DML (#57114)) if job.Type != model.ActionReorganizePartition { // ALTER TABLE ... PARTITION BY // REMOVE PARTITIONING @@ -3094,8 +3237,54 @@ func doPartitionReorgWork(w *worker, d *ddlCtx, t *meta.Meta, job *model.Job, tb } defer w.sessPool.Put(sctx) rh := newReorgHandler(sess.NewSession(sctx)) +<<<<<<< HEAD elements := BuildElements(tbl.Meta().Columns[0], tbl.Meta().Indices) partTbl, ok := tbl.(table.PartitionedTable) +======= + reorgTblInfo := tbl.Meta().Clone() + var elements []*meta.Element + isClustered := tbl.Meta().PKIsHandle || tbl.Meta().IsCommonHandle + if isClustered { + indices := make([]*model.IndexInfo, 0, len(tbl.Meta().Indices)) + for _, index := range tbl.Meta().Indices { + if isNew, ok := tbl.Meta().GetPartitionInfo().DDLChangedIndex[index.ID]; ok && !isNew { + // Skip old replaced indexes, but rebuild all other indexes + continue + } + indices = append(indices, index) + } + elements = BuildElements(tbl.Meta().Columns[0], indices) + } else { + // Non-clustered tables needs to generate new _tidb_rowid for each row, since + // there might be duplicates due to EXCHANGE PARTITION. + // That means that we can not first copy all table records and then + // recreate all indexes, since we cannot determine if a table record + // has been copied or not, since its _tidb_rowid handle has been recreated + // in the new partition. + // So we will read a batch of records from one partition at a time, + // do a BatchGet for all the record keys in the new partitions, + // to see if any of the records is already there with the same handle/_tidb_rowid + // which means they were double written and does not need to be copied. + // use AddRecord for all non-matching records. + // TODO: if there is an issue where we will retry the same batch and we have committed + // backfilled records and indexes without committing the updated reorgInfo start/end key, + // then the DDL can fail due to duplicate key. + reorgTblInfo.Indices = reorgTblInfo.Indices[:0] + for _, index := range tbl.Meta().Indices { + if isNew, ok := tbl.Meta().GetPartitionInfo().DDLChangedIndex[index.ID]; ok && !isNew { + // Skip old replaced indexes, but rebuild all other indexes + continue + } + reorgTblInfo.Indices = append(reorgTblInfo.Indices, index) + } + elements = BuildElements(tbl.Meta().Columns[0], reorgTblInfo.Indices) + } + reorgTbl, err := getTable(jobCtx.getAutoIDRequirement(), job.SchemaID, reorgTblInfo) + if err != nil { + return false, ver, errors.Trace(err) + } + partTbl, ok := reorgTbl.(table.PartitionedTable) +>>>>>>> b6025b97877 (*: Reorg partition fix delete ranges and handling non-clustered tables with concurrent DML (#57114)) if !ok { return false, ver, dbterror.ErrUnsupportedReorganizePartition.GenWithStackByArgs() } @@ -3103,13 +3292,22 @@ func doPartitionReorgWork(w *worker, d *ddlCtx, t *meta.Meta, job *model.Job, tb if err != nil { return false, ver, errors.Trace(err) } +<<<<<<< HEAD reorgInfo, err := getReorgInfoFromPartitions(d.jobContext(job.ID, job.ReorgMeta), d, rh, job, dbInfo, partTbl, physTblIDs, elements) err = w.runReorgJob(reorgInfo, tbl.Meta(), d.lease, func() (reorgErr error) { +======= + reorgInfo, err := getReorgInfoFromPartitions(jobCtx.oldDDLCtx.jobContext(job.ID, job.ReorgMeta), jobCtx, rh, job, dbInfo, partTbl, physTblIDs, elements) + err = w.runReorgJob(reorgInfo, reorgTbl.Meta(), func() (reorgErr error) { +>>>>>>> b6025b97877 (*: Reorg partition fix delete ranges and handling non-clustered tables with concurrent DML (#57114)) defer tidbutil.Recover(metrics.LabelDDL, "doPartitionReorgWork", func() { reorgErr = dbterror.ErrCancelledDDLJob.GenWithStack("reorganize partition for table `%v` panic", tbl.Meta().Name) }, false) +<<<<<<< HEAD return w.reorgPartitionDataAndIndex(tbl, reorgInfo) +======= + return w.reorgPartitionDataAndIndex(jobCtx.stepCtx, reorgTbl, reorgInfo) +>>>>>>> b6025b97877 (*: Reorg partition fix delete ranges and handling non-clustered tables with concurrent DML (#57114)) }) if err != nil { if dbterror.ErrPausedDDLJob.Equal(err) { @@ -3127,8 +3325,14 @@ func doPartitionReorgWork(w *worker, d *ddlCtx, t *meta.Meta, job *model.Job, tb logutil.BgLogger().Warn("reorg partition job failed, RemoveDDLReorgHandle failed, can't convert job to rollback", zap.String("category", "ddl"), zap.String("job", job.String()), zap.Error(err1)) } +<<<<<<< HEAD logutil.BgLogger().Warn("reorg partition job failed, convert job to rollback", zap.String("category", "ddl"), zap.String("job", job.String()), zap.Error(err)) ver, err = convertAddTablePartitionJob2RollbackJob(d, t, job, err, tbl.Meta()) +======= + logutil.DDLLogger().Warn("reorg partition job failed, convert job to rollback", zap.Stringer("job", job), zap.Error(err)) + // TODO: Test and verify that this returns an error on the ALTER TABLE session. + ver, err = rollbackReorganizePartitionWithErr(jobCtx, job, err) +>>>>>>> b6025b97877 (*: Reorg partition fix delete ranges and handling non-clustered tables with concurrent DML (#57114)) return false, ver, errors.Trace(err) } return true, ver, err @@ -3136,6 +3340,7 @@ func doPartitionReorgWork(w *worker, d *ddlCtx, t *meta.Meta, job *model.Job, tb type reorgPartitionWorker struct { *backfillCtx + records int // Static allocated to limit memory allocations rowRecords []*rowRecord rowDecoder *decoder.RowDecoder @@ -3143,6 +3348,15 @@ type reorgPartitionWorker struct { writeColOffsetMap map[int64]int maxOffset int reorgedTbl table.PartitionedTable + // Only used for non-clustered tables, since we need to re-generate _tidb_rowid, + // and check if the old _tidb_rowid was already written or not. + // If the old _tidb_rowid already exists, then the row is already backfilled (double written) + // and can be skipped. Otherwise, we will insert it and generate index entries. + rows [][]types.Datum + // The original _tidb_rowids, used to check if already backfilled (double written). + oldKeys []kv.Key + // partition ids of the new rows + newPids []int64 } func newReorgPartitionWorker(sessCtx sessionctx.Context, i int, t table.PhysicalTable, decodeColMap map[int64]decoder.Column, reorgInfo *reorgInfo, jc *JobContext) (*reorgPartitionWorker, error) { @@ -3190,43 +3404,66 @@ func (w *reorgPartitionWorker) BackfillData(handleRange reorgBackfillTask) (task } txn.SetOption(kv.ResourceGroupName, w.jobContext.resourceGroupName) - rowRecords, nextKey, taskDone, err := w.fetchRowColVals(txn, handleRange) + nextKey, taskDone, err := w.fetchRowColVals(txn, handleRange) if err != nil { return errors.Trace(err) } taskCtx.nextKey = nextKey taskCtx.done = taskDone - warningsMap := make(map[errors.ErrorID]*terror.Error) - warningsCountMap := make(map[errors.ErrorID]int64) - for _, prr := range rowRecords { - taskCtx.scanCount++ + isClustered := w.reorgedTbl.Meta().IsCommonHandle || w.reorgedTbl.Meta().PKIsHandle + if !isClustered { + // non-clustered table, we need to replace the _tidb_rowid handles since + // there may be duplicates across different partitions, due to EXCHANGE PARTITION. + // Meaning we need to check here if a record was double written to the new partition, + // i.e. concurrently written by StateWriteOnly or StateWriteReorganization. + // and we should skip it. + var found map[string][]byte + if len(w.oldKeys) > 0 { + // we must check if old IDs already been written, + // i.e. double written by StateWriteOnly or StateWriteReorganization. + // The good thing is that we can then also skip the index generation for that row and we don't need to + // check if duplicate index entries was already copied either! + // TODO: while waiting for BatchGet to check for duplicate, do another round of reads in parallel? + found, err = txn.BatchGet(ctx, w.oldKeys) + if err != nil { + return errors.Trace(err) + } + } + + for i := 0; i < w.records; i++ { + taskCtx.scanCount++ + if len(w.oldKeys) > 0 { + if _, ok := found[string(w.oldKeys[i])]; ok { + // Alredy filled + continue + } + tbl := w.reorgedTbl.GetPartition(w.newPids[i]) + if tbl == nil { + return dbterror.ErrUnsupportedReorganizePartition.GenWithStackByArgs() + } + // TODO: is this looking up each index entry at a time or in an optimistic way and only checks + // at commit time? + // AddRecord will assign a new _tidb_rowid, since we don't provide one. + _, err = tbl.AddRecord(w.tblCtx, txn, w.rows[i]) + if err != nil { + return errors.Trace(err) + } + taskCtx.addedCount++ + } + } + return nil + } + // Clustered table, use tried implementation + for _, prr := range w.rowRecords { + taskCtx.scanCount++ err = txn.Set(prr.key, prr.vals) if err != nil { return errors.Trace(err) } taskCtx.addedCount++ - if prr.warning != nil { - if _, ok := warningsCountMap[prr.warning.ID()]; ok { - warningsCountMap[prr.warning.ID()]++ - } else { - warningsCountMap[prr.warning.ID()] = 1 - warningsMap[prr.warning.ID()] = prr.warning - } - } - // TODO: Future optimization: also write the indexes here? - // What if the transaction limit is just enough for a single row, without index? - // Hmm, how could that be in the first place? - // For now, implement the batch-txn w.addTableIndex, - // since it already exists and is in use } - - // Collect the warnings. - taskCtx.warnings, taskCtx.warningsCount = warningsMap, warningsCountMap - - // also add the index entries here? And make sure they are not added somewhere else - return nil }) logSlowOperations(time.Since(oprStartTime), "BackfillData", 3000) @@ -3234,15 +3471,24 @@ func (w *reorgPartitionWorker) BackfillData(handleRange reorgBackfillTask) (task return } -func (w *reorgPartitionWorker) fetchRowColVals(txn kv.Transaction, taskRange reorgBackfillTask) ([]*rowRecord, kv.Key, bool, error) { +func (w *reorgPartitionWorker) fetchRowColVals(txn kv.Transaction, taskRange reorgBackfillTask) (kv.Key, bool, error) { w.rowRecords = w.rowRecords[:0] + w.records = 0 + isClustered := w.reorgedTbl.Meta().IsCommonHandle || w.reorgedTbl.Meta().PKIsHandle + if !isClustered { + if cap(w.rows) < w.batchCnt { + w.rows = make([][]types.Datum, w.batchCnt) + } + } + w.oldKeys = w.oldKeys[:0] + w.newPids = w.newPids[:0] startTime := time.Now() // taskDone means that the added handle is out of taskRange.endHandle. taskDone := false sysTZ := w.sessCtx.GetSessionVars().StmtCtx.TimeZone() - tmpRow := make([]types.Datum, w.maxOffset+1) + tmpRow := make([]types.Datum, len(w.reorgedTbl.Cols())) var lastAccessedHandle kv.Key oprStartTime := startTime err := iterateSnapshotKeys(w.jobContext, w.sessCtx.GetStore(), taskRange.priority, w.table.RecordPrefix(), txn.StartTS(), taskRange.startKey, taskRange.endKey, @@ -3253,7 +3499,7 @@ func (w *reorgPartitionWorker) fetchRowColVals(txn kv.Transaction, taskRange reo taskDone = recordKey.Cmp(taskRange.endKey) >= 0 - if taskDone || len(w.rowRecords) >= w.batchCnt { + if taskDone || w.records >= w.batchCnt { return false, nil } @@ -3264,24 +3510,42 @@ func (w *reorgPartitionWorker) fetchRowColVals(txn kv.Transaction, taskRange reo return false, errors.Trace(err) } - // Set the partitioning columns and calculate which partition to write to - for colID, offset := range w.writeColOffsetMap { - d, ok := w.rowMap[colID] - if !ok { - return false, dbterror.ErrUnsupportedReorganizePartition.GenWithStackByArgs() + if isClustered { + // Set all partitioning columns and calculate which partition to write to + for colID, offset := range w.writeColOffsetMap { + d, ok := w.rowMap[colID] + if !ok { + return false, dbterror.ErrUnsupportedReorganizePartition.GenWithStackByArgs() + } + tmpRow[offset] = d + } + } else { + // _tidb_rowid needs to be regenerated, due to EXCHANGE PARTITION, meaning we cannot + // delay the index generation, but need to check if the current _tidb_rowid already exists + // in the new partition or not, before we write the newly generated one. + // and later in the caller of this function write both Record and all indexes + + // Set all columns and calculate which partition to write to + // We will later copy the row, so use all writable columns + for _, col := range w.reorgedTbl.WritableCols() { + d, ok := w.rowMap[col.ID] + if !ok { + return false, dbterror.ErrUnsupportedReorganizePartition.GenWithStackByArgs() + } + tmpRow[col.Offset] = d } - tmpRow[offset] = d } p, err := w.reorgedTbl.GetPartitionByRow(w.sessCtx, tmpRow) if err != nil { return false, errors.Trace(err) } - var newKey kv.Key - if w.reorgedTbl.Meta().PKIsHandle || w.reorgedTbl.Meta().IsCommonHandle { - pid := p.GetPhysicalID() - newKey = tablecodec.EncodeTablePrefix(pid) - newKey = append(newKey, recordKey[len(newKey):]...) + if isClustered { + newKey := tablecodec.EncodeTablePrefix(p.GetPhysicalID()) + newKey = append(newKey, recordKey[tablecodec.TableSplitKeyLen:]...) + w.rowRecords = append(w.rowRecords, &rowRecord{key: newKey, vals: rawRow}) + w.records++ } else { +<<<<<<< HEAD // Non-clustered table / not unique _tidb_rowid for the whole table // Generate new _tidb_rowid if exists. // Due to EXCHANGE PARTITION, the existing _tidb_rowid may collide between partitions! @@ -3300,10 +3564,19 @@ func (w *reorgPartitionWorker) fetchRowColVals(txn kv.Transaction, taskRange reo return false, errors.Trace(err) } newKey = tablecodec.EncodeRecordKey(p.RecordPrefix(), recordID) +======= + if cap(w.rows[w.records]) < len(tmpRow) { + w.rows[w.records] = make([]types.Datum, len(tmpRow)) + } + copy(w.rows[w.records], tmpRow) + w.newPids = append(w.newPids, p.GetPhysicalID()) + + oldKey := tablecodec.EncodeTablePrefix(p.GetPhysicalID()) + oldKey = append(oldKey, recordKey[tablecodec.TableSplitKeyLen:]...) + w.oldKeys = append(w.oldKeys, oldKey) + w.records++ +>>>>>>> b6025b97877 (*: Reorg partition fix delete ranges and handling non-clustered tables with concurrent DML (#57114)) } - w.rowRecords = append(w.rowRecords, &rowRecord{ - key: newKey, vals: rawRow, - }) w.cleanRowMap() lastAccessedHandle = recordKey @@ -3314,12 +3587,20 @@ func (w *reorgPartitionWorker) fetchRowColVals(txn kv.Transaction, taskRange reo return true, nil }) - if len(w.rowRecords) == 0 { + if w.records == 0 { taskDone = true } +<<<<<<< HEAD logutil.BgLogger().Debug("txn fetches handle info", zap.String("category", "ddl"), zap.Uint64("txnStartTS", txn.StartTS()), zap.String("taskRange", taskRange.String()), zap.Duration("takeTime", time.Since(startTime))) return w.rowRecords, getNextHandleKey(taskRange, taskDone, lastAccessedHandle), taskDone, errors.Trace(err) +======= + logutil.DDLLogger().Debug("txn fetches handle info", + zap.Uint64("txnStartTS", txn.StartTS()), + zap.Stringer("taskRange", &taskRange), + zap.Duration("takeTime", time.Since(startTime))) + return getNextHandleKey(taskRange, taskDone, lastAccessedHandle), taskDone, errors.Trace(err) +>>>>>>> b6025b97877 (*: Reorg partition fix delete ranges and handling non-clustered tables with concurrent DML (#57114)) } func (w *reorgPartitionWorker) cleanRowMap() { @@ -3346,9 +3627,16 @@ func (w *worker) reorgPartitionDataAndIndex(t table.Table, reorgInfo *reorgInfo) // Then create all indexes on the AddingDefinitions partitions // for each new index, one partition at a time. + isClustered := t.Meta().PKIsHandle || t.Meta().IsCommonHandle + // Copy the data from the DroppingDefinitions to the AddingDefinitions if bytes.Equal(reorgInfo.currElement.TypeKey, meta.ColumnElementKey) { +<<<<<<< HEAD err := w.updatePhysicalTableRow(t, reorgInfo) +======= + // if non-clustered table it will also create its indexes! + err = w.updatePhysicalTableRow(ctx, t, reorgInfo) +>>>>>>> b6025b97877 (*: Reorg partition fix delete ranges and handling non-clustered tables with concurrent DML (#57114)) if err != nil { return errors.Trace(err) } @@ -3431,10 +3719,24 @@ func (w *worker) reorgPartitionDataAndIndex(t table.Table, reorgInfo *reorgInfo) } reorgInfo.PhysicalTableID = firstNewPartitionID } +<<<<<<< HEAD failpoint.Inject("reorgPartitionAfterIndex", func(val failpoint.Value) { //nolint:forcetypeassert if val.(bool) { panic("panic test in reorgPartitionAfterIndex") +======= + + pi := t.Meta().GetPartitionInfo() + if _, err = findNextPartitionID(reorgInfo.PhysicalTableID, pi.AddingDefinitions); err == nil { + // Now build all the indexes in the new partitions + // apart from non-clustered index tables, where new partitions already + // created its indexes together with the table records. + if isClustered { + err = w.addTableIndex(ctx, t, reorgInfo) + if err != nil { + return errors.Trace(err) + } +>>>>>>> b6025b97877 (*: Reorg partition fix delete ranges and handling non-clustered tables with concurrent DML (#57114)) } }) return nil diff --git a/pkg/ddl/sanity_check.go b/pkg/ddl/sanity_check.go index d8108e71869cd..20df0a1865e2f 100644 --- a/pkg/ddl/sanity_check.go +++ b/pkg/ddl/sanity_check.go @@ -48,7 +48,7 @@ func (d *ddl) checkDeleteRangeCnt(job *model.Job) { panic(err) } if actualCnt != expectedCnt { - panic(fmt.Sprintf("expect delete range count %d, actual count %d", expectedCnt, actualCnt)) + panic(fmt.Sprintf("expect delete range count %d, actual count %d for job type '%s'", expectedCnt, actualCnt, job.Type.String())) } } @@ -106,7 +106,21 @@ func expectedDeleteRangeCnt(ctx delRangeCntCtx, job *model.Job) (int, error) { if err := job.DecodeArgs(&physicalTableIDs); err != nil { return 0, errors.Trace(err) } +<<<<<<< HEAD return len(physicalTableIDs), nil +======= + if job.Type == model.ActionTruncateTable { + return len(args.OldPartitionIDs) + 1, nil + } + return len(args.OldPartitionIDs), nil + case model.ActionDropTablePartition, model.ActionReorganizePartition, + model.ActionRemovePartitioning, model.ActionAlterTablePartitioning: + args, err := model.GetFinishedTablePartitionArgs(job) + if err != nil { + return 0, errors.Trace(err) + } + return len(args.OldPhysicalTblIDs) + len(args.OldGlobalIndexes), nil +>>>>>>> b6025b97877 (*: Reorg partition fix delete ranges and handling non-clustered tables with concurrent DML (#57114)) case model.ActionAddIndex, model.ActionAddPrimaryKey: indexID := make([]int64, 1) ifExists := make([]bool, 1) diff --git a/pkg/ddl/tests/partition/BUILD.bazel b/pkg/ddl/tests/partition/BUILD.bazel index 1183232ee444d..4163d47031772 100644 --- a/pkg/ddl/tests/partition/BUILD.bazel +++ b/pkg/ddl/tests/partition/BUILD.bazel @@ -25,6 +25,7 @@ go_test( "//pkg/sessionctx", "//pkg/sessionctx/variable", "//pkg/sessiontxn", + "//pkg/store/gcworker", "//pkg/store/mockstore", "//pkg/table", "//pkg/table/tables", diff --git a/pkg/ddl/tests/partition/db_partition_test.go b/pkg/ddl/tests/partition/db_partition_test.go index 197c7667ec7fb..d8703d2c82844 100644 --- a/pkg/ddl/tests/partition/db_partition_test.go +++ b/pkg/ddl/tests/partition/db_partition_test.go @@ -1401,16 +1401,41 @@ func TestGlobalIndexInsertInDropPartition(t *testing.T) { partition p2 values less than (20), partition p3 values less than (30) );`) +<<<<<<< HEAD tk.MustExec("alter table test_global add unique index idx_b (b);") tk.MustExec("insert into test_global values (1, 1, 1), (8, 8, 8), (11, 11, 11), (12, 12, 12);") hook := &callback.TestDDLCallback{Do: dom} hook.OnJobRunBeforeExported = func(job *model.Job) { +======= + tk.MustExec("alter table test_global add unique index idx_b (b) global") + tk.MustExec("insert into test_global values (1, 1, 1), (2, 2, 2), (11, 11, 11), (12, 12, 12)") + + doneMap := make(map[model.SchemaState]struct{}) + testfailpoint.EnableCall(t, "github.com/pingcap/tidb/pkg/ddl/onJobRunBefore", func(job *model.Job) { +>>>>>>> b6025b97877 (*: Reorg partition fix delete ranges and handling non-clustered tables with concurrent DML (#57114)) assert.Equal(t, model.ActionDropTablePartition, job.Type) - if job.SchemaState == model.StateDeleteOnly { - tk2 := testkit.NewTestKit(t, store) - tk2.MustExec("use test") - tk2.MustExec("insert into test_global values (9, 9, 9)") + if _, ok := doneMap[job.SchemaState]; ok { + return + } + doneMap[job.SchemaState] = struct{}{} + tk2 := testkit.NewTestKit(t, store) + tk2.MustExec("use test") + switch job.SchemaState { + case model.StatePublic: + tk2.MustExec("insert into test_global values (3, 3, 3)") + tk2.MustExec("insert into test_global values (13, 13, 13)") + case model.StateWriteOnly: + tk2.MustContainErrMsg("insert into test_global values (4, 4, 4)", "[table:1526]Table has no partition for value matching a partition being dropped, 'p1'") + tk2.MustExec("insert into test_global values (14, 14, 14)") + case model.StateDeleteOnly: + tk2.MustExec("insert into test_global values (5, 5, 5)") + tk2.MustExec("insert into test_global values (15, 15, 15)") + case model.StateDeleteReorganization: + tk2.MustExec("insert into test_global values (6, 6, 6)") + tk2.MustExec("insert into test_global values (16, 16, 16)") + default: + require.Fail(t, "invalid schema state '%s'", job.SchemaState.String()) } } dom.DDL().SetHook(hook) @@ -1420,7 +1445,7 @@ func TestGlobalIndexInsertInDropPartition(t *testing.T) { tk1.MustExec("alter table test_global drop partition p1") tk.MustExec("analyze table test_global") - tk.MustQuery("select * from test_global use index(idx_b) order by a").Check(testkit.Rows("9 9 9", "11 11 11", "12 12 12")) + tk.MustQuery("select * from test_global use index(idx_b) order by a").Check(testkit.Rows("5 5 5", "6 6 6", "11 11 11", "12 12 12", "13 13 13", "14 14 14", "15 15 15", "16 16 16")) } func TestUpdateGlobalIndex(t *testing.T) { @@ -3608,10 +3633,10 @@ func TestRemovePartitioningAutoIDs(t *testing.T) { tk3.MustExec(`COMMIT`) tk3.MustQuery(`select _tidb_rowid, a, b from t`).Sort().Check(testkit.Rows( "13 11 11", "14 2 2", "15 12 12", "17 16 18", - "19 18 4", "21 20 5", "23 22 6", "25 24 7", "30 29 9")) + "19 18 4", "21 20 5", "23 22 6", "25 24 7", "29 28 9")) tk2.MustQuery(`select _tidb_rowid, a, b from t`).Sort().Check(testkit.Rows( "13 11 11", "14 2 2", "15 12 12", "17 16 18", - "19 18 4", "23 22 6", "27 26 8", "32 31 10")) + "19 18 4", "23 22 6", "27 26 8", "31 30 10")) waitFor(4, "t", "write reorganization") tk3.MustExec(`BEGIN`) @@ -3621,28 +3646,73 @@ func TestRemovePartitioningAutoIDs(t *testing.T) { tk3.MustExec(`insert into t values (null, 23)`) tk2.MustExec(`COMMIT`) +<<<<<<< HEAD /* waitFor(4, "t", "delete reorganization") tk2.MustExec(`BEGIN`) tk2.MustExec(`insert into t values (null, 24)`) +======= + waitFor(4, "t", "delete reorganization") + tk2.MustExec(`BEGIN`) + tk2.MustExec(`insert into t values (null, 24)`) +>>>>>>> b6025b97877 (*: Reorg partition fix delete ranges and handling non-clustered tables with concurrent DML (#57114)) - tk3.MustExec(`insert into t values (null, 25)`) - tk2.MustExec(`insert into t values (null, 26)`) - */ + tk3.MustExec(`insert into t values (null, 25)`) + tk2.MustExec(`insert into t values (null, 26)`) tk3.MustExec(`COMMIT`) + tk2.MustQuery(`select _tidb_rowid, a, b from t`).Sort().Check(testkit.Rows( + "27 26 8", + "30012 12 12", + "30013 18 4", + "30014 24 7", + "30015 16 18", + "30016 22 6", + "30017 28 9", + "30018 11 11", + "30019 2 2", + "30020 20 5", + "31 30 10", + "35 34 22", + "39 38 24", + "43 42 26")) tk3.MustQuery(`select _tidb_rowid, a, b from t`).Sort().Check(testkit.Rows( - "13 11 11", "14 2 2", "15 12 12", "17 16 18", - "19 18 4", "21 20 5", "23 22 6", "25 24 7", "27 26 8", "30 29 9", - "32 31 10", "35 34 21", "38 37 22", "41 40 23")) - - //waitFor(4, "t", "public") - //tk2.MustExec(`commit`) - // TODO: Investigate and fix, but it is also related to https://github.com/pingcap/tidb/issues/46904 - require.ErrorContains(t, <-alterChan, "[kv:1062]Duplicate entry '31' for key 't.PRIMARY'") + "27 26 8", + "30012 12 12", + "30013 18 4", + "30014 24 7", + "30015 16 18", + "30016 22 6", + "30017 28 9", + "30018 11 11", + "30019 2 2", + "30020 20 5", + "31 30 10", + "33 32 21", + "35 34 22", + "37 36 23", + "41 40 25")) + + waitFor(4, "t", "public") + tk2.MustExec(`commit`) tk3.MustQuery(`select _tidb_rowid, a, b from t`).Sort().Check(testkit.Rows( - "13 11 11", "14 2 2", "15 12 12", "17 16 18", - "19 18 4", "21 20 5", "23 22 6", "25 24 7", "27 26 8", "30 29 9", - "32 31 10", "35 34 21", "38 37 22", "41 40 23")) + "27 26 8", + "30012 12 12", + "30013 18 4", + "30014 24 7", + "30015 16 18", + "30016 22 6", + "30017 28 9", + "30018 11 11", + "30019 2 2", + "30020 20 5", + "31 30 10", + "33 32 21", + "35 34 22", + "37 36 23", + "39 38 24", + "41 40 25", + "43 42 26")) + require.NoError(t, <-alterChan) } func TestAlterLastIntervalPartition(t *testing.T) { diff --git a/pkg/ddl/tests/partition/multi_domain_test.go b/pkg/ddl/tests/partition/multi_domain_test.go new file mode 100644 index 0000000000000..e435bc7cc8708 --- /dev/null +++ b/pkg/ddl/tests/partition/multi_domain_test.go @@ -0,0 +1,1445 @@ +// Copyright 2024 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package partition + +import ( + "context" + "encoding/hex" + "fmt" + "math" + "testing" + "time" + + "github.com/pingcap/tidb/pkg/domain" + "github.com/pingcap/tidb/pkg/kv" + "github.com/pingcap/tidb/pkg/meta/model" + pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/session" + "github.com/pingcap/tidb/pkg/sessiontxn" + "github.com/pingcap/tidb/pkg/store/gcworker" + "github.com/pingcap/tidb/pkg/tablecodec" + "github.com/pingcap/tidb/pkg/testkit" + "github.com/pingcap/tidb/pkg/testkit/testfailpoint" + "github.com/pingcap/tidb/pkg/util/logutil" + "github.com/stretchr/testify/require" + "go.uber.org/zap" +) + +func TestMultiSchemaReorganizePartitionIssue56819(t *testing.T) { + createSQL := `create table t (a int primary key, b varchar(255), unique index idx_b_global (b) global) partition by range (a) (partition p1 values less than (200), partition pMax values less than (maxvalue))` + initFn := func(tkO *testkit.TestKit) { + tkO.MustExec(`insert into t values (1,1),(2,2)`) + } + alterSQL := `alter table t reorganize partition p1 into (partition p0 values less than (100), partition p1 values less than (200))` + loopFn := func(tkO, tkNO *testkit.TestKit) { + res := tkO.MustQuery(`select schema_state from information_schema.DDL_JOBS where table_name = 't' order by job_id desc limit 1`) + schemaState := res.Rows()[0][0].(string) + switch schemaState { + case model.StateDeleteOnly.String(): + tkNO.MustExec(`insert into t values (4,4)`) + tkNO.MustQuery(`select * from t where b = "4"`).Sort().Check(testkit.Rows("4 4")) + tkO.MustQuery(`select * from t where b = "4"`).Sort().Check(testkit.Rows("4 4")) + } + } + runMultiSchemaTest(t, createSQL, alterSQL, initFn, nil, loopFn) +} + +func TestMultiSchemaDropRangePartition(t *testing.T) { + createSQL := `create table t (a int primary key, b varchar(255), unique key (b) global, unique key (b,a) global, unique key (b,a)) partition by range (a) (partition p0 values less than (100), partition p1 values less than (200))` + initFn := func(tkO *testkit.TestKit) { + tkO.MustExec(`insert into t values (1,1),(2,2),(101,101),(102,102)`) + } + alterSQL := `alter table t drop partition p0` + loopFn := func(tkO, tkNO *testkit.TestKit) { + res := tkO.MustQuery(`select schema_state from information_schema.DDL_JOBS where table_name = 't' order by job_id desc limit 1`) + schemaState := res.Rows()[0][0].(string) + // TODO: Test both static and dynamic partition pruning! + switch schemaState { + case "write only": + // tkNO are unaware of the DDL + // tkO see non-readable/non-writable p0 partition, and should try to read from p1 + // in case there is something written to overlapping p1 + tkO.MustContainErrMsg(`insert into t values (1,1)`, "[table:1526]Table has no partition for value matching a partition being dropped, 'p0'") + tkNO.MustContainErrMsg(`insert into t values (1,1)`, "[kv:1062]Duplicate entry '1' for key 't.") + tkO.MustContainErrMsg(`insert into t values (101,101)`, "[kv:1062]Duplicate entry '101' for key 't.") + tkNO.MustContainErrMsg(`insert into t values (101,101)`, "[kv:1062]Duplicate entry '101' for key 't.") + tkNO.MustQuery(`select * from t`).Sort().Check(testkit.Rows("1 1", "101 101", "102 102", "2 2")) + tkO.MustQuery(`select * from t`).Sort().Check(testkit.Rows("101 101", "102 102")) + case "delete only": + // tkNO see non-readable/non-writable p0 partition, and should try to read from p1 + // in case there is something written to overlapping p1 + // tkO is not aware of p0. + tkO.MustExec(`insert into t values (1,20)`) + tkNO.MustContainErrMsg(`insert into t values (1,20)`, "[table:1526]Table has no partition for value matching a partition being dropped, 'p0'") + tkO.MustContainErrMsg(`insert into t values (101,101)`, "[kv:1062]Duplicate entry '101' for key 't.") + tkNO.MustContainErrMsg(`insert into t values (101,101)`, "[kv:1062]Duplicate entry '101' for key 't.") + tkNO.MustQuery(`select * from t`).Sort().Check(testkit.Rows("1 20", "101 101", "102 102")) + // Original row should not be seen in StateWriteOnly + tkNO.MustQuery(`select * from t partition (p0)`).Sort().Check(testkit.Rows()) + tkNO.MustContainErrMsg(`select * from t partition (pNonExisting)`, "[table:1735]Unknown partition 'pnonexisting' in table 't'") + tkNO.MustQuery(`select * from t partition (p1)`).Sort().Check(testkit.Rows("1 20", "101 101", "102 102")) + tkNO.MustQuery(`select * from t where a < 1000`).Sort().Check(testkit.Rows("1 20", "101 101", "102 102")) + tkNO.MustQuery(`select * from t where a > 0`).Sort().Check(testkit.Rows("1 20", "101 101", "102 102")) + tkNO.MustQuery(`select * from t where a = 1`).Sort().Check(testkit.Rows("1 20")) + tkNO.MustQuery(`select * from t where a = 1 or a = 2 or a = 3`).Sort().Check(testkit.Rows("1 20")) + tkNO.MustQuery(`select * from t where a in (1,2,3)`).Sort().Check(testkit.Rows("1 20")) + tkNO.MustQuery(`select * from t where a < 100`).Sort().Check(testkit.Rows("1 20")) + + tkNO.MustQuery(`select * from t where b = 20`).Sort().Check(testkit.Rows("1 20")) + // TODO: Test update and delete! + // TODO: test key, hash and list partition without default partition :) + tkNO.MustQuery(`show create table t`).Check(testkit.Rows("" + + "t CREATE TABLE `t` (\n" + + " `a` int(11) NOT NULL,\n" + + " `b` varchar(255) DEFAULT NULL,\n" + + " PRIMARY KEY (`a`) /*T![clustered_index] CLUSTERED */,\n" + + " UNIQUE KEY `b` (`b`) /*T![global_index] GLOBAL */,\n" + + " UNIQUE KEY `b_2` (`b`,`a`) /*T![global_index] GLOBAL */,\n" + + " UNIQUE KEY `b_3` (`b`,`a`)\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin\n" + + "PARTITION BY RANGE (`a`)\n" + + "(PARTITION `p0` VALUES LESS THAN (100),\n" + + " PARTITION `p1` VALUES LESS THAN (200))")) + tkO.MustQuery(`show create table t`).Check(testkit.Rows("" + + "t CREATE TABLE `t` (\n" + + " `a` int(11) NOT NULL,\n" + + " `b` varchar(255) DEFAULT NULL,\n" + + " PRIMARY KEY (`a`) /*T![clustered_index] CLUSTERED */,\n" + + " UNIQUE KEY `b` (`b`) /*T![global_index] GLOBAL */,\n" + + " UNIQUE KEY `b_2` (`b`,`a`) /*T![global_index] GLOBAL */,\n" + + " UNIQUE KEY `b_3` (`b`,`a`)\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin\n" + + "PARTITION BY RANGE (`a`)\n" + + "(PARTITION `p1` VALUES LESS THAN (200))")) + case "delete reorganization": + // just to not fail :) + case "none": + // just to not fail :) + default: + require.Failf(t, "unhandled schema state '%s'", schemaState) + } + } + runMultiSchemaTest(t, createSQL, alterSQL, initFn, nil, loopFn) +} + +func TestMultiSchemaDropListDefaultPartition(t *testing.T) { + createSQL := `create table t (a int primary key, b varchar(255), unique key (b) global, unique key (b,a) global, unique key (b,a)) partition by list (a) (partition p0 values in (1,2,3), partition p1 values in (100,101,102,DEFAULT))` + initFn := func(tkO *testkit.TestKit) { + tkO.MustExec(`insert into t values (1,1),(2,2),(101,101),(102,102)`) + } + alterSQL := `alter table t drop partition p0` + loopFn := func(tkO, tkNO *testkit.TestKit) { + res := tkO.MustQuery(`select schema_state from information_schema.DDL_JOBS where table_name = 't' order by job_id desc limit 1`) + schemaState := res.Rows()[0][0].(string) + // TODO: Test both static and dynamic partition pruning! + switch schemaState { + case "write only": + // tkNO are unaware of the DDL + // tkO see non-readable/non-writable p0 partition, and should try to read from p1 + // in case there is something written to overlapping p1 + tkO.MustContainErrMsg(`insert into t values (1,1)`, "[table:1526]Table has no partition for value matching a partition being dropped, 'p0'") + tkNO.MustContainErrMsg(`insert into t values (1,1)`, "[kv:1062]Duplicate entry '1' for key 't.") + tkO.MustContainErrMsg(`insert into t values (101,101)`, "[kv:1062]Duplicate entry '101' for key 't.") + tkNO.MustContainErrMsg(`insert into t values (101,101)`, "[kv:1062]Duplicate entry '101' for key 't.") + tkNO.MustQuery(`select * from t`).Sort().Check(testkit.Rows("1 1", "101 101", "102 102", "2 2")) + tkO.MustQuery(`select * from t`).Sort().Check(testkit.Rows("101 101", "102 102")) + case "delete only": + // tkNO see non-readable/non-writable p0 partition, and should try to read from p1 + // in case there is something written to overlapping p1 + // tkO is not aware of p0. + tkO.MustExec(`insert into t values (1,20)`) + tkNO.MustContainErrMsg(`insert into t values (1,20)`, "[table:1526]Table has no partition for value matching a partition being dropped, 'p0'") + tkO.MustContainErrMsg(`insert into t values (101,101)`, "[kv:1062]Duplicate entry '101' for key 't.") + tkNO.MustContainErrMsg(`insert into t values (101,101)`, "[kv:1062]Duplicate entry '101' for key 't.") + tkNO.MustQuery(`select * from t`).Sort().Check(testkit.Rows("1 20", "101 101", "102 102")) + // Original row should not be seen in StateWriteOnly + tkNO.MustQuery(`select * from t partition (p0)`).Sort().Check(testkit.Rows()) + tkNO.MustContainErrMsg(`select * from t partition (pNonExisting)`, "[table:1735]Unknown partition 'pnonexisting' in table 't'") + tkNO.MustQuery(`select * from t partition (p1)`).Sort().Check(testkit.Rows("1 20", "101 101", "102 102")) + tkNO.MustQuery(`select * from t where a < 1000`).Sort().Check(testkit.Rows("1 20", "101 101", "102 102")) + tkNO.MustQuery(`select * from t where a > 0`).Sort().Check(testkit.Rows("1 20", "101 101", "102 102")) + tkNO.MustQuery(`select * from t where a = 1`).Sort().Check(testkit.Rows("1 20")) + tkNO.MustQuery(`select * from t where a = 1 or a = 2 or a = 3`).Sort().Check(testkit.Rows("1 20")) + tkNO.MustQuery(`select * from t where a in (1,2,3)`).Sort().Check(testkit.Rows("1 20")) + tkNO.MustQuery(`select * from t where a < 100`).Sort().Check(testkit.Rows("1 20")) + + tkNO.MustQuery(`select * from t where b = 20`).Sort().Check(testkit.Rows("1 20")) + // TODO: Test update and delete! + // TODO: test key, hash and list partition without default partition :) + // Should we see the partition or not?!? + tkNO.MustQuery(`show create table t`).Check(testkit.Rows("" + + "t CREATE TABLE `t` (\n" + + " `a` int(11) NOT NULL,\n" + + " `b` varchar(255) DEFAULT NULL,\n" + + " PRIMARY KEY (`a`) /*T![clustered_index] CLUSTERED */,\n" + + " UNIQUE KEY `b` (`b`) /*T![global_index] GLOBAL */,\n" + + " UNIQUE KEY `b_2` (`b`,`a`) /*T![global_index] GLOBAL */,\n" + + " UNIQUE KEY `b_3` (`b`,`a`)\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin\n" + + "PARTITION BY LIST (`a`)\n" + + "(PARTITION `p0` VALUES IN (1,2,3),\n" + + " PARTITION `p1` VALUES IN (100,101,102,DEFAULT))")) + tkO.MustQuery(`show create table t`).Check(testkit.Rows("" + + "t CREATE TABLE `t` (\n" + + " `a` int(11) NOT NULL,\n" + + " `b` varchar(255) DEFAULT NULL,\n" + + " PRIMARY KEY (`a`) /*T![clustered_index] CLUSTERED */,\n" + + " UNIQUE KEY `b` (`b`) /*T![global_index] GLOBAL */,\n" + + " UNIQUE KEY `b_2` (`b`,`a`) /*T![global_index] GLOBAL */,\n" + + " UNIQUE KEY `b_3` (`b`,`a`)\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin\n" + + "PARTITION BY LIST (`a`)\n" + + "(PARTITION `p1` VALUES IN (100,101,102,DEFAULT))")) + case "delete reorganization": + // just to not fail :) + case "none": + // just to not fail :) + default: + require.Failf(t, "unhandled schema state '%s'", schemaState) + } + } + runMultiSchemaTest(t, createSQL, alterSQL, initFn, nil, loopFn) +} + +func TestMultiSchemaDropListColumnsDefaultPartition(t *testing.T) { + createSQL := `create table t (a int, b varchar(255), c varchar (255), primary key (a,b), unique key (a) global, unique key (b,a) global, unique key (c) global, unique key (b,a)) partition by list columns (a,b) (partition p0 values in ((1,"1"),(2,"2"),(3,"3")), partition p1 values in ((100,"100"),(101,"101"),(102,"102"),DEFAULT))` + initFn := func(tkO *testkit.TestKit) { + tkO.MustExec(`insert into t values (1,1,1),(2,2,2),(101,101,101),(102,102,102)`) + } + alterSQL := `alter table t drop partition p0` + loopFn := func(tkO, tkNO *testkit.TestKit) { + res := tkO.MustQuery(`select schema_state from information_schema.DDL_JOBS where table_name = 't' order by job_id desc limit 1`) + schemaState := res.Rows()[0][0].(string) + // TODO: Test both static and dynamic partition pruning! + switch schemaState { + case "write only": + // tkNO are unaware of the DDL + // tkO see non-readable/non-writable p0 partition, and should try to read from p1 + // in case there is something written to overlapping p1 + tkO.MustContainErrMsg(`insert into t values (1,1,1)`, "[table:1526]Table has no partition for value matching a partition being dropped, 'p0'") + tkNO.MustContainErrMsg(`insert into t values (1,1,1)`, "[kv:1062]Duplicate entry '1' for key 't.a_2'") + tkO.MustContainErrMsg(`insert into t values (101,101,101)`, "[kv:1062]Duplicate entry '101' for key 't.a_2'") + tkNO.MustContainErrMsg(`insert into t values (101,101,101)`, "[kv:1062]Duplicate entry '101' for key 't.a_2'") + tkNO.MustQuery(`select * from t`).Sort().Check(testkit.Rows("1 1 1", "101 101 101", "102 102 102", "2 2 2")) + tkO.MustQuery(`select * from t`).Sort().Check(testkit.Rows("101 101 101", "102 102 102")) + case "delete only": + // tkNO see non-readable/non-writable p0 partition, and should try to read from p1 + // in case there is something written to overlapping p1 + // tkO is not aware of p0. + tkO.MustExec(`insert into t values (3,3,3)`) + tkO.MustContainErrMsg(`insert into t values (1,1,2)`, "[kv:1062]Duplicate entry '1' for key 't.a_2") + tkNO.MustContainErrMsg(`insert into t values (3,3,3)`, "[table:1526]Table has no partition for value matching a partition being dropped, 'p0'") + tkO.MustContainErrMsg(`insert into t values (101,101,101)`, "[kv:1062]Duplicate entry '101' for key 't.a_2'") + tkNO.MustContainErrMsg(`insert into t values (101,101,101)`, "[kv:1062]Duplicate entry '101' for key 't.a_2'") + tkNO.MustQuery(`select * from t`).Sort().Check(testkit.Rows("101 101 101", "102 102 102", "3 3 3")) + tkO.MustQuery(`select * from t`).Sort().Check(testkit.Rows("101 101 101", "102 102 102", "3 3 3")) + // Original row should not be seen in StateWriteOnly + tkNO.MustQuery(`select * from t partition (p0)`).Sort().Check(testkit.Rows()) + tkNO.MustContainErrMsg(`select * from t partition (pNonExisting)`, "[table:1735]Unknown partition 'pnonexisting' in table 't'") + tkNO.MustQuery(`select * from t partition (p1)`).Sort().Check(testkit.Rows("101 101 101", "102 102 102", "3 3 3")) + tkNO.MustQuery(`select * from t where a < 1000`).Sort().Check(testkit.Rows("101 101 101", "102 102 102", "3 3 3")) + tkNO.MustQuery(`select * from t where a > 0`).Sort().Check(testkit.Rows("101 101 101", "102 102 102", "3 3 3")) + tkNO.MustQuery(`select * from t where a = 3`).Sort().Check(testkit.Rows("3 3 3")) + tkNO.MustQuery(`select * from t where a = 1 or a = 2 or a = 3`).Sort().Check(testkit.Rows("3 3 3")) + tkNO.MustQuery(`select * from t where a in (1,2,3) or b in ("1","2")`).Sort().Check(testkit.Rows("3 3 3")) + tkNO.MustQuery(`select * from t where a in (1,2,3)`).Sort().Check(testkit.Rows("3 3 3")) + tkNO.MustQuery(`select * from t where a < 100`).Sort().Check(testkit.Rows("3 3 3")) + + tkNO.MustQuery(`select * from t where c = "2"`).Sort().Check(testkit.Rows("2 2 2")) + tkNO.MustQuery(`select * from t where b = "3"`).Sort().Check(testkit.Rows("3 3 3")) + // TODO: Test update and delete! + // TODO: test key, hash and list partition without default partition :) + // Should we see the partition or not?!? + tkNO.MustQuery(`show create table t`).Check(testkit.Rows("" + + "t CREATE TABLE `t` (\n" + + " `a` int(11) NOT NULL,\n" + + " `b` varchar(255) NOT NULL,\n" + + " `c` varchar(255) DEFAULT NULL,\n" + + " PRIMARY KEY (`a`,`b`) /*T![clustered_index] CLUSTERED */,\n" + + " UNIQUE KEY `a_2` (`a`) /*T![global_index] GLOBAL */,\n" + + " UNIQUE KEY `b` (`b`,`a`) /*T![global_index] GLOBAL */,\n" + + " UNIQUE KEY `c` (`c`) /*T![global_index] GLOBAL */,\n" + + " UNIQUE KEY `b_2` (`b`,`a`)\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin\n" + + "PARTITION BY LIST COLUMNS(`a`,`b`)\n" + + "(PARTITION `p0` VALUES IN ((1,'1'),(2,'2'),(3,'3')),\n" + + " PARTITION `p1` VALUES IN ((100,'100'),(101,'101'),(102,'102'),DEFAULT))")) + tkO.MustQuery(`show create table t`).Check(testkit.Rows("" + + "t CREATE TABLE `t` (\n" + + " `a` int(11) NOT NULL,\n" + + " `b` varchar(255) NOT NULL,\n" + + " `c` varchar(255) DEFAULT NULL,\n" + + " PRIMARY KEY (`a`,`b`) /*T![clustered_index] CLUSTERED */,\n" + + " UNIQUE KEY `a_2` (`a`) /*T![global_index] GLOBAL */,\n" + + " UNIQUE KEY `b` (`b`,`a`) /*T![global_index] GLOBAL */,\n" + + " UNIQUE KEY `c` (`c`) /*T![global_index] GLOBAL */,\n" + + " UNIQUE KEY `b_2` (`b`,`a`)\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin\n" + + "PARTITION BY LIST COLUMNS(`a`,`b`)\n" + + "(PARTITION `p1` VALUES IN ((100,'100'),(101,'101'),(102,'102'),DEFAULT))")) + case "delete reorganization": + // just to not fail :) + case "none": + // just to not fail :) + default: + require.Failf(t, "unhandled schema state '%s'", schemaState) + } + } + runMultiSchemaTest(t, createSQL, alterSQL, initFn, nil, loopFn) +} + +func TestMultiSchemaReorganizePartition(t *testing.T) { + createSQL := `create table t (a int primary key, b varchar(255), unique index idx_b_global (b) global) partition by range (a) (partition p1 values less than (200), partition pMax values less than (maxvalue))` + initFn := func(tkO *testkit.TestKit) { + tkO.MustExec(`insert into t values (1,1),(2,2),(101,101),(102,102),(998,998),(999,999)`) + } + alterSQL := `alter table t reorganize partition p1 into (partition p0 values less than (100), partition p1 values less than (200))` + + testID := 4 + loopFn := func(tkO, tkNO *testkit.TestKit) { + res := tkO.MustQuery(`select schema_state from information_schema.DDL_JOBS where table_name = 't' order by job_id desc limit 1`) + schemaState := res.Rows()[0][0].(string) + dbgStr := ` /* ` + schemaState + ` */` + + // Check for every state + tkO.MustContainErrMsg(`insert into t values (1,2)`+dbgStr, "[kv:1062]Duplicate entry") + tkNO.MustContainErrMsg(`insert into t values (1,2)`+dbgStr, "[kv:1062]Duplicate entry") + tkO.MustContainErrMsg(`insert into t values (101,101)`+dbgStr, "[kv:1062]Duplicate entry") + tkNO.MustContainErrMsg(`insert into t values (101,101)`+dbgStr, "[kv:1062]Duplicate entry") + tkO.MustContainErrMsg(`insert into t values (999,999)`+dbgStr, "[kv:1062]Duplicate entry '999' for key 't.") + tkNO.MustContainErrMsg(`insert into t values (999,999)`+dbgStr, "[kv:1062]Duplicate entry '999' for key 't.") + tkNO.MustQuery(`select * from t where a = 1` + dbgStr).Sort().Check(testkit.Rows("1 1")) + tkNO.MustQuery(`select * from t where a = 1 or a = 2 or a = 3` + dbgStr).Sort().Check(testkit.Rows("1 1", "2 2")) + tkNO.MustQuery(`select * from t where a in (1,2,3)` + dbgStr).Sort().Check(testkit.Rows("1 1", "2 2")) + tkNO.MustQuery(`select * from t where b = "2"` + dbgStr).Sort().Check(testkit.Rows("2 2")) + + highID := testID + 980 + tkO.MustExec(fmt.Sprintf(`insert into t values (%d,%d)`+dbgStr, highID, highID)) + res = tkNO.MustQuery(fmt.Sprintf(`select * from t where b = "%d"`+dbgStr, highID)) + if len(res.Rows()) != 1 { + tkNO.MustQuery(fmt.Sprintf(`explain select * from t where b = "%d"`+dbgStr, highID)).Check(testkit.Rows(fmt.Sprintf("%d %d", highID, highID))) + } + res.Check(testkit.Rows(fmt.Sprintf("%d %d", highID, highID))) + + highID++ + tkNO.MustExec(fmt.Sprintf(`insert into t values (%d,%d)`+dbgStr, highID, highID)) + tkO.MustQuery(fmt.Sprintf(`select * from t where b = "%d"`+dbgStr, highID)).Check(testkit.Rows(fmt.Sprintf("%d %d", highID, highID))) + + testID++ + tkO.MustExec(fmt.Sprintf(`insert into t values (%d,%d)`+dbgStr, testID, testID)) + tkNO.MustQuery(fmt.Sprintf(`select * from t where b = "%d"`+dbgStr, testID)).Check(testkit.Rows(fmt.Sprintf("%d %d", testID, testID))) + + logutil.BgLogger().Info("inserting rows", zap.Int("testID", testID), zap.String("state", schemaState)) + + testID++ + tkNO.MustExec(fmt.Sprintf(`insert into t values (%d,%d)`+dbgStr, testID, testID)) + tkO.MustQuery(fmt.Sprintf(`select * from t where b = "%d"`+dbgStr, testID)).Check(testkit.Rows(fmt.Sprintf("%d %d", testID, testID))) + + // Test for Index, specially between WriteOnly and DeleteOnly, but better to test all states. + // if tkNO (DeleteOnly) updates a row, the new index should be deleted, but not inserted. + // It will be inserted by backfill in WriteReorganize. + // If not deleted, then there would be an orphan entry in the index! + tkO.MustExec(fmt.Sprintf(`update t set b = %d where a = %d`+dbgStr, testID+100, testID)) + tkNO.MustQuery(fmt.Sprintf(`select a, b from t where a = %d`+dbgStr, testID)).Check(testkit.Rows(fmt.Sprintf("%d %d", testID, testID+100))) + tkNO.MustQuery(fmt.Sprintf(`select a, b from t where b = "%d"`+dbgStr, testID+100)).Check(testkit.Rows(fmt.Sprintf("%d %d", testID, testID+100))) + tkNO.MustExec(fmt.Sprintf(`update t set b = %d where a = %d`+dbgStr, testID+99, testID-1)) + tkO.MustQuery(fmt.Sprintf(`select a, b from t where a = %d`+dbgStr, testID-1)).Check(testkit.Rows(fmt.Sprintf("%d %d", testID-1, testID+99))) + tkO.MustQuery(fmt.Sprintf(`select a, b from t where b = "%d"`+dbgStr, testID+99)).Check(testkit.Rows(fmt.Sprintf("%d %d", testID-1, testID+99))) + tkNO.MustExec(fmt.Sprintf(`update t set b = %d where a = %d`+dbgStr, testID, testID)) + tkO.MustExec(fmt.Sprintf(`update t set b = %d where a = %d`+dbgStr, testID-1, testID-1)) + + switch schemaState { + case model.StateDeleteOnly.String(): + // tkNO sees original table/partitions as before the DDL stated + // tkO uses the original table/partitions, but should also delete from the newly created + // Global Index, to replace the existing one. + + tkNO.MustQuery(`select * from t`).Sort().Check(testkit.Rows("1 1", "101 101", "102 102", "2 2", "5 5", "6 6", "984 984", "985 985", "998 998", "999 999")) + tkNO.MustQuery(`select * from t where a < 1000`).Sort().Check(testkit.Rows("1 1", "101 101", "102 102", "2 2", "5 5", "6 6", "984 984", "985 985", "998 998", "999 999")) + tkNO.MustQuery(`select * from t where a > 0`).Sort().Check(testkit.Rows("1 1", "101 101", "102 102", "2 2", "5 5", "6 6", "984 984", "985 985", "998 998", "999 999")) + tkNO.MustQuery(`select * from t where a < 100`).Sort().Check(testkit.Rows("1 1", "2 2", "5 5", "6 6")) + case model.StateWriteOnly.String(): + // Both tkO and tkNO uses the original table/partitions, + // but tkO should also update the newly created + // Global Index, and tkNO should only delete from it. + case model.StateWriteReorganization.String(): + // Both tkO and tkNO uses the original table/partitions, + // and should also update the newly created Global Index. + case model.StateDeleteReorganization.String(): + // Both tkO now sees the new partitions, and should use the new Global Index, + // plus double write to the old one. + // tkNO uses the original table/partitions, + // and should also update the newly created Global Index. + // TODO: Test update and delete! + // TODO: test key, hash and list partition without default partition :) + tkNO.MustQuery(`show create table t`).Check(testkit.Rows("" + + "t CREATE TABLE `t` (\n" + + " `a` int(11) NOT NULL,\n" + + " `b` varchar(255) DEFAULT NULL,\n" + + " PRIMARY KEY (`a`) /*T![clustered_index] CLUSTERED */,\n" + + " UNIQUE KEY `idx_b_global` (`b`) /*T![global_index] GLOBAL */\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin\n" + + "PARTITION BY RANGE (`a`)\n" + + "(PARTITION `p1` VALUES LESS THAN (200),\n" + + " PARTITION `pMax` VALUES LESS THAN (MAXVALUE))")) + tkO.MustQuery(`show create table t`).Check(testkit.Rows("" + + "t CREATE TABLE `t` (\n" + + " `a` int(11) NOT NULL,\n" + + " `b` varchar(255) DEFAULT NULL,\n" + + " PRIMARY KEY (`a`) /*T![clustered_index] CLUSTERED */,\n" + + " UNIQUE KEY `idx_b_global` (`b`) /*T![global_index] GLOBAL */\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin\n" + + "PARTITION BY RANGE (`a`)\n" + + "(PARTITION `p0` VALUES LESS THAN (100),\n" + + " PARTITION `p1` VALUES LESS THAN (200),\n" + + " PARTITION `pMax` VALUES LESS THAN (MAXVALUE))")) + case model.StatePublic.String(): + // not tested, both tkO and tkNO sees same partitions + case model.StateNone.String(): + // not tested, both tkO and tkNO sees same partitions + default: + require.Failf(t, "unhandled schema state", "State '%s'", schemaState) + } + } + postFn := func(tkO *testkit.TestKit, store kv.Storage) { + tkO.MustQuery(`select * from t where b = 5`).Sort().Check(testkit.Rows("5 5")) + tkO.MustQuery(`select * from t where b = "5"`).Sort().Check(testkit.Rows("5 5")) + tkO.MustExec(`admin check table t`) + tkO.MustQuery(`select * from t`).Sort().Check(testkit.Rows("1 1", "10 10", "101 101", "102 102", "11 11", "12 12", "13 13", "14 14", "15 15", "16 16", "2 2", "5 5", "6 6", "7 7", "8 8", "9 9", "984 984", "985 985", "986 986", "987 987", "988 988", "989 989", "990 990", "991 991", "992 992", "993 993", "994 994", "995 995", "998 998", "999 999")) + tkO.MustQuery(`show create table t`).Check(testkit.Rows("" + + "t CREATE TABLE `t` (\n" + + " `a` int(11) NOT NULL,\n" + + " `b` varchar(255) DEFAULT NULL,\n" + + " PRIMARY KEY (`a`) /*T![clustered_index] CLUSTERED */,\n" + + " UNIQUE KEY `idx_b_global` (`b`) /*T![global_index] GLOBAL */\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin\n" + + "PARTITION BY RANGE (`a`)\n" + + "(PARTITION `p0` VALUES LESS THAN (100),\n" + + " PARTITION `p1` VALUES LESS THAN (200),\n" + + " PARTITION `pMax` VALUES LESS THAN (MAXVALUE))")) + } + runMultiSchemaTest(t, createSQL, alterSQL, initFn, postFn, loopFn) +} + +// Also tests for conversions of unique indexes +// 1 unique non-global - to become global +// 2 unique global - to become non-global +// 3 unique non-global - to stay non-global +// 4 unique global - to stay global +func TestMultiSchemaPartitionByGlobalIndex(t *testing.T) { + createSQL := `create table t (a int primary key nonclustered global, b varchar(255), c bigint, unique index idx_b_global (b) global, unique key idx_ba (b,a), unique key idx_ab (a,b) global, unique key idx_c_global (c) global, unique key idx_cab (c,a,b)) partition by key (a,b) partitions 3` + initFn := func(tkO *testkit.TestKit) { + tkO.MustExec(`insert into t values (1,1,1),(2,2,2),(101,101,101),(102,102,102)`) + } + alterSQL := `alter table t partition by key (b,a) partitions 5 update indexes (idx_ba global, idx_ab local)` + doneStateWriteReorganize := false + loopFn := func(tkO, tkNO *testkit.TestKit) { + res := tkO.MustQuery(`select schema_state from information_schema.DDL_JOBS where table_name = 't' order by job_id desc limit 1`) + schemaState := res.Rows()[0][0].(string) + switch schemaState { + case model.StateDeleteOnly.String(): + // tkNO sees original table/partitions as before the DDL stated + // tkO uses the original table/partitions, but should also delete from the newly created + // Global Index, to replace the existing one. + tkO.MustContainErrMsg(`insert into t values (1,2,3)`, "[kv:1062]Duplicate entry '2' for key 't.idx_b") + tkNO.MustContainErrMsg(`insert into t values (1,2,3)`, "[kv:1062]Duplicate entry '2' for key 't.idx_b") + tkO.MustContainErrMsg(`insert into t values (101,101,101)`, "[kv:1062]Duplicate entry '101' for key 't.idx_b") + tkNO.MustContainErrMsg(`insert into t values (101,101,101)`, "[kv:1062]Duplicate entry '101' for key 't.idx_b") + tkNO.MustQuery(`select * from t`).Sort().Check(testkit.Rows("1 1 1", "101 101 101", "102 102 102", "2 2 2")) + tkNO.MustQuery(`select * from t where a < 1000`).Sort().Check(testkit.Rows("1 1 1", "101 101 101", "102 102 102", "2 2 2")) + tkNO.MustQuery(`select * from t where a > 0`).Sort().Check(testkit.Rows("1 1 1", "101 101 101", "102 102 102", "2 2 2")) + tkNO.MustQuery(`select * from t where a = 1`).Sort().Check(testkit.Rows("1 1 1")) + tkNO.MustQuery(`select * from t where a = 1 or a = 2 or a = 3`).Sort().Check(testkit.Rows("1 1 1", "2 2 2")) + tkNO.MustQuery(`select * from t where a in (1,2,3)`).Sort().Check(testkit.Rows("1 1 1", "2 2 2")) + tkNO.MustQuery(`select * from t where a < 100`).Sort().Check(testkit.Rows("1 1 1", "2 2 2")) + + tkNO.MustQuery(`select * from t where b = 2`).Sort().Check(testkit.Rows("2 2 2")) + tkO.MustExec(`insert into t values (3,3,3)`) + tkNO.MustExec(`insert into t values (4,4,4)`) + tkNO.MustQuery(`select * from t where a = 3`).Sort().Check(testkit.Rows("3 3 3")) + tkO.MustQuery(`select * from t where a = 4`).Sort().Check(testkit.Rows("4 4 4")) + case model.StateWriteOnly.String(): + // Both tkO and tkNO uses the original table/partitions, + // but tkO should also update the newly created + // Global Index, and tkNO should only delete from it. + tkO.MustContainErrMsg(`insert into t values (1,1,1)`, "[kv:1062]Duplicate entry '1' for key 't.idx_b") + tkNO.MustContainErrMsg(`insert into t values (1,1,1)`, "[kv:1062]Duplicate entry '1' for key 't.idx_b") + tkO.MustContainErrMsg(`insert into t values (101,101,101)`, "[kv:1062]Duplicate entry '101' for key 't.idx_b") + tkNO.MustContainErrMsg(`insert into t values (101,101,101)`, "[kv:1062]Duplicate entry '101' for key 't.idx_b") + tkNO.MustQuery(`select * from t`).Sort().Check(testkit.Rows("1 1 1", "101 101 101", "102 102 102", "2 2 2", "3 3 3", "4 4 4")) + tkO.MustQuery(`select * from t`).Sort().Check(testkit.Rows("1 1 1", "101 101 101", "102 102 102", "2 2 2", "3 3 3", "4 4 4")) + logutil.BgLogger().Info("insert into t values (5,5,5)") + tkO.MustExec(`insert into t values (5,5,5)`) + tkNO.MustExec(`insert into t values (6,6,6)`) + tkNO.MustQuery(`select * from t where a = 5`).Sort().Check(testkit.Rows("5 5 5")) + tkO.MustQuery(`select * from t where a = 6`).Sort().Check(testkit.Rows("6 6 6")) + case model.StateWriteReorganization.String(): + // It will go through StateWriteReorg more than once. + if doneStateWriteReorganize { + break + } + doneStateWriteReorganize = true + // Both tkO and tkNO uses the original table/partitions, + // and should also update the newly created Global Index. + tkO.MustExec(`insert into t values (7,7,7)`) + tkNO.MustExec(`insert into t values (8,8,8)`) + tkNO.MustQuery(`select * from t where b = 7`).Check(testkit.Rows("7 7 7")) + tkO.MustQuery(`select * from t where b = 8`).Check(testkit.Rows("8 8 8")) + case model.StateDeleteReorganization.String(): + // Both tkO now sees the new partitions, and should use the new Global Index, + // plus double write to the old one. + // tkNO uses the original table/partitions, + // and should also update the newly created Global Index. + tkO.MustExec(`insert into t values (9,9,9)`) + tkNO.MustExec(`insert into t values (10,10,10)`) + tkNO.MustQuery(`select * from t where b = 9`).Check(testkit.Rows("9 9 9")) + tkO.MustQuery(`select * from t where b = 10`).Check(testkit.Rows("10 10 10")) + // TODO: Test update and delete! + // TODO: test key, hash and list partition without default partition :) + tkNO.MustQuery(`show create table t`).Check(testkit.Rows("" + + "t CREATE TABLE `t` (\n" + + " `a` int(11) NOT NULL,\n" + + " `b` varchar(255) DEFAULT NULL,\n" + + " `c` bigint(20) DEFAULT NULL,\n" + + " UNIQUE KEY `idx_b_global` (`b`) /*T![global_index] GLOBAL */,\n" + + " UNIQUE KEY `idx_ba` (`b`,`a`),\n" + + " UNIQUE KEY `idx_ab` (`a`,`b`) /*T![global_index] GLOBAL */,\n" + + " UNIQUE KEY `idx_c_global` (`c`) /*T![global_index] GLOBAL */,\n" + + " UNIQUE KEY `idx_cab` (`c`,`a`,`b`),\n" + + " PRIMARY KEY (`a`) /*T![clustered_index] NONCLUSTERED */ /*T![global_index] GLOBAL */\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin\n" + + "PARTITION BY KEY (`a`,`b`) PARTITIONS 3")) + tkO.MustQuery(`show create table t`).Check(testkit.Rows("" + + "t CREATE TABLE `t` (\n" + + " `a` int(11) NOT NULL,\n" + + " `b` varchar(255) DEFAULT NULL,\n" + + " `c` bigint(20) DEFAULT NULL,\n" + + " UNIQUE KEY `idx_cab` (`c`,`a`,`b`),\n" + + " UNIQUE KEY `idx_b_global` (`b`) /*T![global_index] GLOBAL */,\n" + + " UNIQUE KEY `idx_ba` (`b`,`a`) /*T![global_index] GLOBAL */,\n" + + " UNIQUE KEY `idx_ab` (`a`,`b`),\n" + + " UNIQUE KEY `idx_c_global` (`c`) /*T![global_index] GLOBAL */,\n" + + " PRIMARY KEY (`a`) /*T![clustered_index] NONCLUSTERED */ /*T![global_index] GLOBAL */\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin\n" + + "PARTITION BY KEY (`b`,`a`) PARTITIONS 5")) + case model.StatePublic.String(): + tkO.MustExec(`insert into t values (11,11,11)`) + tkNO.MustExec(`insert into t values (12,12,12)`) + case model.StateNone.String(): + tkO.MustExec(`insert into t values (13,13,13)`) + tkNO.MustExec(`insert into t values (14,14,14)`) + tkO.MustQuery(`select * from t where b = 11`).Check(testkit.Rows("11 11 11")) + default: + require.Failf(t, "unhandled schema state '%s'", schemaState) + } + } + postFn := func(tkO *testkit.TestKit, _ kv.Storage) { + tkO.MustQuery(`select * from t where b = 5`).Check(testkit.Rows("5 5 5")) + tkO.MustExec(`admin check table t`) + tkO.MustQuery(`select * from t`).Sort().Check(testkit.Rows(""+ + "1 1 1", + "10 10 10", + "101 101 101", + "102 102 102", + "11 11 11", + "12 12 12", + "13 13 13", + "14 14 14", + "2 2 2", + "3 3 3", + "4 4 4", + "5 5 5", + "6 6 6", + "7 7 7", + "8 8 8", + "9 9 9")) + } + runMultiSchemaTest(t, createSQL, alterSQL, initFn, postFn, loopFn) +} + +// TestMultiSchemaModifyColumn to show behavior when changing a column +func TestMultiSchemaModifyColumn(t *testing.T) { + createSQL := `create table t (a int primary key, b varchar(255), unique key uk_b (b))` + initFn := func(tkO *testkit.TestKit) { + tkO.MustExec(`insert into t values (1,1),(2,2),(3,3),(4,4),(5,5),(6,6),(7,7),(8,8),(9,9)`) + } + alterSQL := `alter table t modify column b int unsigned not null` + loopFn := func(tkO, tkNO *testkit.TestKit) { + res := tkO.MustQuery(`select schema_state from information_schema.DDL_JOBS where table_name = 't' order by job_id desc limit 1`) + schemaState := res.Rows()[0][0].(string) + switch schemaState { + case model.StateDeleteOnly.String(): + // we are only interested in StateWriteReorganization + case model.StateWriteOnly.String(): + // we are only interested in StateDeleteReorganization->StatePublic + case model.StateWriteReorganization.String(): + case model.StatePublic.String(): + // tkNO sees varchar column and tkO sees int column + tkO.MustQuery(`show create table t`).Check(testkit.Rows("" + + "t CREATE TABLE `t` (\n" + + " `a` int(11) NOT NULL,\n" + + " `b` int(10) unsigned NOT NULL,\n" + + " PRIMARY KEY (`a`) /*T![clustered_index] CLUSTERED */,\n" + + " UNIQUE KEY `uk_b` (`b`)\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin")) + tkNO.MustQuery(`show create table t`).Check(testkit.Rows("" + + "t CREATE TABLE `t` (\n" + + " `a` int(11) NOT NULL,\n" + + " `b` varchar(255) DEFAULT NULL,\n" + + " PRIMARY KEY (`a`) /*T![clustered_index] CLUSTERED */,\n" + + " UNIQUE KEY `uk_b` (`b`)\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin")) + + tkO.MustExec(`insert into t values (10, " 09.60 ")`) + // No warning!? Same in MySQL... + tkNO.MustQuery(`show warnings`).Check(testkit.Rows()) + tkNO.MustContainErrMsg(`insert into t values (11, "09.60")`, "[kv:1062]Duplicate entry '10' for key 't._Idx$_uk_b_0'") + tkO.MustQuery(`select * from t where a = 10`).Check(testkit.Rows("10 10")) + // ?!? + tkNO.MustQuery(`select * from t where a = 10`).Check(testkit.Rows("10 ")) + // If the original b was defined as 'NOT NULL', then it would give an error: + // [table:1364]Field 'b' doesn't have a default value + + tkNO.MustExec(`insert into t values (11, " 011.50 ")`) + tkNO.MustQuery(`show warnings`).Check(testkit.Rows()) + // Anomaly, the different sessions sees different data. + // So it should be acceptable for partitioning DDLs as well. + // It may be possible to check that writes from StateWriteOnly convert 1:1 + // to the new type, and block writes otherwise. But then it would break the first tkO insert above... + tkO.MustQuery(`select * from t where a = 11`).Check(testkit.Rows("11 12")) + tkNO.MustQuery(`select * from t where a = 11`).Check(testkit.Rows("11 011.50 ")) + tblO, err := tkO.Session().GetInfoSchema().TableInfoByName(pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + require.NoError(t, err) + tblNO, err := tkNO.Session().GetInfoSchema().TableInfoByName(pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + require.NoError(t, err) + require.Greater(t, tblO.Columns[1].ID, tblNO.Columns[1].ID) + // This also means that old copies of the columns will be left in the row, until the row is updated or deleted. + // But I guess that is at least documented. + default: + require.Failf(t, "unhandled schema state '%s'", schemaState) + } + } + runMultiSchemaTest(t, createSQL, alterSQL, initFn, nil, loopFn) +} + +// TestMultiSchemaDropUniqueIndex to show behavior when +// dropping a unique index +func TestMultiSchemaDropUniqueIndex(t *testing.T) { + createSQL := `create table t (a int primary key, b varchar(255), unique key uk_b (b))` + initFn := func(tkO *testkit.TestKit) { + tkO.MustExec(`insert into t values (1,1),(2,2),(3,3),(4,4),(5,5),(6,6),(7,7),(8,8),(9,9)`) + } + alterSQL := `alter table t drop index uk_b` + loopFn := func(tkO, tkNO *testkit.TestKit) { + res := tkO.MustQuery(`select schema_state from information_schema.DDL_JOBS where table_name = 't' order by job_id desc limit 1`) + schemaState := res.Rows()[0][0].(string) + switch schemaState { + case "write only": + tkNO.MustQuery(`show create table t`).Check(testkit.Rows("" + + "t CREATE TABLE `t` (\n" + + " `a` int(11) NOT NULL,\n" + + " `b` varchar(255) DEFAULT NULL,\n" + + " PRIMARY KEY (`a`) /*T![clustered_index] CLUSTERED */,\n" + + " UNIQUE KEY `uk_b` (`b`)\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin")) + tkO.MustQuery(`show create table t`).Check(testkit.Rows("" + + "t CREATE TABLE `t` (\n" + + " `a` int(11) NOT NULL,\n" + + " `b` varchar(255) DEFAULT NULL,\n" + + " PRIMARY KEY (`a`) /*T![clustered_index] CLUSTERED */\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin")) + tkO.MustContainErrMsg(`insert into t values (10,1)`, "[kv:1062]Duplicate entry '1' for key 't.uk_b'") + tkNO.MustContainErrMsg(`insert into t values (10,1)`, "[kv:1062]Duplicate entry '1' for key 't.uk_b'") + case "delete only": + // Delete only from the uk_b unique index, cannot have errors + tkO.MustExec(`insert into t values (10,1)`) + tkO.MustExec(`insert into t values (11,11)`) + tkO.MustExec(`delete from t where a = 2`) + // Write only for uk_b, we cannot find anything through the index or read from the index, but still gives duplicate keys on insert/updates + // So we already have two duplicates of b = 1, but only one in the unique index uk_a, so here we cannot insert any. + tkNO.MustContainErrMsg(`insert into t values (12,1)`, "[kv:1062]Duplicate entry '1' for key 't.uk_b'") + tkNO.MustContainErrMsg(`update t set b = 1 where a = 9`, "[kv:1062]Duplicate entry '1' for key 't.uk_b'") + // Deleted from the index! + tkNO.MustExec(`insert into t values (13,2)`) + tkNO.MustContainErrMsg(`insert into t values (14,3)`, "[kv:1062]Duplicate entry '3' for key 't.uk_b'") + // b = 11 never written to the index! + tkNO.MustExec(`insert into t values (15,11)`) + tkNO.MustQuery(`show create table t`).Check(testkit.Rows("" + + "t CREATE TABLE `t` (\n" + + " `a` int(11) NOT NULL,\n" + + " `b` varchar(255) DEFAULT NULL,\n" + + " PRIMARY KEY (`a`) /*T![clustered_index] CLUSTERED */\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin")) + case "delete reorganization": + tkO.MustQuery(`show create table t`).Check(testkit.Rows("" + + "t CREATE TABLE `t` (\n" + + " `a` int(11) NOT NULL,\n" + + " `b` varchar(255) DEFAULT NULL,\n" + + " PRIMARY KEY (`a`) /*T![clustered_index] CLUSTERED */\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin")) + case "none": + tkO.MustQuery(`show create table t`).Check(testkit.Rows("" + + "t CREATE TABLE `t` (\n" + + " `a` int(11) NOT NULL,\n" + + " `b` varchar(255) DEFAULT NULL,\n" + + " PRIMARY KEY (`a`) /*T![clustered_index] CLUSTERED */\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin")) + default: + require.Failf(t, "unhandled schema state '%s'", schemaState) + } + } + runMultiSchemaTest(t, createSQL, alterSQL, initFn, nil, loopFn) +} + +// TODO: Also add test for REMOVE PARTITIONING! +///* +//// TODO: complete this test, so that we test all four changes: +//1 unique non-global - to become global +//2 unique global - to become non-global +//3 unique non-global - to stay non-global +//4 unique global - to stay global +//func TestMultiSchemaPartitionByGlobalIndex(t *testing.T) { +// createSQL := `create table t (a int primary key, b varchar(255), c bigint, unique index idx_b_global (b) global, unique key idx_b (b), unique key idx_c_global (c), unique key idx_c (c)) partition by key (a,b) partitions 3` +// initFn := func(tkO *testkit.TestKit) { +// tkO.MustExec(`insert into t values (1,1),(2,2),(101,101),(102,102)`) +// } +// alterSQL := `alter table t partition by key (b,a) partitions 5` +// loopFn := func(tkO, tkNO *testkit.TestKit) { +// res := tkO.MustQuery(`select schema_state from information_schema.DDL_JOBS where table_name = 't' order by job_id desc limit 1`) +// schemaState := res.Rows()[0][0].(string) +// switch schemaState { +// case model.StateDeleteOnly.String(): +// // tkNO sees original table/partitions as before the DDL stated +// // tkO uses the original table/partitions, but should also delete from the newly created +// // Global Index, to replace the existing one. +// tkO.MustContainErrMsg(`insert into t values (1,2)`, "[kv:1062]Duplicate entry '2' for key 't.idx_b'") +// tkNO.MustContainErrMsg(`insert into t values (1,2)`, "[kv:1062]Duplicate entry '2' for key 't.idx_b'") +// tkO.MustContainErrMsg(`insert into t values (101,101)`, "[kv:1062]Duplicate entry '101' for key 't.idx_b'") +// tkNO.MustContainErrMsg(`insert into t values (101,101)`, "[kv:1062]Duplicate entry '101' for key 't.idx_b'") +// tkNO.MustQuery(`select * from t`).Sort().Check(testkit.Rows("1 1", "101 101", "102 102", "2 2")) +// tkNO.MustQuery(`select * from t where a < 1000`).Sort().Check(testkit.Rows("1 1", "101 101", "102 102", "2 2")) +// tkNO.MustQuery(`select * from t where a > 0`).Sort().Check(testkit.Rows("1 1", "101 101", "102 102", "2 2")) +// tkNO.MustQuery(`select * from t where a = 1`).Sort().Check(testkit.Rows("1 1")) +// tkNO.MustQuery(`select * from t where a = 1 or a = 2 or a = 3`).Sort().Check(testkit.Rows("1 1", "2 2")) +// tkNO.MustQuery(`select * from t where a in (1,2,3)`).Sort().Check(testkit.Rows("1 1", "2 2")) +// tkNO.MustQuery(`select * from t where a < 100`).Sort().Check(testkit.Rows("1 1", "2 2")) +// +// tkNO.MustQuery(`select * from t where b = 2`).Sort().Check(testkit.Rows("2 2")) +// tkO.MustExec(`insert into t values (3,3)`) +// tkNO.MustExec(`insert into t values (4,4)`) +// tkNO.MustQuery(`select * from t where a = 3`).Sort().Check(testkit.Rows("3 3")) +// tkO.MustQuery(`select * from t where a = 4`).Sort().Check(testkit.Rows("4 4")) +// case model.StateWriteOnly.String(): +// // Both tkO and tkNO uses the original table/partitions, +// // but tkO should also update the newly created +// // Global Index, and tkNO should only delete from it. +// /* +// tkO.MustContainErrMsg(`insert into t values (1,1)`, "[kv:1062]Duplicate entry '1' for key 't.idx_b'") +// tkNO.MustContainErrMsg(`insert into t values (1,1)`, "[kv:1062]Duplicate entry '1' for key 't.idx_b'") +// tkO.MustContainErrMsg(`insert into t values (101,101)`, "[kv:1062]Duplicate entry '101' for key 't.idx_b'") +// tkNO.MustContainErrMsg(`insert into t values (101,101)`, "[kv:1062]Duplicate entry '101' for key 't.idx_b'") +// tkNO.MustQuery(`select * from t`).Sort().Check(testkit.Rows("1 1", "101 101", "102 102", "2 2", "3 3", "4 4")) +// tkO.MustQuery(`select * from t`).Sort().Check(testkit.Rows("1 1", "101 101", "102 102", "2 2", "3 3", "4 4")) +// +// */ +// logutil.BgLogger().Info("insert into t values (5,5)") +// tkO.MustExec(`insert into t values (5,5)`) +// tkNO.MustExec(`insert into t values (6,6)`) +// tkNO.MustQuery(`select * from t where a = 5`).Sort().Check(testkit.Rows("5 5")) +// tkO.MustQuery(`select * from t where a = 6`).Sort().Check(testkit.Rows("6 6")) +// case model.StateWriteReorganization.String(): +// // Both tkO and tkNO uses the original table/partitions, +// // and should also update the newly created Global Index. +// tkO.MustExec(`insert into t values (7,7)`) +// tkNO.MustExec(`insert into t values (8,8)`) +// tkNO.MustQuery(`select * from t where b = 7`).Check(testkit.Rows("7 7")) +// tkO.MustQuery(`select * from t where b = 8`).Check(testkit.Rows("8 8")) +// case model.StateDeleteReorganization.String(): +// // Both tkO now sees the new partitions, and should use the new Global Index, +// // plus double write to the old one. +// // tkNO uses the original table/partitions, +// // and should also update the newly created Global Index. +// tkO.MustExec(`insert into t values (9,9)`) +// tkNO.MustExec(`insert into t values (10,10)`) +// tkNO.MustQuery(`select * from t where b = 9`).Check(testkit.Rows("9 9")) +// tkO.MustQuery(`select * from t where b = 10`).Check(testkit.Rows("10 10")) +// // TODO: Test update and delete! +// // TODO: test key, hash and list partition without default partition :) +// tkNO.MustQuery(`show create table t`).Check(testkit.Rows("" + +// "t CREATE TABLE `t` (\n" + +// " `a` int(11) NOT NULL,\n" + +// " `b` varchar(255) DEFAULT NULL,\n" + +// " PRIMARY KEY (`a`) /*T![clustered_index] CLUSTERED */,\n" + +// " UNIQUE KEY idx_b (`b`) /*T![global_index] GLOBAL */\n" + +// ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin\n" + +// "PARTITION BY RANGE (`a`)\n" + +// "(PARTITION `p1` VALUES LESS THAN (200))")) +// tkO.MustQuery(`show create table t`).Check(testkit.Rows("" + +// "t CREATE TABLE `t` (\n" + +// " `a` int(11) NOT NULL,\n" + +// " `b` varchar(255) DEFAULT NULL,\n" + +// " PRIMARY KEY (`a`) /*T![clustered_index] CLUSTERED */,\n" + +// " UNIQUE KEY idx_b (`b`) /*T![global_index] GLOBAL */\n" + +// ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin\n" + +// "PARTITION BY RANGE (`a`)\n" + +// "(PARTITION `p0` VALUES LESS THAN (100),\n" + +// " PARTITION `p1` VALUES LESS THAN (200))")) +// tkO.MustExec(`insert into t values (3,3)`) +// case model.StateNone.String(): +// // just to not fail :) +// default: +// require.Failf(t, "unhandled schema state '%s'", schemaState) +// } +// } +// postFn := func(tkO *testkit.TestKit) { +// tkO.MustQuery(`select * from t where b = 5`).Sort().Check(testkit.Rows("5 5")) +// tkO.MustExec(`admin check table t`) +// } +// runMultiSchemaTest(t, createSQL, alterSQL, initFn, postFn, loopFn) +//} + +func runMultiSchemaTest(t *testing.T, createSQL, alterSQL string, initFn func(*testkit.TestKit), postFn func(*testkit.TestKit, kv.Storage), loopFn func(tO, tNO *testkit.TestKit)) { + // When debugging, increase the lease, so the schema does not auto reload :) + distCtx := testkit.NewDistExecutionContextWithLease(t, 2, 15*time.Second) + store := distCtx.Store + domOwner := distCtx.GetDomain(0) + domNonOwner := distCtx.GetDomain(1) + + if !domOwner.DDL().OwnerManager().IsOwner() { + domOwner, domNonOwner = domNonOwner, domOwner + } + + seOwner, err := session.CreateSessionWithDomain(store, domOwner) + require.NoError(t, err) + seNonOwner, err := session.CreateSessionWithDomain(store, domNonOwner) + require.NoError(t, err) + + tkDDLOwner := testkit.NewTestKitWithSession(t, store, seOwner) + tkDDLOwner.MustExec(`use test`) + tkDDLOwner.MustExec(`set @@global.tidb_enable_global_index = 1`) + tkDDLOwner.MustExec(`set @@session.tidb_enable_global_index = 1`) + tkO := testkit.NewTestKitWithSession(t, store, seOwner) + tkO.MustExec(`use test`) + tkNO := testkit.NewTestKitWithSession(t, store, seNonOwner) + tkNO.MustExec(`use test`) + + tkDDLOwner.MustExec(createSQL) + domOwner.Reload() + domNonOwner.Reload() + + originalPartitions := make([]int64, 0, 2) + originalIndexIDs := make([]int64, 0, 1) + originalGlobalIndexIDs := make([]int64, 0, 1) + ctx := tkO.Session() + is := domain.GetDomain(ctx).InfoSchema() + tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + require.NoError(t, err) + tableID := tbl.Meta().ID + if tbl.Meta().Partition != nil { + for _, def := range tbl.Meta().Partition.Definitions { + originalPartitions = append(originalPartitions, def.ID) + } + } + for _, idx := range tbl.Meta().Indices { + if idx.Global { + originalGlobalIndexIDs = append(originalGlobalIndexIDs, idx.ID) + continue + } + originalIndexIDs = append(originalIndexIDs, idx.ID) + } + + initFn(tkO) + + verStart := domNonOwner.InfoSchema().SchemaMetaVersion() + hookChan := make(chan struct{}) + hookFunc := func(job *model.Job) { + hookChan <- struct{}{} + logutil.BgLogger().Info("XXXXXXXXXXX Hook now waiting", zap.String("job.State", job.State.String()), zap.String("job.SchemaStage", job.SchemaState.String())) + <-hookChan + logutil.BgLogger().Info("XXXXXXXXXXX Hook released", zap.String("job.State", job.State.String()), zap.String("job.SchemaStage", job.SchemaState.String())) + } + testfailpoint.EnableCall(t, "github.com/pingcap/tidb/pkg/ddl/onJobRunAfter", hookFunc) + alterChan := make(chan error) + go func() { + err := tkDDLOwner.ExecToErr(alterSQL) + logutil.BgLogger().Info("XXXXXXXXXXX DDL done!", zap.String("alterSQL", alterSQL)) + alterChan <- err + }() + // Skip the first state, since we want to compare before vs after in the loop + <-hookChan + hookChan <- struct{}{} + verCurr := verStart + 1 + i := 0 + for { + // Waiting for the next State change to be done (i.e. blocking the state after) + releaseHook := true + for { + select { + case <-hookChan: + case err := <-alterChan: + require.NoError(t, err) + releaseHook = false + logutil.BgLogger().Info("XXXXXXXXXXX release hook") + break + } + domOwner.Reload() + if domNonOwner.InfoSchema().SchemaMetaVersion() == domOwner.InfoSchema().SchemaMetaVersion() { + // looping over reorganize data/indexes + logutil.BgLogger().Info("XXXXXXXXXXX Schema Version has not changed") + hookChan <- struct{}{} + continue + } + break + } + logutil.BgLogger().Info("XXXXXXXXXXX states loop", zap.Int64("verCurr", verCurr), zap.Int64("NonOwner ver", domNonOwner.InfoSchema().SchemaMetaVersion()), zap.Int64("Owner ver", domOwner.InfoSchema().SchemaMetaVersion())) + domOwner.Reload() + require.Equal(t, verCurr-1, domNonOwner.InfoSchema().SchemaMetaVersion()) + require.Equal(t, verCurr, domOwner.InfoSchema().SchemaMetaVersion()) + loopFn(tkO, tkNO) + domNonOwner.Reload() + if !releaseHook { + // Alter done! + break + } + // Continue to next state + verCurr++ + i++ + hookChan <- struct{}{} + } + logutil.BgLogger().Info("XXXXXXXXXXX states loop done") + // Verify that there are no KV entries for old partitions or old indexes!!! + gcWorker, err := gcworker.NewMockGCWorker(store) + require.NoError(t, err) + err = gcWorker.DeleteRanges(context.Background(), uint64(math.MaxInt64)) + require.NoError(t, err) + delRange := tkO.MustQuery(`select * from mysql.gc_delete_range_done`).Rows() + s := "" + for _, row := range delRange { + if s != "" { + s += "\n" + } + for i, col := range row { + if i != 0 { + s += " " + } + s += col.(string) + } + } + logutil.BgLogger().Info("gc_delete_range_done", zap.String("rows", s)) + tkO.MustQuery(`select * from mysql.gc_delete_range`).Check(testkit.Rows()) + ctx = tkO.Session() + is = domain.GetDomain(ctx).InfoSchema() + tbl, err = is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + require.NoError(t, err) + newTableID := tbl.Meta().ID + if tableID != newTableID { + require.False(t, HaveEntriesForTableIndex(t, tkO, tableID, 0), "Old table id %d has still entries!", tableID) + } +GlobalLoop: + for _, globIdx := range originalGlobalIndexIDs { + for _, idx := range tbl.Meta().Indices { + if idx.ID == globIdx { + continue GlobalLoop + } + } + // Global index removed + require.False(t, HaveEntriesForTableIndex(t, tkO, tableID, globIdx), "Global index id %d for table id %d has still entries!", globIdx, tableID) + } +LocalLoop: + for _, locIdx := range originalIndexIDs { + for _, idx := range tbl.Meta().Indices { + if idx.ID == locIdx { + continue LocalLoop + } + } + // local index removed + if tbl.Meta().Partition != nil { + for _, part := range tbl.Meta().Partition.Definitions { + require.False(t, HaveEntriesForTableIndex(t, tkO, part.ID, locIdx), "Local index id %d for partition id %d has still entries!", locIdx, tableID) + } + } + } +PartitionLoop: + for _, partID := range originalPartitions { + for _, def := range tbl.Meta().Partition.Definitions { + if def.ID == partID { + continue PartitionLoop + } + } + // old partitions removed + require.False(t, HaveEntriesForTableIndex(t, tkO, partID, 0), "Reorganized partition id %d for table id %d has still entries!", partID, tableID) + } + if postFn != nil { + postFn(tkO, store) + } + // NOT deferring this, since it might hang on test failures... + domOwner.Close() + domNonOwner.Close() + store.Close() +} + +// HaveEntriesForTableIndex returns number of entries in the KV range of table+index or just the table if index is 0. +// Also checks with gc_delete_range +func HaveEntriesForTableIndex(t *testing.T, tk *testkit.TestKit, tableID, indexID int64) bool { + var start kv.Key + var end kv.Key + if indexID == 0 { + start = tablecodec.EncodeTablePrefix(tableID) + end = tablecodec.EncodeTablePrefix(tableID + 1) + } else { + start = tablecodec.EncodeTableIndexPrefix(tableID, indexID) + end = tablecodec.EncodeTableIndexPrefix(tableID, indexID+1) + } + ctx := tk.Session() + require.NoError(t, sessiontxn.NewTxn(context.Background(), ctx)) + txn, err := ctx.Txn(true) + require.NoError(t, err) + it, err := txn.Iter(start, end) + require.NoError(t, err) + defer it.Close() + count := 0 + for it.Valid() { + count++ + logutil.BgLogger().Info("HaveEntriesForTableIndex", zap.String("key", hex.EncodeToString(it.Key())), zap.String("value", hex.EncodeToString(it.Value()))) + err = it.Next() + require.NoError(t, err) + } + if count > 0 { + logutil.BgLogger().Info("HaveEntriesForTableIndex", zap.Int64("tableID", tableID), zap.Int64("indexID", indexID), zap.Int("count", count)) + return true + } + return false +} + +func TestMultiSchemaReorganizeNoPK(t *testing.T) { + createSQL := `create table t (c1 INT, c2 CHAR(255), c3 CHAR(255), c4 CHAR(255), c5 CHAR(255)) partition by range (c1) (partition p1 values less than (200), partition pMax values less than (maxvalue))` + i := 1 + initFn := func(tkO *testkit.TestKit) { + tkO.MustExec(fmt.Sprintf(`insert into t values (%d,repeat('%d', 25),repeat('%d', 25),repeat('%d', 25),repeat('%d', 25))`, i, 9786756453-i, 6821527184-i, 4185725186-i, 7483634197-i)) + i++ + tkO.MustExec(fmt.Sprintf(`insert into t values (%d,repeat('%d', 25),repeat('%d', 25),repeat('%d', 25),repeat('%d', 25))`, i, 9786756453-i, 6821527184-i, 4185725186-i, 7483634197-i)) + i++ + } + alterSQL := `alter table t reorganize partition p1 into (partition p0 values less than (100), partition p1 values less than (200))` + loopFn := func(tkO, tkNO *testkit.TestKit) { + res := tkO.MustQuery(`select schema_state from information_schema.DDL_JOBS where table_name = 't' order by job_id desc limit 1`) + schemaState := res.Rows()[0][0].(string) + tkO.MustExec(fmt.Sprintf(`insert into t values (%d,'%s',concat('O-', repeat('%d', 25)),repeat('%d', 25),repeat('%d', 25))`, i, schemaState, 6821527184-i, 4185725186-i, 7483634197-i)) + i++ + tkNO.MustExec(fmt.Sprintf(`insert into t values (%d,'%s',concat('NO-',repeat('%d', 25)),repeat('%d', 25),repeat('%d', 25))`, i, schemaState, 6821527184-i, 4185725186-i, 7483634197-i)) + i++ + } + postFn := func(tkO *testkit.TestKit, _ kv.Storage) { + tkO.MustQuery(`select * from t`).Sort().Check(testkit.Rows(""+ + "1 9786756452978675645297867564529786756452978675645297867564529786756452978675645297867564529786756452978675645297867564529786756452978675645297867564529786756452978675645297867564529786756452978675645297867564529786756452978675645297867564529786756452 6821527183682152718368215271836821527183682152718368215271836821527183682152718368215271836821527183682152718368215271836821527183682152718368215271836821527183682152718368215271836821527183682152718368215271836821527183682152718368215271836821527183 4185725185418572518541857251854185725185418572518541857251854185725185418572518541857251854185725185418572518541857251854185725185418572518541857251854185725185418572518541857251854185725185418572518541857251854185725185418572518541857251854185725185 7483634196748363419674836341967483634196748363419674836341967483634196748363419674836341967483634196748363419674836341967483634196748363419674836341967483634196748363419674836341967483634196748363419674836341967483634196748363419674836341967483634196", + "10 delete reorganization NO-6821527174682152717468215271746821527174682152717468215271746821527174682152717468215271746821527174682152717468215271746821527174682152717468215271746821527174682152717468215271746821527174682152717468215271746821527174682152717468215271746821527174 4185725176418572517641857251764185725176418572517641857251764185725176418572517641857251764185725176418572517641857251764185725176418572517641857251764185725176418572517641857251764185725176418572517641857251764185725176418572517641857251764185725176 7483634187748363418774836341877483634187748363418774836341877483634187748363418774836341877483634187748363418774836341877483634187748363418774836341877483634187748363418774836341877483634187748363418774836341877483634187748363418774836341877483634187", + "11 public O-6821527173682152717368215271736821527173682152717368215271736821527173682152717368215271736821527173682152717368215271736821527173682152717368215271736821527173682152717368215271736821527173682152717368215271736821527173682152717368215271736821527173 4185725175418572517541857251754185725175418572517541857251754185725175418572517541857251754185725175418572517541857251754185725175418572517541857251754185725175418572517541857251754185725175418572517541857251754185725175418572517541857251754185725175 7483634186748363418674836341867483634186748363418674836341867483634186748363418674836341867483634186748363418674836341867483634186748363418674836341867483634186748363418674836341867483634186748363418674836341867483634186748363418674836341867483634186", + "12 public NO-6821527172682152717268215271726821527172682152717268215271726821527172682152717268215271726821527172682152717268215271726821527172682152717268215271726821527172682152717268215271726821527172682152717268215271726821527172682152717268215271726821527172 4185725174418572517441857251744185725174418572517441857251744185725174418572517441857251744185725174418572517441857251744185725174418572517441857251744185725174418572517441857251744185725174418572517441857251744185725174418572517441857251744185725174 7483634185748363418574836341857483634185748363418574836341857483634185748363418574836341857483634185748363418574836341857483634185748363418574836341857483634185748363418574836341857483634185748363418574836341857483634185748363418574836341857483634185", + "13 none O-6821527171682152717168215271716821527171682152717168215271716821527171682152717168215271716821527171682152717168215271716821527171682152717168215271716821527171682152717168215271716821527171682152717168215271716821527171682152717168215271716821527171 4185725173418572517341857251734185725173418572517341857251734185725173418572517341857251734185725173418572517341857251734185725173418572517341857251734185725173418572517341857251734185725173418572517341857251734185725173418572517341857251734185725173 7483634184748363418474836341847483634184748363418474836341847483634184748363418474836341847483634184748363418474836341847483634184748363418474836341847483634184748363418474836341847483634184748363418474836341847483634184748363418474836341847483634184", + "14 none NO-6821527170682152717068215271706821527170682152717068215271706821527170682152717068215271706821527170682152717068215271706821527170682152717068215271706821527170682152717068215271706821527170682152717068215271706821527170682152717068215271706821527170 4185725172418572517241857251724185725172418572517241857251724185725172418572517241857251724185725172418572517241857251724185725172418572517241857251724185725172418572517241857251724185725172418572517241857251724185725172418572517241857251724185725172 7483634183748363418374836341837483634183748363418374836341837483634183748363418374836341837483634183748363418374836341837483634183748363418374836341837483634183748363418374836341837483634183748363418374836341837483634183748363418374836341837483634183", + "2 9786756451978675645197867564519786756451978675645197867564519786756451978675645197867564519786756451978675645197867564519786756451978675645197867564519786756451978675645197867564519786756451978675645197867564519786756451978675645197867564519786756451 6821527182682152718268215271826821527182682152718268215271826821527182682152718268215271826821527182682152718268215271826821527182682152718268215271826821527182682152718268215271826821527182682152718268215271826821527182682152718268215271826821527182 4185725184418572518441857251844185725184418572518441857251844185725184418572518441857251844185725184418572518441857251844185725184418572518441857251844185725184418572518441857251844185725184418572518441857251844185725184418572518441857251844185725184 7483634195748363419574836341957483634195748363419574836341957483634195748363419574836341957483634195748363419574836341957483634195748363419574836341957483634195748363419574836341957483634195748363419574836341957483634195748363419574836341957483634195", + "3 delete only O-6821527181682152718168215271816821527181682152718168215271816821527181682152718168215271816821527181682152718168215271816821527181682152718168215271816821527181682152718168215271816821527181682152718168215271816821527181682152718168215271816821527181 4185725183418572518341857251834185725183418572518341857251834185725183418572518341857251834185725183418572518341857251834185725183418572518341857251834185725183418572518341857251834185725183418572518341857251834185725183418572518341857251834185725183 7483634194748363419474836341947483634194748363419474836341947483634194748363419474836341947483634194748363419474836341947483634194748363419474836341947483634194748363419474836341947483634194748363419474836341947483634194748363419474836341947483634194", + "4 delete only NO-6821527180682152718068215271806821527180682152718068215271806821527180682152718068215271806821527180682152718068215271806821527180682152718068215271806821527180682152718068215271806821527180682152718068215271806821527180682152718068215271806821527180 4185725182418572518241857251824185725182418572518241857251824185725182418572518241857251824185725182418572518241857251824185725182418572518241857251824185725182418572518241857251824185725182418572518241857251824185725182418572518241857251824185725182 7483634193748363419374836341937483634193748363419374836341937483634193748363419374836341937483634193748363419374836341937483634193748363419374836341937483634193748363419374836341937483634193748363419374836341937483634193748363419374836341937483634193", + "5 write only O-6821527179682152717968215271796821527179682152717968215271796821527179682152717968215271796821527179682152717968215271796821527179682152717968215271796821527179682152717968215271796821527179682152717968215271796821527179682152717968215271796821527179 4185725181418572518141857251814185725181418572518141857251814185725181418572518141857251814185725181418572518141857251814185725181418572518141857251814185725181418572518141857251814185725181418572518141857251814185725181418572518141857251814185725181 7483634192748363419274836341927483634192748363419274836341927483634192748363419274836341927483634192748363419274836341927483634192748363419274836341927483634192748363419274836341927483634192748363419274836341927483634192748363419274836341927483634192", + "6 write only NO-6821527178682152717868215271786821527178682152717868215271786821527178682152717868215271786821527178682152717868215271786821527178682152717868215271786821527178682152717868215271786821527178682152717868215271786821527178682152717868215271786821527178 4185725180418572518041857251804185725180418572518041857251804185725180418572518041857251804185725180418572518041857251804185725180418572518041857251804185725180418572518041857251804185725180418572518041857251804185725180418572518041857251804185725180 7483634191748363419174836341917483634191748363419174836341917483634191748363419174836341917483634191748363419174836341917483634191748363419174836341917483634191748363419174836341917483634191748363419174836341917483634191748363419174836341917483634191", + "7 write reorganization O-6821527177682152717768215271776821527177682152717768215271776821527177682152717768215271776821527177682152717768215271776821527177682152717768215271776821527177682152717768215271776821527177682152717768215271776821527177682152717768215271776821527177 4185725179418572517941857251794185725179418572517941857251794185725179418572517941857251794185725179418572517941857251794185725179418572517941857251794185725179418572517941857251794185725179418572517941857251794185725179418572517941857251794185725179 7483634190748363419074836341907483634190748363419074836341907483634190748363419074836341907483634190748363419074836341907483634190748363419074836341907483634190748363419074836341907483634190748363419074836341907483634190748363419074836341907483634190", + "8 write reorganization NO-6821527176682152717668215271766821527176682152717668215271766821527176682152717668215271766821527176682152717668215271766821527176682152717668215271766821527176682152717668215271766821527176682152717668215271766821527176682152717668215271766821527176 4185725178418572517841857251784185725178418572517841857251784185725178418572517841857251784185725178418572517841857251784185725178418572517841857251784185725178418572517841857251784185725178418572517841857251784185725178418572517841857251784185725178 7483634189748363418974836341897483634189748363418974836341897483634189748363418974836341897483634189748363418974836341897483634189748363418974836341897483634189748363418974836341897483634189748363418974836341897483634189748363418974836341897483634189", + "9 delete reorganization O-6821527175682152717568215271756821527175682152717568215271756821527175682152717568215271756821527175682152717568215271756821527175682152717568215271756821527175682152717568215271756821527175682152717568215271756821527175682152717568215271756821527175 4185725177418572517741857251774185725177418572517741857251774185725177418572517741857251774185725177418572517741857251774185725177418572517741857251774185725177418572517741857251774185725177418572517741857251774185725177418572517741857251774185725177 7483634188748363418874836341887483634188748363418874836341887483634188748363418874836341887483634188748363418874836341887483634188748363418874836341887483634188748363418874836341887483634188748363418874836341887483634188748363418874836341887483634188")) + } + runMultiSchemaTest(t, createSQL, alterSQL, initFn, postFn, loopFn) +} + +// TestMultiSchemaTruncatePartitionWithGlobalIndex to show behavior when +// truncating a partition with a global index +func TestMultiSchemaTruncatePartitionWithGlobalIndex(t *testing.T) { + // TODO: Also test non-int PK, multi-column PK + createSQL := `create table t (a int primary key, b varchar(255), c varchar(255) default 'Filler', unique key uk_b (b) global) partition by hash (a) partitions 2` + initFn := func(tkO *testkit.TestKit) { + tkO.MustExec(`insert into t (a,b) values (1,1),(2,2),(3,3),(4,4),(5,5),(6,6),(7,7)`) + } + alterSQL := `alter table t truncate partition p1` + loopFn := func(tkO, tkNO *testkit.TestKit) { + res := tkO.MustQuery(`select schema_state from information_schema.DDL_JOBS where table_name = 't' order by job_id desc limit 1`) + schemaState := res.Rows()[0][0].(string) + logutil.BgLogger().Info("XXXXXXXXXXX loopFn", zap.String("schemaState", schemaState)) + switch schemaState { + case "write only": + // tkNO is seeing state None, so unaware of DDL + // tkO is seeing state write only, so using the old partition, + // but are aware of new ids, so should filter them from global index reads. + // Duplicate key errors (from delete only state) are allowed on insert/update, + // even if it cannot read them from the global index, due to filtering. + rows := tkNO.MustQuery(`select * from t`).Sort().Rows() + tkO.MustQuery(`select * from t`).Sort().Check(rows) + tblNO, err := tkNO.Session().GetInfoSchema().TableInfoByName(pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + require.NoError(t, err) + require.Equal(t, model.StateNone, tblNO.Partition.DDLState) + tblO, err := tkO.Session().GetInfoSchema().TableInfoByName(pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + require.NoError(t, err) + require.Equal(t, model.StateWriteOnly, tblO.Partition.DDLState) + require.Equal(t, tblNO.Partition.Definitions[1].ID, tblO.Partition.Definitions[1].ID) + tkNO.MustContainErrMsg(`insert into t values (1,1,"Duplicate key")`, "[kv:1062]Duplicate entry '1' for key 't.") + tkO.MustContainErrMsg(`insert into t values (1,1,"Duplicate key")`, "[kv:1062]Duplicate entry '1' for key 't.") + case "delete only": + // tkNO is seeing state write only, so still can access the dropped partition + // tkO is seeing state delete only, so cannot see the dropped partition, + // but must still write to the shared global indexes. + // So they will get errors on the same entries in the global index. + + tkNO.MustContainErrMsg(`insert into t values (1,1,"Duplicate key")`, "[kv:1062]Duplicate entry '1' for key 't.") + tkO.MustContainErrMsg(`insert into t values (1,1,"Duplicate key")`, "[kv:1062]Duplicate entry '1' for key 't.") + tblNO, err := tkNO.Session().GetInfoSchema().TableInfoByName(pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + require.NoError(t, err) + require.Equal(t, model.StateWriteOnly, tblNO.Partition.DDLState) + tblO, err := tkO.Session().GetInfoSchema().TableInfoByName(pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + require.NoError(t, err) + require.Equal(t, model.StateDeleteOnly, tblO.Partition.DDLState) + require.NotEqual(t, tblNO.Partition.Definitions[1].ID, tblO.Partition.Definitions[1].ID) + + tkNO.MustExec(`insert into t values (21,21,"OK")`) + tkNO.MustExec(`insert into t values (23,23,"OK")`) + tkO.MustContainErrMsg(`insert into t values (21,21,"Duplicate key")`, "[kv:1062]Duplicate entry '21' for key 't.") + tkO.MustContainErrMsg(`insert into t values (6,23,"Duplicate key")`, "[kv:1062]Duplicate entry '") + // Primary is not global, so here we can insert into the new partition, without + // conflicting to the old one + tkO.MustExec(`insert into t values (21,25,"OK")`) + tkO.MustExec(`insert into t values (99,99,"OK")`) + tkNO.MustContainErrMsg(`insert into t values (8,25,"Duplicate key")`, "[kv:1062]Duplicate entry '25' for key 't.uk_b'") + // type differences, cannot use index + tkNO.MustQuery(`select count(*) from t where b = 25`).Check(testkit.Rows("0")) + tkNO.MustQuery(`select b from t where b = 25`).Check(testkit.Rows()) + // PointGet should not find new partitions for StateWriteOnly + tkNO.MustQuery(`select count(*) from t where b = "25"`).Check(testkit.Rows("0")) + tkNO.MustQuery(`select b from t where b = "25"`).Check(testkit.Rows()) + tkNO.MustExec(`update t set a = 2, c = "'a' Updated" where b = "25"`) + require.Equal(t, uint64(0), tkNO.Session().GetSessionVars().StmtCtx.AffectedRows()) + tkNO.MustExec(`update t set a = 2, c = "'a' Updated" where b = "25"`) + require.Equal(t, uint64(0), tkNO.Session().GetSessionVars().StmtCtx.AffectedRows()) + // Primary is not global, so here we can insert into the old partition, without + // conflicting to the new one + tkO.MustQuery(`select count(*) from t where a = 99`).Check(testkit.Rows("1")) + tkNO.MustExec(`insert into t values (99,27,"OK")`) + + tkO.MustQuery(`select count(*) from t where b = "23"`).Check(testkit.Rows("0")) + tkO.MustExec(`update t set a = 2, c = "'a' Updated" where b = "23"`) + require.Equal(t, uint64(0), tkO.Session().GetSessionVars().StmtCtx.AffectedRows()) + tkNO.MustQuery(`select count(*) from t where a = 23`).Check(testkit.Rows("1")) + tkNO.MustQuery(`select * from t where a = 23`).Check(testkit.Rows("23 23 OK")) + tkNO.MustExec(`update t set b = 10 where a = 23`) + require.Equal(t, uint64(1), tkNO.Session().GetSessionVars().StmtCtx.AffectedRows()) + tkNO.MustExec(`update t set b = 23 where a = 23`) + require.Equal(t, uint64(1), tkNO.Session().GetSessionVars().StmtCtx.AffectedRows()) + tkNO.MustContainErrMsg(`update t set b = 25 where a = 23`, "[kv:1062]Duplicate entry '25' for key 't.uk_b'") + tkO.MustExec(`update t set b = 23 where a = 25`) + require.Equal(t, uint64(0), tkO.Session().GetSessionVars().StmtCtx.AffectedRows()) + tkO.MustContainErrMsg(`update t set b = 21 where a = 21`, "[kv:1062]Duplicate entry '21' for key 't.uk_b'") + tkO.MustContainErrMsg(`update t set b = 23 where b = "25"`, "[kv:1062]Duplicate entry '23' for key 't.uk_b'") + + tkO.MustExec(`update t set b = 29 where a = 21`) + require.Equal(t, uint64(1), tkO.Session().GetSessionVars().StmtCtx.AffectedRows()) + tkNO.MustExec(`update t set b = 25 where b = "27"`) + require.Equal(t, uint64(1), tkNO.Session().GetSessionVars().StmtCtx.AffectedRows()) + tkO.MustExec(`update t set b = 27, a = 27 where b = "29"`) + require.Equal(t, uint64(1), tkO.Session().GetSessionVars().StmtCtx.AffectedRows()) + + tkNO.MustQuery(`select * from t`).Sort().Check(testkit.Rows(""+ + "1 1 Filler", + "2 2 Filler", + "21 21 OK", + "23 23 OK", + "3 3 Filler", + "4 4 Filler", + "5 5 Filler", + "6 6 Filler", + "7 7 Filler", + "99 25 OK")) + tkNO.MustQuery(`select b from t order by b`).Check(testkit.Rows(""+ + "1", + "2", + "21", + "23", + "25", + "3", + "4", + "5", + "6", + "7")) + + tkO.MustQuery(`select * from t`).Sort().Check(testkit.Rows(""+ + "2 2 Filler", + "27 27 OK", + "4 4 Filler", + "6 6 Filler", + "99 99 OK")) + tkO.MustQuery(`select b from t order by b`).Check(testkit.Rows(""+ + "2", + "27", + "4", + "6", + "99")) + // TODO: Add tests for delete + case "delete reorganization": + // tkNO is seeing state delete only, so cannot see the dropped partition, + // but must still must give duplicate errors when writes to the global indexes collide + // with the dropped partitions. + // tkO is seeing state delete reorganization, so cannot see the dropped partition, + // and can ignore the dropped partitions entries in the Global Indexes, i.e. overwrite them! + rows := tkO.MustQuery(`select * from t`).Sort().Rows() + tkNO.MustQuery(`select * from t`).Sort().Check(rows) + rows = tkO.MustQuery(`select b from t order by b`).Rows() + tkNO.MustQuery(`select b from t order by b`).Check(rows) + tblNO, err := tkNO.Session().GetInfoSchema().TableInfoByName(pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + require.NoError(t, err) + require.Equal(t, model.StateDeleteOnly, tblNO.Partition.DDLState) + tblO, err := tkO.Session().GetInfoSchema().TableInfoByName(pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + require.NoError(t, err) + require.Equal(t, model.StateDeleteReorganization, tblO.Partition.DDLState) + require.Equal(t, tblNO.Partition.Definitions[1].ID, tblO.Partition.Definitions[1].ID) + tkO.MustQuery(`select b from t where b = "1"`).Check(testkit.Rows()) + tkO.MustExec(`insert into t values (1,1,"OK")`) + tkO.MustQuery(`select b from t where b = "1"`).Check(testkit.Rows("1")) + tkO.MustQuery(`select b from t where b = 1`).Check(testkit.Rows("1")) + tkO.MustContainErrMsg(`insert into t values (3,1,"Duplicate")`, "[kv:1062]Duplicate entry '1' for key 't.uk_b'") + // b = 23 was inserted into the dropped partition, OK to delete + tkO.MustExec(`insert into t values (10,23,"OK")`) + tkNO.MustExec(`insert into t values (41,41,"OK")`) + tkNO.MustContainErrMsg(`insert into t values (12,25,"Duplicate key")`, "[kv:1062]Duplicate entry '25' for key 't.uk_b'") + tkNO.MustContainErrMsg(`insert into t values (25,25,"Duplicate key")`, "[kv:1062]Duplicate entry '25' for key 't.") + tkNO.MustContainErrMsg(`insert into t values (41,27,"Duplicate key")`, "[kv:1062]Duplicate entry '") + tkO.MustExec(`insert into t values (43,43,"OK")`) + tkO.MustContainErrMsg(`insert into t values (44,43,"Duplicate key")`, "[kv:1062]Duplicate entry '") + tkNO.MustContainErrMsg(`update t set b = 5 where a = 41`, "[kv:1062]Duplicate entry '5' for key 't.uk_b'") + tkNO.MustExec(`update t set a = 5 where b = "41"`) + require.Equal(t, uint64(1), tkNO.Session().GetSessionVars().StmtCtx.AffectedRows()) + tkO.MustExec(`update t set a = 7 where b = "43"`) + require.Equal(t, uint64(1), tkO.Session().GetSessionVars().StmtCtx.AffectedRows()) + // This should be silently deleted / overwritten + tkO.MustExec(`update t set b = 5 where b = "43"`) + require.Equal(t, uint64(1), tkO.Session().GetSessionVars().StmtCtx.AffectedRows()) + tkO.MustExec(`update t set b = 3 where b = 41`) + require.Equal(t, uint64(1), tkO.Session().GetSessionVars().StmtCtx.AffectedRows()) + rows = tkNO.MustQuery(`select * from t`).Sort().Rows() + tkO.MustQuery(`select * from t`).Sort().Check(rows) + case "none": + tkNO.MustExec(`insert into t values (81,81,"OK")`) + tkO.MustContainErrMsg(`insert into t values (81,81,"Duplicate key")`, "[kv:1062]Duplicate entry '81' for key 't.") + tkNO.MustExec(`insert into t values (85,85,"OK")`) + tkO.MustExec(`insert into t values (87,87,"OK")`) + rows := tkNO.MustQuery(`select * from t`).Sort().Rows() + tkO.MustQuery(`select * from t`).Sort().Check(rows) + tblNO, err := tkNO.Session().GetInfoSchema().TableInfoByName(pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + require.NoError(t, err) + require.Equal(t, model.StateDeleteReorganization, tblNO.Partition.DDLState) + tblO, err := tkO.Session().GetInfoSchema().TableInfoByName(pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + require.NoError(t, err) + require.Equal(t, model.StateNone, tblO.Partition.DDLState) + default: + require.Failf(t, "unhandled schema state '%s'", schemaState) + } + } + runMultiSchemaTest(t, createSQL, alterSQL, initFn, nil, loopFn) +} + +func TestMultiSchemaTruncatePartitionWithPKGlobal(t *testing.T) { + createSQL := `create table t (a int primary key nonclustered global, b int, c varchar(255) default 'Filler', unique key uk_b (b)) partition by hash (b) partitions 2` + initFn := func(tkO *testkit.TestKit) { + tkO.MustExec(`insert into t (a,b) values (0,0),(1,1),(2,2),(3,3),(4,4),(5,5),(6,6),(7,7)`) + } + alterSQL := `alter table t truncate partition p1` + loopFn := func(tkO, tkNO *testkit.TestKit) { + res := tkO.MustQuery(`select schema_state from information_schema.DDL_JOBS where table_name = 't' order by job_id desc limit 1`) + schemaState := res.Rows()[0][0].(string) + switch schemaState { + case "write only": + // tkNO is seeing state None, so unaware of DDL + // tkO is seeing state write only, so using the old partition, + // but are aware of new ids, so should filter them from global index reads. + // Duplicate key errors (from delete only state) are allowed on insert/update, + // even if it cannot read them from the global index, due to filtering. + rows := tkNO.MustQuery(`select * from t`).Sort().Rows() + tkO.MustQuery(`select * from t`).Sort().Check(rows) + tblNO, err := tkNO.Session().GetInfoSchema().TableInfoByName(pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + require.NoError(t, err) + require.Equal(t, model.StateNone, tblNO.Partition.DDLState) + tblO, err := tkO.Session().GetInfoSchema().TableInfoByName(pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + require.NoError(t, err) + require.Equal(t, model.StateWriteOnly, tblO.Partition.DDLState) + require.Equal(t, tblNO.Partition.Definitions[1].ID, tblO.Partition.Definitions[1].ID) + case "delete only": + // tkNO is seeing state write only, so still can access the dropped partition + // tkO is seeing state delete only, so cannot see the dropped partition, + // but must still write to the shared global indexes. + // So they will get errors on the same entries in the global index. + + tkNO.MustContainErrMsg(`insert into t values (1,1,"Duplicate key")`, "[kv:1062]Duplicate entry '1' for key 't.") + tkNO.MustContainErrMsg(`insert into t values (11,1,"Duplicate key")`, "[kv:1062]Duplicate entry '1' for key 't.uk_b'") + + tkO.MustQuery(`select a from t where a = 1`).Check(testkit.Rows()) + // OK! PK violation due to old partition is still accessible!!! + // Similar to when dropping a unique index, see TestMultiSchemaDropUniqueIndex + tkO.MustContainErrMsg(`insert into t values (1,1,"Duplicate key")`, "[kv:1062]Duplicate entry '1' for key 't.PRIMARY'") + // Note that PK (global) is not violated! and Unique key (b) is not global, + // and the partition is dropped, so OK to write. + tkO.MustExec(`insert into t values (11,1,"OK, non global unique index")`) + // The anomaly here is that tkNO and tkO sees different versions of the table, + // and therefore different data! + tkO.MustQuery(`select * from t where b = 1`).Check(testkit.Rows("11 1 OK, non global unique index")) + tkNO.MustQuery(`select * from t where b = 1`).Check(testkit.Rows("1 1 Filler")) + + tkO.MustExec(`insert into t values (13,13,"OK")`) + tkNO.MustExec(`insert into t values (15,13,"OK, non global unique index")`) + tkO.MustQuery(`select * from t where b = 13`).Check(testkit.Rows("13 13 OK")) + tkNO.MustQuery(`select * from t where b = 13`).Check(testkit.Rows("15 13 OK, non global unique index")) + + tblNO, err := tkNO.Session().GetInfoSchema().TableInfoByName(pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + require.NoError(t, err) + require.Equal(t, model.StateWriteOnly, tblNO.Partition.DDLState) + tblO, err := tkO.Session().GetInfoSchema().TableInfoByName(pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + require.NoError(t, err) + require.Equal(t, model.StateDeleteOnly, tblO.Partition.DDLState) + require.NotEqual(t, tblNO.Partition.Definitions[1].ID, tblO.Partition.Definitions[1].ID) + + tkNO.MustExec(`insert into t values (21,21,"OK")`) + tkNO.MustExec(`insert into t values (23,23,"OK")`) + tkO.MustContainErrMsg(`insert into t values (21,21,"Duplicate key")`, "[kv:1062]Duplicate entry '21' for key 't.PRIMARY'") + tkO.MustContainErrMsg(`insert into t values (6,23,"Duplicate key")`, "[kv:1062]Duplicate entry '6' for key 't.PRIMARY'") + // Primary is global, so here we cannot insert into the new partition, without + // conflicting to the old one + tkO.MustContainErrMsg(`insert into t values (21,25,"Duplicate key")`, "[kv:1062]Duplicate entry '21' for key 't.PRIMARY'") + tkO.MustExec(`insert into t values (25,25,"OK")`) + // Should be able to insert to the new partition, with a duplicate of non-global key + tkNO.MustExec(`insert into t values (95,25,"OK, non global unique key")`) + tkNO.MustContainErrMsg(`insert into t values (25,95,"Duplicate key")`, "[kv:1062]Duplicate entry '25' for key 't.PRIMARY'") + // PointGet should not find new partitions for StateWriteOnly + tkNO.MustQuery(`select count(*) from t where a = 25`).Check(testkit.Rows("0")) + tkNO.MustQuery(`select count(*) from t where b = 25`).Check(testkit.Rows("1")) + tkNO.MustQuery(`select * from t where b = 25`).Check(testkit.Rows("95 25 OK, non global unique key")) + tkNO.MustExec(`update t set a = 17, c = "Updated" where b = 25`) + require.Equal(t, uint64(1), tkNO.Session().GetSessionVars().StmtCtx.AffectedRows()) + + tkO.MustQuery(`select count(*) from t where b = 23`).Check(testkit.Rows("0")) + tkO.MustExec(`update t set a = 19, c = "Updated" where b = 23`) + require.Equal(t, uint64(0), tkO.Session().GetSessionVars().StmtCtx.AffectedRows()) + tkNO.MustQuery(`select count(*) from t where a = 23`).Check(testkit.Rows("1")) + tkNO.MustQuery(`select * from t where a = 23`).Check(testkit.Rows("23 23 OK")) + tkNO.MustExec(`update t set b = 10 where a = 23`) + require.Equal(t, uint64(1), tkNO.Session().GetSessionVars().StmtCtx.AffectedRows()) + tkNO.MustExec(`update t set b = 23 where a = 23`) + require.Equal(t, uint64(1), tkNO.Session().GetSessionVars().StmtCtx.AffectedRows()) + tkNO.MustContainErrMsg(`update t set b = 25 where a = 23`, "[kv:1062]Duplicate entry '25' for key 't.uk_b'") + tkO.MustExec(`update t set b = 23 where a = 25`) + require.Equal(t, uint64(1), tkO.Session().GetSessionVars().StmtCtx.AffectedRows()) + // non-global unique index + // Same state's partition: + tkO.MustContainErrMsg(`update t set b = 23 where a = 13`, "[kv:1062]Duplicate entry '23' for key 't.uk_b'") + tkNO.MustContainErrMsg(`update t set b = 23 where a = 21`, "[kv:1062]Duplicate entry '23' for key 't.uk_b'") + // Others state's partition: + tkO.MustExec(`update t set b = 21, c = "Updated" where a = 13`) + tkO.MustExec(`insert into t values (19,19, "OK")`) + tkNO.MustExec(`update t set b = 19, c = "Updated" where a = 21`) + + // PK + // Same state's partition: + tkO.MustContainErrMsg(`update t set a = 13 where b = 19`, "[kv:1062]Duplicate entry '13' for key 't.PRIMARY'") + tkNO.MustContainErrMsg(`update t set a = 7 where b = 3`, "[kv:1062]Duplicate entry '7' for key 't.PRIMARY'") + // Others state's partition: + tkO.MustContainErrMsg(`update t set a = 7 where b = 19`, "[kv:1062]Duplicate entry '7' for key 't.PRIMARY'") + tkNO.MustContainErrMsg(`update t set a = 13 where b = 13`, "[kv:1062]Duplicate entry '13' for key 't.PRIMARY'") + + tkNO.MustQuery(`select * from t`).Sort().Check(testkit.Rows(""+ + "0 0 Filler", + "1 1 Filler", + "15 13 OK, non global unique index", + "17 25 Updated", + "2 2 Filler", + "21 19 Updated", + "23 23 OK", + "3 3 Filler", + "4 4 Filler", + "5 5 Filler", + "6 6 Filler", + "7 7 Filler")) + + tkO.MustQuery(`select * from t`).Sort().Check(testkit.Rows(""+ + "0 0 Filler", + "11 1 OK, non global unique index", + "13 21 Updated", + "19 19 OK", + "2 2 Filler", + "25 23 OK", + "4 4 Filler", + "6 6 Filler")) + tkO.MustExec(`admin check table t`) + tkNO.MustExec(`admin check table t`) + // TODO: Add tests for delete as well + + case "delete reorganization": + // tkNO is seeing state delete only, so cannot see the dropped partition, + // but must still must give duplicate errors when writes to the global indexes collide + // with the dropped partitions. + // tkO is seeing state delete reorganization, so cannot see the dropped partition, + // and can ignore the dropped partitions entries in the Global Indexes, i.e. overwrite them! + rows := tkO.MustQuery(`select * from t`).Sort().Rows() + tkNO.MustQuery(`select * from t`).Sort().Check(rows) + tblNO, err := tkNO.Session().GetInfoSchema().TableInfoByName(pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + require.NoError(t, err) + require.Equal(t, model.StateDeleteOnly, tblNO.Partition.DDLState) + tblO, err := tkO.Session().GetInfoSchema().TableInfoByName(pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + require.NoError(t, err) + require.Equal(t, model.StateDeleteReorganization, tblO.Partition.DDLState) + require.Equal(t, tblNO.Partition.Definitions[1].ID, tblO.Partition.Definitions[1].ID) + tkO.MustQuery(`select a, b from t where b = 1`).Check(testkit.Rows("11 1")) + tkO.MustQuery(`select b from t where a = 1`).Check(testkit.Rows()) + tkO.MustContainErrMsg(`insert into t values (3,1,"Duplicate")`, "[kv:1062]Duplicate entry '1' for key 't.uk_b'") + // Old partition should be OK to overwrite for tkO, but not tkNO! + tkO.MustExec(`insert into t values (3,3,"OK")`) + tkNO.MustContainErrMsg(`insert into t values (5,5, "Duplicate pk")`, "[kv:1062]Duplicate entry '5' for key 't.PRIMARY'") + tkO.MustExec(`update t set a = 5 where b = 3`) + tkNO.MustContainErrMsg(`update t set a = 7 where b = 3`, "[kv:1062]Duplicate entry '7' for key 't.PRIMARY'") + res := tkNO.MustQuery(`select * from t`).Sort() + res.Check(testkit.Rows(""+ + "0 0 Filler", + "11 1 OK, non global unique index", + "13 21 Updated", + "19 19 OK", + "2 2 Filler", + "25 23 OK", + "4 4 Filler", + "5 3 OK", + "6 6 Filler")) + tkO.MustQuery(`select * from t`).Sort().Check(res.Rows()) + + tkO.MustExec(`admin check table t`) + tkNO.MustExec(`admin check table t`) + case "none": + tkNO.MustExec(`insert into t values (81,81,"OK")`) + tkO.MustContainErrMsg(`insert into t values (81,81,"Duplicate key")`, "[kv:1062]Duplicate entry '81' for key 't.") + tkNO.MustExec(`insert into t values (85,85,"OK")`) + tkO.MustExec(`insert into t values (87,87,"OK")`) + rows := tkNO.MustQuery(`select * from t`).Sort().Rows() + tkO.MustQuery(`select * from t`).Sort().Check(rows) + tblNO, err := tkNO.Session().GetInfoSchema().TableInfoByName(pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + require.NoError(t, err) + require.Equal(t, model.StateDeleteReorganization, tblNO.Partition.DDLState) + tblO, err := tkO.Session().GetInfoSchema().TableInfoByName(pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + require.NoError(t, err) + require.Equal(t, model.StateNone, tblO.Partition.DDLState) + require.Equal(t, tblNO.Partition.Definitions[1].ID, tblO.Partition.Definitions[1].ID) + default: + require.Fail(t, "Unhandled schema state", "State: '%s'", schemaState) + } + } + runMultiSchemaTest(t, createSQL, alterSQL, initFn, nil, loopFn) +} diff --git a/pkg/ddl/tests/partition/reorg_partition_test.go b/pkg/ddl/tests/partition/reorg_partition_test.go new file mode 100644 index 0000000000000..78f42c992baf2 --- /dev/null +++ b/pkg/ddl/tests/partition/reorg_partition_test.go @@ -0,0 +1,1022 @@ +// Copyright 2023 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package partition + +import ( + "context" + "encoding/hex" + "fmt" + "strconv" + "testing" + + "github.com/pingcap/tidb/pkg/ddl" + "github.com/pingcap/tidb/pkg/ddl/logutil" + "github.com/pingcap/tidb/pkg/domain" + "github.com/pingcap/tidb/pkg/errno" + "github.com/pingcap/tidb/pkg/meta/model" + pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/sessionctx" + "github.com/pingcap/tidb/pkg/sessiontxn" + "github.com/pingcap/tidb/pkg/table" + "github.com/pingcap/tidb/pkg/tablecodec" + "github.com/pingcap/tidb/pkg/testkit" + "github.com/pingcap/tidb/pkg/testkit/external" + "github.com/pingcap/tidb/pkg/testkit/testfailpoint" + "github.com/pingcap/tidb/pkg/util/dbterror" + "github.com/stretchr/testify/require" + "go.uber.org/zap" +) + +type allTableData struct { + keys [][]byte + vals [][]byte + tp []string +} + +// TODO: Create a more generic function that gets all accessible table ids +// from all schemas, and checks the full key space so that there are no +// keys for non-existing table IDs. Also figure out how to wait for deleteRange +// Checks that there are no accessible data after an existing table +// assumes that tableIDs are only increasing. +// To be used during failure testing of ALTER, to make sure cleanup is done. +func noNewTablesAfter(t *testing.T, tk *testkit.TestKit, ctx sessionctx.Context, tbl table.Table, msg string) { + waitForGC := tk.MustQuery(`select start_key, end_key from mysql.gc_delete_range union all select start_key, end_key from mysql.gc_delete_range_done`).Rows() + require.NoError(t, sessiontxn.NewTxn(context.Background(), ctx)) + txn, err := ctx.Txn(true) + require.NoError(t, err) + defer func() { + err := txn.Rollback() + require.NoError(t, err) + }() + // Get max tableID (if partitioned) + tblID := tbl.Meta().ID + if pt := tbl.GetPartitionedTable(); pt != nil { + defs := pt.Meta().Partition.Definitions + { + for i := range defs { + tblID = max(tblID, defs[i].ID) + } + } + } + prefix := tablecodec.EncodeTablePrefix(tblID + 1) + it, err := txn.Iter(prefix, nil) + require.NoError(t, err) + for _, rowGC := range waitForGC { + logutil.DDLLogger().Info("GC", + zap.String("start", fmt.Sprintf("%v", rowGC[0])), + zap.String("end", fmt.Sprintf("%v", rowGC[1]))) + } +ROW: + for it.Valid() { + for _, rowGC := range waitForGC { + // OK if queued for range delete / GC + startHex := fmt.Sprintf("%v", rowGC[0]) + endHex := fmt.Sprintf("%v", rowGC[1]) + end, err := hex.DecodeString(endHex) + require.NoError(t, err) + keyHex := hex.EncodeToString(it.Key()) + if startHex <= keyHex && keyHex < endHex { + it.Close() + it, err = txn.Iter(end, nil) + require.NoError(t, err) + continue ROW + } + logutil.DDLLogger().Info("not found in GC", + zap.String("key", keyHex), + zap.String("start", startHex), + zap.String("end", endHex)) + } + foundTblID := tablecodec.DecodeTableID(it.Key()) + // There are internal table ids starting from MaxInt48 -1 and allocating decreasing ids + // Allow 0xFF of them, See JobTableID, ReorgTableID, HistoryTableID, MDLTableID + if it.Key()[0] == 't' && foundTblID < 0xFFFFFFFFFF00 { + is := sessiontxn.GetTxnManager(tk.Session()).GetTxnInfoSchema() + tbl, found := is.TableByID(context.Background(), foundTblID) + tblmsg := " Table ID no longer maps to a table" + if found { + tblmsg = fmt.Sprintf(" Table name: %s", tbl.Meta().Name.O) + } + require.False(t, true, "Found table data after highest physical Table ID %d < %d (%s) "+msg+tblmsg, tblID, foundTblID, it.Key()) + } + break + } +} + +func getAllDataForTableID(t *testing.T, ctx sessionctx.Context, tableID int64) allTableData { + require.NoError(t, sessiontxn.NewTxn(context.Background(), ctx)) + txn, err := ctx.Txn(true) + require.NoError(t, err) + defer func() { + err := txn.Rollback() + require.NoError(t, err) + }() + + all := allTableData{ + keys: make([][]byte, 0), + vals: make([][]byte, 0), + tp: make([]string, 0), + } + prefix := tablecodec.EncodeTablePrefix(tableID) + it, err := txn.Iter(prefix, nil) + require.NoError(t, err) + for it.Valid() { + if !it.Key().HasPrefix(prefix) { + break + } + all.keys = append(all.keys, it.Key()) + all.vals = append(all.vals, it.Value()) + if tablecodec.IsRecordKey(it.Key()) { + all.tp = append(all.tp, "Record") + tblID, kv, _ := tablecodec.DecodeRecordKey(it.Key()) + require.Equal(t, tableID, tblID) + vals, _ := tablecodec.DecodeValuesBytesToStrings(it.Value()) + logutil.DDLLogger().Info("Record", + zap.Int64("pid", tblID), + zap.Stringer("key", kv), + zap.Strings("values", vals)) + } else if tablecodec.IsIndexKey(it.Key()) { + all.tp = append(all.tp, "Index") + } else { + all.tp = append(all.tp, "Other") + } + err = it.Next() + require.NoError(t, err) + } + return all +} + +func TestReorgPartitionFailures(t *testing.T) { + create := `create table t (a int unsigned PRIMARY KEY, b varchar(255), c int, key (b), key (c,b))` + + ` partition by range (a) ` + + `(partition p0 values less than (10),` + + ` partition p1 values less than (20),` + + ` partition p2 values less than (30),` + + ` partition pMax values less than (MAXVALUE))` + alter := "alter table t reorganize partition p1,p2 into (partition p1 values less than (17), partition p1b values less than (24), partition p2 values less than (30))" + beforeDML := []string{ + `insert into t values (1,"1",1),(2,"2",2),(12,"12",21),(13,"13",13),(17,"17",17),(18,"18",18),(23,"23",32),(34,"34",43),(45,"45",54),(56,"56",65)`, + `update t set a = 11, b = "11", c = 11 where a = 17`, + `update t set b = "21", c = 12 where c = 12`, + `delete from t where a = 13`, + `delete from t where b = "56"`, + } + beforeResult := testkit.Rows( + "1 1 1", "11 11 11", "12 12 21", "18 18 18", "2 2 2", "23 23 32", "34 34 43", "45 45 54", + ) + afterDML := []string{ + `insert into t values (5,"5",5),(13,"13",13)`, + `update t set a = 17, b = "17", c = 17 where a = 11`, + `update t set b = "12", c = 21 where c = 12`, + `delete from t where a = 34`, + `delete from t where b = "56"`, + } + afterResult := testkit.Rows( + "1 1 1", "12 12 21", "13 13 13", "17 17 17", "18 18 18", "2 2 2", "23 23 32", "45 45 54", "5 5 5", + ) + testReorganizePartitionFailures(t, create, alter, beforeDML, beforeResult, afterDML, afterResult) +} + +func TestRemovePartitionFailures(t *testing.T) { + create := `create table t (a int unsigned primary key nonclustered, b int not null, c varchar(255)) partition by range(a) ( + partition p0 values less than (100), + partition p1 values less than (200))` + alter := `alter table t remove partitioning` + beforeDML := []string{ + `insert into t values (1,1,1),(2,2,2),(3,3,3),(101,101,101),(102,102,102),(103,103,103)`, + `update t set a = 11, b = "11", c = 11 where a = 1`, + `update t set b = "12", c = 12 where b = 2`, + `delete from t where a = 102`, + `delete from t where b = 103`, + } + beforeResult := testkit.Rows("101 101 101", "11 11 11", "2 12 12", "3 3 3") + afterDML := []string{ + `insert into t values (4,4,4),(5,5,5),(104,104,104)`, + `update t set a = 1, b = 1, c = 1 where a = 11`, + `update t set b = 2, c = 2 where c = 12`, + `update t set a = 9, b = 9 where a = 104`, + `delete from t where a = 5`, + `delete from t where b = 102`, + } + afterResult := testkit.Rows("1 1 1", "101 101 101", "2 2 2", "3 3 3", "4 4 4", "9 9 104") + testReorganizePartitionFailures(t, create, alter, beforeDML, beforeResult, afterDML, afterResult) +} + +func TestPartitionByFailures(t *testing.T) { + create := `create table t (a int unsigned primary key nonclustered, b int not null, c varchar(255)) partition by range(a) ( + partition p0 values less than (100), + partition p1 values less than (200))` + alter := "alter table t partition by range (b) (partition pNoneC values less than (150), partition p2 values less than (300)) update indexes (`primary` global)" + beforeDML := []string{ + `insert into t values (1,1,1),(2,2,2),(3,3,3),(101,101,101),(102,102,102),(103,103,103)`, + `update t set a = 11, b = "11", c = 11 where a = 1`, + `update t set b = "12", c = 12 where b = 2`, + `delete from t where a = 102`, + `delete from t where b = 103`, + } + beforeResult := testkit.Rows("101 101 101", "11 11 11", "2 12 12", "3 3 3") + afterDML := []string{ + `insert into t values (4,4,4),(5,5,5),(104,104,104)`, + `update t set a = 1, b = 1, c = 1 where a = 11`, + `update t set b = 2, c = 2 where c = 12`, + `update t set a = 9, b = 9 where a = 104`, + `delete from t where a = 5`, + `delete from t where b = 102`, + } + afterResult := testkit.Rows("1 1 1", "101 101 101", "2 2 2", "3 3 3", "4 4 4", "9 9 104") + testReorganizePartitionFailures(t, create, alter, beforeDML, beforeResult, afterDML, afterResult) +} + +func TestReorganizePartitionListFailures(t *testing.T) { + create := `create table t (a int unsigned primary key nonclustered global, b int not null, c varchar(255), unique index (c) global) partition by list(b) ( + partition p0 values in (1,2,3), + partition p1 values in (4,5,6), + partition p2 values in (7,8,9))` + alter := `alter table t reorganize partition p0,p2 into (partition pNone1 values in (1,9), partition pNone2 values in (2,8), partition pNone3 values in (3,7))` + beforeDML := []string{ + `insert into t values (1,1,1),(2,2,2),(4,4,4),(8,8,8),(9,9,9),(6,6,6)`, + `update t set a = 7, b = 7, c = 7 where a = 1`, + `update t set b = 3, c = 3 where c = 4`, + `delete from t where a = 8`, + `delete from t where b = 2`, + } + beforeResult := testkit.Rows("4 3 3", "6 6 6", "7 7 7", "9 9 9") + afterDML := []string{ + `insert into t values (1,1,1),(5,5,5),(8,8,8)`, + `update t set a = 2, b = 2, c = 2 where a = 1`, + `update t set a = 1, b = 1, c = 1 where c = 6`, + `update t set a = 6, b = 6 where a = 9`, + `delete from t where a = 5`, + `delete from t where b = 3`, + } + afterResult := testkit.Rows("1 1 1", "2 2 2", "6 6 9", "7 7 7", "8 8 8") + testReorganizePartitionFailures(t, create, alter, beforeDML, beforeResult, afterDML, afterResult) +} + +func TestPartitionByListFailures(t *testing.T) { + create := `create table t (a int unsigned primary key nonclustered global, b int not null, c varchar(255), unique index (b), unique index (c) global) partition by list(b) ( + partition p0 values in (1,2,3,4,5,6), + partition p1 values in (11,10,9,8,7))` + alter := `alter table t partition by list columns (c) (partition pNone1 values in (1,11,3,5,7,9), partition pNone2 values in (2,4,8,10,6)) update indexes (b global, c local)` + beforeDML := []string{ + `insert into t values (1,1,1),(2,2,2),(4,4,4),(8,8,8),(9,9,9),(6,6,6)`, + `update t set a = 7, b = 7, c = 7 where a = 1`, + `update t set b = 3, c = 3 where c = "4"`, + `delete from t where a = 8`, + `delete from t where b = 2`, + } + beforeResult := testkit.Rows("4 3 3", "6 6 6", "7 7 7", "9 9 9") + afterDML := []string{ + `insert into t values (1,1,1),(5,5,5),(8,8,8)`, + `update t set a = 2, b = 2, c = 2 where a = 1`, + `update t set a = 1, b = 1, c = 1 where c = "6"`, + `update t set a = 6, b = 6 where a = 9`, + `delete from t where a = 5`, + `delete from t where b = 3`, + } + afterResult := testkit.Rows("1 1 1", "2 2 2", "6 6 9", "7 7 7", "8 8 8") + testReorganizePartitionFailures(t, create, alter, beforeDML, beforeResult, afterDML, afterResult) +} + +func TestAddHashPartitionFailures(t *testing.T) { + create := `create table t (a int unsigned primary key nonclustered global, b int not null, c varchar(255), unique index (c) global) partition by hash(b) partitions 3` + alter := `alter table t add partition partitions 2` + beforeDML := []string{ + `insert into t values (1,1,1),(2,2,2),(4,4,4),(8,8,8),(9,9,9),(6,6,6)`, + `update t set a = 7, b = 7, c = 7 where a = 1`, + `update t set b = 3, c = 3 where c = "4"`, + `delete from t where a = 8`, + `delete from t where b = 2`, + } + beforeResult := testkit.Rows("4 3 3", "6 6 6", "7 7 7", "9 9 9") + afterDML := []string{ + `insert into t values (1,1,1),(5,5,5),(8,8,8)`, + `update t set a = 2, b = 2, c = 2 where a = 1`, + `update t set a = 1, b = 1, c = 1 where c = "6"`, + `update t set a = 6, b = 6 where a = 9`, + `delete from t where a = 5`, + `delete from t where b = 3`, + } + afterResult := testkit.Rows("1 1 1", "2 2 2", "6 6 9", "7 7 7", "8 8 8") + testReorganizePartitionFailures(t, create, alter, beforeDML, beforeResult, afterDML, afterResult) +} + +func TestCoalesceKeyPartitionFailures(t *testing.T) { + create := `create table t (a int unsigned primary key nonclustered global, b int not null, c varchar(255), unique index (b) global, unique index (c)) partition by key(c) partitions 5` + alter := `alter table t coalesce partition 2` + beforeDML := []string{ + `insert into t values (1,1,1),(2,2,2),(4,4,4),(8,8,8),(9,9,9),(6,6,6)`, + `update t set a = 7, b = 7, c = 7 where a = 1`, + `update t set b = 3, c = 3 where c = "4"`, + `delete from t where a = 8`, + `delete from t where b = 2`, + } + beforeResult := testkit.Rows("4 3 3", "6 6 6", "7 7 7", "9 9 9") + afterDML := []string{ + `insert into t values (1,1,1),(5,5,5),(8,8,8)`, + `update t set a = 2, b = 2, c = 2 where a = 1`, + `update t set a = 1, b = 1, c = 1 where c = "6"`, + `update t set a = 6, b = 6 where a = 9`, + `delete from t where a = 5`, + `delete from t where b = 3`, + } + afterResult := testkit.Rows("1 1 1", "2 2 2", "6 6 9", "7 7 7", "8 8 8") + testReorganizePartitionFailures(t, create, alter, beforeDML, beforeResult, afterDML, afterResult) +} + +func TestPartitionByNonPartitionedTable(t *testing.T) { + create := `create table t (a int)` + alter := `alter table t partition by range (a) (partition p0 values less than (20))` + beforeResult := testkit.Rows() + afterResult := testkit.Rows() + testReorganizePartitionFailures(t, create, alter, nil, beforeResult, nil, afterResult) +} + +func testReorganizePartitionFailures(t *testing.T, createSQL, alterSQL string, beforeDML []string, beforeResult [][]any, afterDML []string, afterResult [][]any, skipTests ...string) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("set tidb_enable_global_index=true") + defer func() { + tk.MustExec("set tidb_enable_global_index=default") + }() + // Fail means we simply inject an error, and set the error count very high to see what happens + // we do expect to do best effort rollback here as well! + // Cancel means we set job.State = JobStateCancelled, as in no need to do more + // Rollback means we do full rollback before returning error. + tests := []struct { + name string + count int + rollForwardFrom int + }{ + { + "Cancel", + 1, + -1, + }, + { + "Fail", + 5, + 4, + }, + { + "Rollback", + 4, + -1, + }, + } + oldWaitTimeWhenErrorOccurred := ddl.WaitTimeWhenErrorOccurred + defer func() { + ddl.WaitTimeWhenErrorOccurred = oldWaitTimeWhenErrorOccurred + }() + ddl.WaitTimeWhenErrorOccurred = 0 + for _, test := range tests { + SUBTEST: + for i := 1; i <= test.count; i++ { + suffix := test.name + strconv.Itoa(i) + for _, skip := range skipTests { + if suffix == skip { + continue SUBTEST + } + } + suffixComment := ` /* ` + suffix + ` */` + tk.MustExec(createSQL + suffixComment) + for _, sql := range beforeDML { + tk.MustExec(sql + suffixComment) + } + tk.MustQuery(`select * from t ` + suffixComment).Sort().Check(beforeResult) + tOrg := external.GetTableByName(t, tk, "test", "t") + var idxID int64 + if len(tOrg.Meta().Indices) > 0 { + idxID = tOrg.Meta().Indices[0].ID + } + oldCreate := tk.MustQuery(`show create table t` + suffixComment).Rows() + name := "github.com/pingcap/tidb/pkg/ddl/reorgPart" + suffix + term := "return(true)" + if test.rollForwardFrom > 0 && test.rollForwardFrom <= i { + term = "10*" + term + } + testfailpoint.Enable(t, name, term) + err := tk.ExecToErr(alterSQL + suffixComment) + tt := external.GetTableByName(t, tk, "test", "t") + partition := tt.Meta().Partition + rollback := false + if test.rollForwardFrom > 0 && test.rollForwardFrom <= i { + require.NoError(t, err) + } else { + rollback = true + require.Error(t, err, "failpoint reorgPart"+suffix) + // TODO: gracefully handle failures during WriteReorg also for nonclustered tables + // with unique indexes. + // Currently it can also do: + // Error "[kv:1062]Duplicate entry '7' for key 't.c'" does not contain "Injected error by reorgPartFail2" + //require.ErrorContains(t, err, "Injected error by reorgPart"+suffix) + tk.MustQuery(`show create table t` + suffixComment).Check(oldCreate) + if partition == nil { + require.Nil(t, tOrg.Meta().Partition, suffix) + } else { + require.Equal(t, len(tOrg.Meta().Partition.Definitions), len(partition.Definitions), suffix) + require.Equal(t, 0, len(partition.AddingDefinitions), suffix) + require.Equal(t, 0, len(partition.DroppingDefinitions), suffix) + } + noNewTablesAfter(t, tk, tk.Session(), tOrg, suffix) + } + testfailpoint.Disable(t, name) + require.Equal(t, len(tOrg.Meta().Indices), len(tt.Meta().Indices), suffix) + if rollback && idxID != 0 { + require.Equal(t, idxID, tt.Meta().Indices[0].ID, suffix) + } + tk.MustExec(`admin check table t` + suffixComment) + for _, sql := range afterDML { + tk.MustExec(sql + suffixComment) + } + tk.MustQuery(`select * from t` + suffixComment).Sort().Check(afterResult) + tk.MustExec(`drop table t` + suffixComment) + // TODO: Check TiFlash replicas + // TODO: Check Label rules + // TODO: Check bundles + // TODO: Check autoIDs + } + } +} + +func TestReorgPartitionConcurrent(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + schemaName := "ReorgPartConcurrent" + tk.MustExec("create database " + schemaName) + tk.MustExec("use " + schemaName) + tk.MustExec(`create table t (a int unsigned PRIMARY KEY, b varchar(255), c int, key (b), key (c,b))` + + ` partition by range (a) ` + + `(partition p0 values less than (10),` + + ` partition p1 values less than (20),` + + ` partition pMax values less than (MAXVALUE))`) + tk.MustExec(`insert into t values (1,"1",1), (10,"10",10),(23,"23",32),(34,"34",43),(45,"45",54),(56,"56",65)`) + syncOnChanged := make(chan bool) + defer close(syncOnChanged) + testfailpoint.EnableCall(t, "github.com/pingcap/tidb/pkg/ddl/afterReorganizePartition", func() { + <-syncOnChanged + // We want to wait here + <-syncOnChanged + }) + + wait := make(chan bool) + defer close(wait) + + currState := model.StateNone + testfailpoint.EnableCall(t, "github.com/pingcap/tidb/pkg/ddl/onJobRunBefore", func(job *model.Job) { + if job.Type == model.ActionReorganizePartition && + (job.SchemaState == model.StateDeleteOnly || + job.SchemaState == model.StateWriteOnly || + job.SchemaState == model.StateWriteReorganization || + job.SchemaState == model.StateDeleteReorganization || + job.SchemaState == model.StatePublic) && + currState != job.SchemaState { + currState = job.SchemaState + <-wait + <-wait + } + }) + alterErr := make(chan error, 1) + go backgroundExec(store, schemaName, "alter table t reorganize partition p1 into (partition p1a values less than (15), partition p1b values less than (20))", alterErr) + + wait <- true + // StateDeleteOnly + deleteOnlyInfoSchema := sessiontxn.GetTxnManager(tk.Session()).GetTxnInfoSchema() + wait <- true + + // StateWriteOnly + wait <- true + tk.MustExec(`insert into t values (11, "11", 11),(12,"12",21)`) + tk.MustExec(`admin check table t`) + writeOnlyInfoSchema := sessiontxn.GetTxnManager(tk.Session()).GetTxnInfoSchema() + require.Equal(t, int64(1), writeOnlyInfoSchema.SchemaMetaVersion()-deleteOnlyInfoSchema.SchemaMetaVersion()) + deleteOnlyTbl, err := deleteOnlyInfoSchema.TableByName(context.Background(), pmodel.NewCIStr(schemaName), pmodel.NewCIStr("t")) + require.NoError(t, err) + writeOnlyTbl, err := writeOnlyInfoSchema.TableByName(context.Background(), pmodel.NewCIStr(schemaName), pmodel.NewCIStr("t")) + require.NoError(t, err) + writeOnlyParts := writeOnlyTbl.Meta().Partition + writeOnlyTbl.Meta().Partition = deleteOnlyTbl.Meta().Partition + // If not DeleteOnly is working, then this would show up when reorg is done + tk.MustExec(`delete from t where a = 11`) + tk.MustExec(`update t set b = "12b", c = 12 where a = 12`) + tk.MustExec(`admin check table t`) + writeOnlyTbl.Meta().Partition = writeOnlyParts + tk.MustExec(`admin check table t`) + wait <- true + + // StateWriteReorganization + wait <- true + tk.MustExec(`insert into t values (14, "14", 14),(15, "15",15)`) + writeReorgInfoSchema := sessiontxn.GetTxnManager(tk.Session()).GetTxnInfoSchema() + tk.MustQuery(`show create table t`).Check(testkit.Rows("" + + "t CREATE TABLE `t` (\n" + + " `a` int(10) unsigned NOT NULL,\n" + + " `b` varchar(255) DEFAULT NULL,\n" + + " `c` int(11) DEFAULT NULL,\n" + + " PRIMARY KEY (`a`) /*T![clustered_index] CLUSTERED */,\n" + + " KEY `b` (`b`),\n" + + " KEY `c` (`c`,`b`)\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin\n" + + "PARTITION BY RANGE (`a`)\n" + + "(PARTITION `p0` VALUES LESS THAN (10),\n" + + " PARTITION `p1` VALUES LESS THAN (20),\n" + + " PARTITION `pMax` VALUES LESS THAN (MAXVALUE))")) + wait <- true + + // StateDeleteReorganization + wait <- true + tk.MustQuery(`select * from t where c between 10 and 22`).Sort().Check(testkit.Rows(""+ + "10 10 10", + "12 12b 12", + "14 14 14", + "15 15 15")) + deleteReorgInfoSchema := sessiontxn.GetTxnManager(tk.Session()).GetTxnInfoSchema() + require.Equal(t, int64(1), deleteReorgInfoSchema.SchemaMetaVersion()-writeReorgInfoSchema.SchemaMetaVersion()) + tk.MustExec(`insert into t values (16, "16", 16)`) + oldTbl, err := writeReorgInfoSchema.TableByName(context.Background(), pmodel.NewCIStr(schemaName), pmodel.NewCIStr("t")) + require.NoError(t, err) + partDef := oldTbl.Meta().Partition.Definitions[1] + require.Equal(t, "p1", partDef.Name.O) + rows := getNumRowsFromPartitionDefs(t, tk, oldTbl, oldTbl.Meta().Partition.Definitions[1:2]) + require.Equal(t, 5, rows) + currTbl, err := deleteReorgInfoSchema.TableByName(context.Background(), pmodel.NewCIStr(schemaName), pmodel.NewCIStr("t")) + require.NoError(t, err) + currPart := currTbl.Meta().Partition + currTbl.Meta().Partition = oldTbl.Meta().Partition + tk.MustQuery(`select * from t where b = "16"`).Sort().Check(testkit.Rows("16 16 16")) + tk.MustExec(`admin check table t`) + tk.MustQuery(`show create table t`).Check(testkit.Rows("" + + "t CREATE TABLE `t` (\n" + + " `a` int(10) unsigned NOT NULL,\n" + + " `b` varchar(255) DEFAULT NULL,\n" + + " `c` int(11) DEFAULT NULL,\n" + + " PRIMARY KEY (`a`) /*T![clustered_index] CLUSTERED */,\n" + + " KEY `b` (`b`),\n" + + " KEY `c` (`c`,`b`)\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin\n" + + "PARTITION BY RANGE (`a`)\n" + + "(PARTITION `p0` VALUES LESS THAN (10),\n" + + " PARTITION `p1` VALUES LESS THAN (20),\n" + + " PARTITION `pMax` VALUES LESS THAN (MAXVALUE))")) + tk.MustQuery(`select * from t partition (p1)`).Sort().Check(testkit.Rows(""+ + "10 10 10", + "12 12b 12", + "14 14 14", + "15 15 15", + "16 16 16")) + currTbl.Meta().Partition = currPart + tk.MustQuery(`show create table t`).Check(testkit.Rows("" + + "t CREATE TABLE `t` (\n" + + " `a` int(10) unsigned NOT NULL,\n" + + " `b` varchar(255) DEFAULT NULL,\n" + + " `c` int(11) DEFAULT NULL,\n" + + " PRIMARY KEY (`a`) /*T![clustered_index] CLUSTERED */,\n" + + " KEY `b` (`b`),\n" + + " KEY `c` (`c`,`b`)\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin\n" + + "PARTITION BY RANGE (`a`)\n" + + "(PARTITION `p0` VALUES LESS THAN (10),\n" + + " PARTITION `p1a` VALUES LESS THAN (15),\n" + + " PARTITION `p1b` VALUES LESS THAN (20),\n" + + " PARTITION `pMax` VALUES LESS THAN (MAXVALUE))")) + wait <- true + + // StatePublic + wait <- true + tk.MustQuery(`select * from t where c between 10 and 22`).Sort().Check(testkit.Rows(""+ + "10 10 10", + "12 12b 12", + "14 14 14", + "15 15 15", + "16 16 16")) + publicInfoSchema := sessiontxn.GetTxnManager(tk.Session()).GetTxnInfoSchema() + require.Equal(t, int64(1), publicInfoSchema.SchemaMetaVersion()-deleteReorgInfoSchema.SchemaMetaVersion()) + tk.MustExec(`insert into t values (17, "17", 17)`) + oldTbl, err = deleteReorgInfoSchema.TableByName(context.Background(), pmodel.NewCIStr(schemaName), pmodel.NewCIStr("t")) + require.NoError(t, err) + partDef = oldTbl.Meta().Partition.Definitions[1] + require.Equal(t, "p1a", partDef.Name.O) + rows = getNumRowsFromPartitionDefs(t, tk, oldTbl, oldTbl.Meta().Partition.Definitions[1:2]) + require.Equal(t, 3, rows) + tk.MustQuery(`select * from t partition (p1a)`).Sort().Check(testkit.Rows("10 10 10", "12 12b 12", "14 14 14")) + currTbl, err = publicInfoSchema.TableByName(context.Background(), pmodel.NewCIStr(schemaName), pmodel.NewCIStr("t")) + require.NoError(t, err) + currPart = currTbl.Meta().Partition + currTbl.Meta().Partition = oldTbl.Meta().Partition + tk.MustQuery(`select * from t where b = "17"`).Sort().Check(testkit.Rows("17 17 17")) + tk.MustExec(`admin check table t`) + tk.MustQuery(`show create table t`).Check(testkit.Rows("" + + "t CREATE TABLE `t` (\n" + + " `a` int(10) unsigned NOT NULL,\n" + + " `b` varchar(255) DEFAULT NULL,\n" + + " `c` int(11) DEFAULT NULL,\n" + + " PRIMARY KEY (`a`) /*T![clustered_index] CLUSTERED */,\n" + + " KEY `b` (`b`),\n" + + " KEY `c` (`c`,`b`)\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin\n" + + "PARTITION BY RANGE (`a`)\n" + + "(PARTITION `p0` VALUES LESS THAN (10),\n" + + " PARTITION `p1a` VALUES LESS THAN (15),\n" + + " PARTITION `p1b` VALUES LESS THAN (20),\n" + + " PARTITION `pMax` VALUES LESS THAN (MAXVALUE))")) + currTbl.Meta().Partition = currPart + wait <- true + syncOnChanged <- true + // This reads the new schema (Schema update completed) + tk.MustQuery(`select * from t where c between 10 and 22`).Sort().Check(testkit.Rows(""+ + "10 10 10", + "12 12b 12", + "14 14 14", + "15 15 15", + "16 16 16", + "17 17 17")) + tk.MustExec(`admin check table t`) + newInfoSchema := sessiontxn.GetTxnManager(tk.Session()).GetTxnInfoSchema() + require.Equal(t, int64(1), newInfoSchema.SchemaMetaVersion()-publicInfoSchema.SchemaMetaVersion()) + oldTbl, err = publicInfoSchema.TableByName(context.Background(), pmodel.NewCIStr(schemaName), pmodel.NewCIStr("t")) + require.NoError(t, err) + partDef = oldTbl.Meta().Partition.Definitions[1] + require.Equal(t, "p1a", partDef.Name.O) + tk.MustQuery(`show create table t`).Check(testkit.Rows("" + + "t CREATE TABLE `t` (\n" + + " `a` int(10) unsigned NOT NULL,\n" + + " `b` varchar(255) DEFAULT NULL,\n" + + " `c` int(11) DEFAULT NULL,\n" + + " PRIMARY KEY (`a`) /*T![clustered_index] CLUSTERED */,\n" + + " KEY `b` (`b`),\n" + + " KEY `c` (`c`,`b`)\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin\n" + + "PARTITION BY RANGE (`a`)\n" + + "(PARTITION `p0` VALUES LESS THAN (10),\n" + + " PARTITION `p1a` VALUES LESS THAN (15),\n" + + " PARTITION `p1b` VALUES LESS THAN (20),\n" + + " PARTITION `pMax` VALUES LESS THAN (MAXVALUE))")) + newTbl, err := newInfoSchema.TableByName(context.Background(), pmodel.NewCIStr(schemaName), pmodel.NewCIStr("t")) + require.NoError(t, err) + newPart := newTbl.Meta().Partition + newTbl.Meta().Partition = oldTbl.Meta().Partition + tk.MustQuery(`show create table t`).Check(testkit.Rows("" + + "t CREATE TABLE `t` (\n" + + " `a` int(10) unsigned NOT NULL,\n" + + " `b` varchar(255) DEFAULT NULL,\n" + + " `c` int(11) DEFAULT NULL,\n" + + " PRIMARY KEY (`a`) /*T![clustered_index] CLUSTERED */,\n" + + " KEY `b` (`b`),\n" + + " KEY `c` (`c`,`b`)\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin\n" + + "PARTITION BY RANGE (`a`)\n" + + "(PARTITION `p0` VALUES LESS THAN (10),\n" + + " PARTITION `p1a` VALUES LESS THAN (15),\n" + + " PARTITION `p1b` VALUES LESS THAN (20),\n" + + " PARTITION `pMax` VALUES LESS THAN (MAXVALUE))")) + tk.MustExec(`admin check table t`) + newTbl.Meta().Partition = newPart + syncOnChanged <- true + require.NoError(t, <-alterErr) +} + +func TestReorgPartitionFailConcurrent(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + schemaName := "ReorgPartFailConcurrent" + tk.MustExec("create database " + schemaName) + tk.MustExec("use " + schemaName) + tk.MustExec(`create table t (a int unsigned PRIMARY KEY, b varchar(255), c int, key (b), key (c,b))` + + ` partition by range (a) ` + + `(partition p0 values less than (10),` + + ` partition p1 values less than (20),` + + ` partition pMax values less than (MAXVALUE))`) + tk.MustExec(`insert into t values (1,"1",1), (12,"12",21),(23,"23",32),(34,"34",43),(45,"45",54),(56,"56",65)`) + + wait := make(chan bool) + defer close(wait) + + // Test insert of duplicate key during copy phase + injected := false + testfailpoint.EnableCall(t, "github.com/pingcap/tidb/pkg/ddl/onJobRunBefore", func(job *model.Job) { + if job.Type == model.ActionReorganizePartition && job.SchemaState == model.StateWriteReorganization && !injected { + injected = true + <-wait + <-wait + } + }) + alterErr := make(chan error, 1) + go backgroundExec(store, schemaName, "alter table t reorganize partition p1 into (partition p1a values less than (15), partition p1b values less than (20))", alterErr) + wait <- true + tk.MustExec(`insert into t values (14, "14", 14),(15, "15",15)`) + tk.MustGetErrCode(`insert into t values (11, "11", 11),(12,"duplicate PK 💥", 13)`, errno.ErrDupEntry) + tk.MustExec(`admin check table t`) + wait <- true + require.NoError(t, <-alterErr) + tk.MustQuery(`select * from t where c between 10 and 22`).Sort().Check(testkit.Rows(""+ + "12 12 21", + "14 14 14", + "15 15 15")) + tk.MustExec(`admin check table t`) + tk.MustQuery(`show create table t`).Check(testkit.Rows("" + + "t CREATE TABLE `t` (\n" + + " `a` int(10) unsigned NOT NULL,\n" + + " `b` varchar(255) DEFAULT NULL,\n" + + " `c` int(11) DEFAULT NULL,\n" + + " PRIMARY KEY (`a`) /*T![clustered_index] CLUSTERED */,\n" + + " KEY `b` (`b`),\n" + + " KEY `c` (`c`,`b`)\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin\n" + + "PARTITION BY RANGE (`a`)\n" + + "(PARTITION `p0` VALUES LESS THAN (10),\n" + + " PARTITION `p1a` VALUES LESS THAN (15),\n" + + " PARTITION `p1b` VALUES LESS THAN (20),\n" + + " PARTITION `pMax` VALUES LESS THAN (MAXVALUE))")) + + // Test reorg of duplicate key + prevState := model.StateNone + testfailpoint.EnableCall(t, "github.com/pingcap/tidb/pkg/ddl/onJobRunBefore", func(job *model.Job) { + if job.Type == model.ActionReorganizePartition && + job.SchemaState == model.StateWriteReorganization && + job.SnapshotVer == 0 && + prevState != job.SchemaState { + prevState = job.SchemaState + <-wait + <-wait + } + if job.Type == model.ActionReorganizePartition && + job.SchemaState == model.StateDeleteReorganization && + prevState != job.SchemaState { + prevState = job.SchemaState + <-wait + <-wait + } + }) + go backgroundExec(store, schemaName, "alter table t reorganize partition p1a,p1b into (partition p1a values less than (14), partition p1b values less than (17), partition p1c values less than (20))", alterErr) + wait <- true + infoSchema := sessiontxn.GetTxnManager(tk.Session()).GetTxnInfoSchema() + tbl, err := infoSchema.TableByName(context.Background(), pmodel.NewCIStr(schemaName), pmodel.NewCIStr("t")) + require.NoError(t, err) + require.Equal(t, 0, getNumRowsFromPartitionDefs(t, tk, tbl, tbl.Meta().Partition.AddingDefinitions)) + tk.MustExec(`delete from t where a = 14`) + tk.MustExec(`insert into t values (13, "13", 31),(14,"14b",14),(16, "16",16)`) + tk.MustExec(`admin check table t`) + wait <- true + wait <- true + tbl, err = infoSchema.TableByName(context.Background(), pmodel.NewCIStr(schemaName), pmodel.NewCIStr("t")) + require.NoError(t, err) + require.Equal(t, 5, getNumRowsFromPartitionDefs(t, tk, tbl, tbl.Meta().Partition.AddingDefinitions)) + tk.MustExec(`delete from t where a = 15`) + tk.MustExec(`insert into t values (11, "11", 11),(15,"15b",15),(17, "17",17)`) + tk.MustExec(`admin check table t`) + wait <- true + require.NoError(t, <-alterErr) + + tk.MustExec(`admin check table t`) + tk.MustQuery(`select * from t where a between 10 and 22`).Sort().Check(testkit.Rows(""+ + "11 11 11", + "12 12 21", + "13 13 31", + "14 14b 14", + "15 15b 15", + "16 16 16", + "17 17 17")) + tk.MustQuery(`select * from t where c between 10 and 22`).Sort().Check(testkit.Rows(""+ + "11 11 11", + "12 12 21", + "14 14b 14", + "15 15b 15", + "16 16 16", + "17 17 17")) + tk.MustQuery(`select * from t where b between "10" and "22"`).Sort().Check(testkit.Rows(""+ + "11 11 11", + "12 12 21", + "13 13 31", + "14 14b 14", + "15 15b 15", + "16 16 16", + "17 17 17")) +} + +func getNumRowsFromPartitionDefs(t *testing.T, tk *testkit.TestKit, tbl table.Table, defs []model.PartitionDefinition) int { + ctx := tk.Session() + pt := tbl.GetPartitionedTable() + require.NotNil(t, pt) + cnt := 0 + for _, def := range defs { + data := getAllDataForTableID(t, ctx, def.ID) + require.True(t, len(data.keys) == len(data.vals)) + require.True(t, len(data.keys) == len(data.tp)) + for _, s := range data.tp { + if s == "Record" { + cnt++ + } + } + } + return cnt +} + +func TestReorgPartitionFailInject(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + schemaName := "ReorgPartFailInjectConcurrent" + tk.MustExec("create database " + schemaName) + tk.MustExec("use " + schemaName) + tk.MustExec(`create table t (a int unsigned PRIMARY KEY, b varchar(255), c int, key (b), key (c,b))` + + ` partition by range (a) ` + + `(partition p0 values less than (10),` + + ` partition p1 values less than (20),` + + ` partition pMax values less than (MAXVALUE))`) + tk.MustExec(`insert into t values (1,"1",1), (12,"12",21),(23,"23",32),(34,"34",43),(45,"45",54),(56,"56",65)`) + + wait := make(chan bool) + defer close(wait) + + injected := false + testfailpoint.EnableCall(t, "github.com/pingcap/tidb/pkg/ddl/onJobRunBefore", func(job *model.Job) { + if job.Type == model.ActionReorganizePartition && job.SchemaState == model.StateWriteReorganization && !injected { + injected = true + <-wait + <-wait + } + }) + alterErr := make(chan error, 1) + go backgroundExec(store, schemaName, "alter table t reorganize partition p1 into (partition p1a values less than (15), partition p1b values less than (20))", alterErr) + wait <- true + tk.MustExec(`insert into t values (14, "14", 14),(15, "15",15)`) + tk.MustGetErrCode(`insert into t values (11, "11", 11),(12,"duplicate PK 💥", 13)`, errno.ErrDupEntry) + tk.MustExec(`admin check table t`) + wait <- true + require.NoError(t, <-alterErr) + tk.MustExec(`admin check table t`) + tk.MustQuery(`select * from t where c between 10 and 22`).Sort().Check(testkit.Rows(""+ + "12 12 21", + "14 14 14", + "15 15 15")) + tk.MustQuery(`show create table t`).Check(testkit.Rows("" + + "t CREATE TABLE `t` (\n" + + " `a` int(10) unsigned NOT NULL,\n" + + " `b` varchar(255) DEFAULT NULL,\n" + + " `c` int(11) DEFAULT NULL,\n" + + " PRIMARY KEY (`a`) /*T![clustered_index] CLUSTERED */,\n" + + " KEY `b` (`b`),\n" + + " KEY `c` (`c`,`b`)\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin\n" + + "PARTITION BY RANGE (`a`)\n" + + "(PARTITION `p0` VALUES LESS THAN (10),\n" + + " PARTITION `p1a` VALUES LESS THAN (15),\n" + + " PARTITION `p1b` VALUES LESS THAN (20),\n" + + " PARTITION `pMax` VALUES LESS THAN (MAXVALUE))")) +} + +func TestReorgPartitionRollback(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + schemaName := "ReorgPartRollback" + tk.MustExec("create database " + schemaName) + tk.MustExec("use " + schemaName) + tk.MustExec(`create table t (a int unsigned PRIMARY KEY, b varchar(255), c int, key (b), key (c,b))` + + ` partition by range (a) ` + + `(partition p0 values less than (10),` + + ` partition p1 values less than (20),` + + ` partition pMax values less than (MAXVALUE))`) + tk.MustExec(`insert into t values (1,"1",1), (12,"12",21),(23,"23",32),(34,"34",43),(45,"45",54),(56,"56",65)`) + // TODO: Check that there are no additional placement rules, + // bundles, or ranges with non-completed tableIDs + // (partitions used during reorg, but was dropped) + testfailpoint.Enable(t, "github.com/pingcap/tidb/pkg/ddl/mockUpdateVersionAndTableInfoErr", `return(1)`) + tk.MustExecToErr("alter table t reorganize partition p1 into (partition p1a values less than (15), partition p1b values less than (20))") + tk.MustExec(`admin check table t`) + testfailpoint.Disable(t, "github.com/pingcap/tidb/pkg/ddl/mockUpdateVersionAndTableInfoErr") + ctx := tk.Session() + is := domain.GetDomain(ctx).InfoSchema() + tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr(schemaName), pmodel.NewCIStr("t")) + require.NoError(t, err) + noNewTablesAfter(t, tk, ctx, tbl, "Reorganize rollback") + testfailpoint.Enable(t, "github.com/pingcap/tidb/pkg/ddl/reorgPartitionAfterDataCopy", `return(true)`) + defer func() { + testfailpoint.Disable(t, "github.com/pingcap/tidb/pkg/ddl/reorgPartitionAfterDataCopy") + }() + tk.MustExecToErr("alter table t reorganize partition p1 into (partition p1a values less than (15), partition p1b values less than (20))") + tk.MustExec(`admin check table t`) + tk.MustQuery(`show create table t`).Check(testkit.Rows("" + + "t CREATE TABLE `t` (\n" + + " `a` int(10) unsigned NOT NULL,\n" + + " `b` varchar(255) DEFAULT NULL,\n" + + " `c` int(11) DEFAULT NULL,\n" + + " PRIMARY KEY (`a`) /*T![clustered_index] CLUSTERED */,\n" + + " KEY `b` (`b`),\n" + + " KEY `c` (`c`,`b`)\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin\n" + + "PARTITION BY RANGE (`a`)\n" + + "(PARTITION `p0` VALUES LESS THAN (10),\n" + + " PARTITION `p1` VALUES LESS THAN (20),\n" + + " PARTITION `pMax` VALUES LESS THAN (MAXVALUE))")) + + tbl, err = is.TableByName(context.Background(), pmodel.NewCIStr(schemaName), pmodel.NewCIStr("t")) + require.NoError(t, err) + noNewTablesAfter(t, tk, ctx, tbl, "Reorganize rollback") +} + +func TestPartitionByColumnChecks(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + cols := "(i int, f float, c char(20), b bit(2), b32 bit(32), b64 bit(64), d date, dt datetime, dt6 datetime(6), ts timestamp, ts6 timestamp(6), j json)" + vals := `(1, 2.2, "A and c", b'10', b'10001000100010001000100010001000', b'1000100010001000100010001000100010001000100010001000100010001000', '2024-09-24', '2024-09-24 13:01:02', '2024-09-24 13:01:02.123456', '2024-09-24 13:01:02', '2024-09-24 13:01:02.123456', '{"key1": "value1", "key2": "value2"}')` + tk.MustExec(`create table t ` + cols) + testCases := []struct { + partClause string + err error + }{ + {"key (c) partitions 2", nil}, + {"key (j) partitions 2", dbterror.ErrNotAllowedTypeInPartition}, + {"list (c) (partition pDef default)", dbterror.ErrNotAllowedTypeInPartition}, + {"list (b) (partition pDef default)", nil}, + {"list (f) (partition pDef default)", dbterror.ErrNotAllowedTypeInPartition}, + {"list (j) (partition pDef default)", dbterror.ErrNotAllowedTypeInPartition}, + {"list columns (b) (partition pDef default)", dbterror.ErrNotAllowedTypeInPartition}, + {"list columns (f) (partition pDef default)", dbterror.ErrNotAllowedTypeInPartition}, + {"list columns (ts) (partition pDef default)", dbterror.ErrNotAllowedTypeInPartition}, + {"list columns (j) (partition pDef default)", dbterror.ErrNotAllowedTypeInPartition}, + {"hash (year(ts)) partitions 2", dbterror.ErrWrongExprInPartitionFunc}, + {"hash (ts) partitions 2", dbterror.ErrNotAllowedTypeInPartition}, + {"hash (ts6) partitions 2", dbterror.ErrNotAllowedTypeInPartition}, + {"hash (d) partitions 2", dbterror.ErrNotAllowedTypeInPartition}, + {"hash (f) partitions 2", dbterror.ErrNotAllowedTypeInPartition}, + {"range (c) (partition pMax values less than (maxvalue))", dbterror.ErrNotAllowedTypeInPartition}, + {"range (f) (partition pMax values less than (maxvalue))", dbterror.ErrNotAllowedTypeInPartition}, + {"range (d) (partition pMax values less than (maxvalue))", dbterror.ErrNotAllowedTypeInPartition}, + {"range (dt) (partition pMax values less than (maxvalue))", dbterror.ErrNotAllowedTypeInPartition}, + {"range (dt6) (partition pMax values less than (maxvalue))", dbterror.ErrNotAllowedTypeInPartition}, + {"range (ts) (partition pMax values less than (maxvalue))", dbterror.ErrNotAllowedTypeInPartition}, + {"range (ts6) (partition pMax values less than (maxvalue))", dbterror.ErrNotAllowedTypeInPartition}, + {"range (j) (partition pMax values less than (maxvalue))", dbterror.ErrNotAllowedTypeInPartition}, + {"range columns (b) (partition pMax values less than (maxvalue))", dbterror.ErrNotAllowedTypeInPartition}, + {"range columns (b64) (partition pMax values less than (maxvalue))", dbterror.ErrNotAllowedTypeInPartition}, + {"range columns (c) (partition pMax values less than (maxvalue))", nil}, + {"range columns (f) (partition pMax values less than (maxvalue))", dbterror.ErrNotAllowedTypeInPartition}, + {"range columns (d) (partition pMax values less than (maxvalue))", nil}, + {"range columns (dt) (partition pMax values less than (maxvalue))", nil}, + {"range columns (dt6) (partition pMax values less than (maxvalue))", nil}, + {"range columns (ts) (partition pMax values less than (maxvalue))", dbterror.ErrNotAllowedTypeInPartition}, + {"range columns (ts6) (partition pMax values less than (maxvalue))", dbterror.ErrNotAllowedTypeInPartition}, + {"range columns (j) (partition pMax values less than (maxvalue))", dbterror.ErrNotAllowedTypeInPartition}, + } + for _, testCase := range testCases { + err := tk.ExecToErr(`create table tt ` + cols + ` partition by ` + testCase.partClause) + require.ErrorIs(t, err, testCase.err, testCase.partClause) + if testCase.err == nil { + tk.MustExec(`drop table tt`) + } + err = tk.ExecToErr(`alter table t partition by ` + testCase.partClause) + require.ErrorIs(t, err, testCase.err) + } + + // Not documented or tested!! + // KEY - Allows more types than documented, should be OK! + tk.MustExec(`create table kb ` + cols + ` partition by key(b) partitions 2`) + tk.MustExec(`create table kf ` + cols + ` partition by key(f) partitions 2`) + tk.MustExec(`create table kts ` + cols + ` partition by key(ts) partitions 2`) + tk.MustExec(`create table hb ` + cols + ` partition by hash(b) partitions 2`) + tk.MustExec(`insert into hb values ` + vals) + tk.MustQuery(`select count(*) from hb where b = b'10'`).Check(testkit.Rows("1")) + tk.MustExec(`alter table hb partition by hash(b) partitions 3`) + tk.MustExec(`insert into hb values ` + vals) + tk.MustQuery(`select count(*) from hb where b = b'10'`).Check(testkit.Rows("2")) + tk.MustExec(`create table hb32 ` + cols + ` partition by hash(b32) partitions 2`) + tk.MustExec(`insert into hb32 values ` + vals) + tk.MustExec(`alter table hb32 partition by hash(b32) partitions 3`) + tk.MustExec(`insert into hb32 values ` + vals) + tk.MustExec(`create table rb ` + cols + ` partition by range (b) (partition pMax values less than (MAXVALUE))`) + tk.MustExec(`insert into rb values ` + vals) + tk.MustExec(`alter table rb partition by range(b) (partition pMax values less than (MAXVALUE))`) + tk.MustExec(`insert into rb values ` + vals) + tk.MustExec(`create table rb32 ` + cols + ` partition by range (b32) (partition pMax values less than (MAXVALUE))`) + tk.MustExec(`insert into rb32 values ` + vals) + tk.MustExec(`alter table rb32 partition by range(b32) (partition pMax values less than (MAXVALUE))`) + tk.MustExec(`insert into rb32 values ` + vals) + tk.MustExec(`create table rb64 ` + cols + ` partition by range (b64) (partition pMax values less than (MAXVALUE))`) + tk.MustExec(`insert into rb64 values ` + vals) + tk.MustExec(`alter table rb64 partition by range(b64) (partition pMax values less than (MAXVALUE))`) + tk.MustExec(`insert into rb64 values ` + vals) +} + +func TestPartitionIssue56634(t *testing.T) { + testfailpoint.Enable(t, "github.com/pingcap/tidb/pkg/ddl/updateVersionAndTableInfoErrInStateDeleteReorganization", `4*return(1)`) + + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t (a int)") + // Changed, since StatePublic can no longer rollback! + tk.MustExec("alter table t partition by range(a) (partition p1 values less than (20))") +} diff --git a/pkg/meta/model/job_args.go b/pkg/meta/model/job_args.go new file mode 100644 index 0000000000000..4669a3b9b76b3 --- /dev/null +++ b/pkg/meta/model/job_args.go @@ -0,0 +1,1719 @@ +// Copyright 2024 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package model + +import ( + "encoding/json" + + "github.com/pingcap/errors" + "github.com/pingcap/tidb/pkg/parser/ast" + pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/mysql" + "github.com/pingcap/tidb/pkg/util/intest" + pdhttp "github.com/tikv/pd/client/http" +) + +// AutoIDGroup represents a group of auto IDs of a specific table. +type AutoIDGroup struct { + RowID int64 + IncrementID int64 + RandomID int64 +} + +// RecoverTableInfo contains information needed by DDL.RecoverTable. +type RecoverTableInfo struct { + SchemaID int64 + TableInfo *TableInfo + DropJobID int64 + SnapshotTS uint64 + AutoIDs AutoIDGroup + OldSchemaName string + OldTableName string +} + +// RecoverSchemaInfo contains information needed by DDL.RecoverSchema. +type RecoverSchemaInfo struct { + *DBInfo + RecoverTableInfos []*RecoverTableInfo + // LoadTablesOnExecute is the new logic to avoid a large RecoverTabsInfo can't be + // persisted. If it's true, DDL owner will recover RecoverTabsInfo instead of the + // job submit node. + LoadTablesOnExecute bool + DropJobID int64 + SnapshotTS uint64 + OldSchemaName pmodel.CIStr +} + +// getOrDecodeArgsV1 get the args v1 from job, if the job.Args is nil, decode job.RawArgs +// and cache in job.Args. +// as there is no way to create a generic struct with a type parameter in Go, we +// have to pass one instance of the struct to the function. +func getOrDecodeArgsV1[T JobArgs](args T, job *Job) (T, error) { + intest.Assert(job.Version == JobVersion1, "job version is not v1") + var v T + if err := args.decodeV1(job); err != nil { + return v, errors.Trace(err) + } + return args, nil +} + +// getOrDecodeArgsV2 get the args v2 from job, if the job.Args is nil, decode job.RawArgs +// and cache in job.Args. +func getOrDecodeArgsV2[T JobArgs](job *Job) (T, error) { + intest.Assert(job.Version == JobVersion2, "job version is not v2") + if len(job.args) > 0 { + intest.Assert(len(job.args) == 1, "job args length is not 1") + return job.args[0].(T), nil + } + var v T + if err := json.Unmarshal(job.RawArgs, &v); err != nil { + return v, errors.Trace(err) + } + job.args = []any{v} + return v, nil +} + +func getOrDecodeArgs[T JobArgs](args T, job *Job) (T, error) { + if job.Version == JobVersion1 { + return getOrDecodeArgsV1[T](args, job) + } + return getOrDecodeArgsV2[T](job) +} + +// JobArgs is the interface for job arguments. +type JobArgs interface { + // getArgsV1 gets the job args for v1. we make it private to avoid calling it + // directly, use Job.FillArgs to fill the job args. + getArgsV1(job *Job) []any + decodeV1(job *Job) error +} + +// FinishedJobArgs is the interface for finished job arguments. +// in most cases, job args are cleared out after the job is finished, but some jobs +// will write some args back to the job for other components. +type FinishedJobArgs interface { + JobArgs + // getFinishedArgsV1 fills the job args for finished job. we make it private + // to avoid calling it directly, use Job.FillFinishedArgs to fill the job args. + getFinishedArgsV1(job *Job) []any +} + +// EmptyArgs is the args for ddl job with no args. +type EmptyArgs struct{} + +func (*EmptyArgs) getArgsV1(*Job) []any { + return nil +} + +func (*EmptyArgs) decodeV1(*Job) error { + return nil +} + +// CreateSchemaArgs is the arguments for create schema job. +type CreateSchemaArgs struct { + DBInfo *DBInfo `json:"db_info,omitempty"` +} + +func (a *CreateSchemaArgs) getArgsV1(*Job) []any { + return []any{a.DBInfo} +} + +func (a *CreateSchemaArgs) decodeV1(job *Job) error { + a.DBInfo = &DBInfo{} + return errors.Trace(job.decodeArgs(a.DBInfo)) +} + +// GetCreateSchemaArgs gets the args for create schema job. +func GetCreateSchemaArgs(job *Job) (*CreateSchemaArgs, error) { + return getOrDecodeArgs[*CreateSchemaArgs](&CreateSchemaArgs{}, job) +} + +// DropSchemaArgs is the arguments for drop schema job. +type DropSchemaArgs struct { + // this is the args for job submission, it's invalid if the job is finished. + FKCheck bool `json:"fk_check,omitempty"` + // this is the args for finished job. this list include all partition IDs too. + AllDroppedTableIDs []int64 `json:"all_dropped_table_ids,omitempty"` +} + +func (a *DropSchemaArgs) getArgsV1(*Job) []any { + return []any{a.FKCheck} +} + +func (a *DropSchemaArgs) getFinishedArgsV1(*Job) []any { + return []any{a.AllDroppedTableIDs} +} + +func (a *DropSchemaArgs) decodeV1(job *Job) error { + return job.decodeArgs(&a.FKCheck) +} + +// GetDropSchemaArgs gets the args for drop schema job. +func GetDropSchemaArgs(job *Job) (*DropSchemaArgs, error) { + return getOrDecodeArgs[*DropSchemaArgs](&DropSchemaArgs{}, job) +} + +// GetFinishedDropSchemaArgs gets the args for drop schema job after the job is finished. +func GetFinishedDropSchemaArgs(job *Job) (*DropSchemaArgs, error) { + if job.Version == JobVersion1 { + var physicalTableIDs []int64 + if err := job.decodeArgs(&physicalTableIDs); err != nil { + return nil, err + } + return &DropSchemaArgs{AllDroppedTableIDs: physicalTableIDs}, nil + } + return getOrDecodeArgsV2[*DropSchemaArgs](job) +} + +// ModifySchemaArgs is the arguments for modify schema job. +type ModifySchemaArgs struct { + // below 2 are used for modify schema charset and collate. + ToCharset string `json:"to_charset,omitempty"` + ToCollate string `json:"to_collate,omitempty"` + // used for modify schema placement policy. + // might be nil, means set it to default. + PolicyRef *PolicyRefInfo `json:"policy_ref,omitempty"` +} + +func (a *ModifySchemaArgs) getArgsV1(job *Job) []any { + if job.Type == ActionModifySchemaCharsetAndCollate { + return []any{a.ToCharset, a.ToCollate} + } + return []any{a.PolicyRef} +} + +func (a *ModifySchemaArgs) decodeV1(job *Job) error { + if job.Type == ActionModifySchemaCharsetAndCollate { + return errors.Trace(job.decodeArgs(&a.ToCharset, &a.ToCollate)) + } + return errors.Trace(job.decodeArgs(&a.PolicyRef)) +} + +// GetModifySchemaArgs gets the modify schema args. +func GetModifySchemaArgs(job *Job) (*ModifySchemaArgs, error) { + return getOrDecodeArgs[*ModifySchemaArgs](&ModifySchemaArgs{}, job) +} + +// CreateTableArgs is the arguments for create table/view/sequence job. +type CreateTableArgs struct { + TableInfo *TableInfo `json:"table_info,omitempty"` + // below 2 are used for create view. + OnExistReplace bool `json:"on_exist_replace,omitempty"` + OldViewTblID int64 `json:"old_view_tbl_id,omitempty"` + // used for create table. + FKCheck bool `json:"fk_check,omitempty"` +} + +func (a *CreateTableArgs) getArgsV1(job *Job) []any { + switch job.Type { + case ActionCreateTable: + return []any{a.TableInfo, a.FKCheck} + case ActionCreateView: + return []any{a.TableInfo, a.OnExistReplace, a.OldViewTblID} + case ActionCreateSequence: + return []any{a.TableInfo} + } + return nil +} + +func (a *CreateTableArgs) decodeV1(job *Job) error { + a.TableInfo = &TableInfo{} + switch job.Type { + case ActionCreateTable: + return errors.Trace(job.decodeArgs(a.TableInfo, &a.FKCheck)) + case ActionCreateView: + return errors.Trace(job.decodeArgs(a.TableInfo, &a.OnExistReplace, &a.OldViewTblID)) + case ActionCreateSequence: + return errors.Trace(job.decodeArgs(a.TableInfo)) + } + return nil +} + +// GetCreateTableArgs gets the create-table args. +func GetCreateTableArgs(job *Job) (*CreateTableArgs, error) { + return getOrDecodeArgs[*CreateTableArgs](&CreateTableArgs{}, job) +} + +// BatchCreateTableArgs is the arguments for batch create table job. +type BatchCreateTableArgs struct { + Tables []*CreateTableArgs `json:"tables,omitempty"` +} + +func (a *BatchCreateTableArgs) getArgsV1(*Job) []any { + infos := make([]*TableInfo, 0, len(a.Tables)) + for _, info := range a.Tables { + infos = append(infos, info.TableInfo) + } + return []any{infos, a.Tables[0].FKCheck} +} + +func (a *BatchCreateTableArgs) decodeV1(job *Job) error { + var ( + tableInfos []*TableInfo + fkCheck bool + ) + if err := job.decodeArgs(&tableInfos, &fkCheck); err != nil { + return errors.Trace(err) + } + a.Tables = make([]*CreateTableArgs, 0, len(tableInfos)) + for _, info := range tableInfos { + a.Tables = append(a.Tables, &CreateTableArgs{TableInfo: info, FKCheck: fkCheck}) + } + return nil +} + +// GetBatchCreateTableArgs gets the batch create-table args. +func GetBatchCreateTableArgs(job *Job) (*BatchCreateTableArgs, error) { + return getOrDecodeArgs[*BatchCreateTableArgs](&BatchCreateTableArgs{}, job) +} + +// DropTableArgs is the arguments for drop table/view/sequence job. +// when dropping multiple objects, each object will have a separate job +type DropTableArgs struct { + // below fields are only for drop table. + // when dropping multiple tables, the Identifiers is the same, but each drop-table + // runs in a separate job. + Identifiers []ast.Ident `json:"identifiers,omitempty"` + FKCheck bool `json:"fk_check,omitempty"` + + // below fields are finished job args + StartKey []byte `json:"start_key,omitempty"` + OldPartitionIDs []int64 `json:"old_partition_ids,omitempty"` + OldRuleIDs []string `json:"old_rule_ids,omitempty"` +} + +func (a *DropTableArgs) getArgsV1(job *Job) []any { + // only drop-table job has in args, drop view/sequence job has no args. + if job.Type == ActionDropTable { + return []any{a.Identifiers, a.FKCheck} + } + return nil +} + +func (a *DropTableArgs) getFinishedArgsV1(*Job) []any { + return []any{a.StartKey, a.OldPartitionIDs, a.OldRuleIDs} +} + +func (a *DropTableArgs) decodeV1(job *Job) error { + if job.Type == ActionDropTable { + return job.decodeArgs(&a.Identifiers, &a.FKCheck) + } + return nil +} + +// GetDropTableArgs gets the drop-table args. +func GetDropTableArgs(job *Job) (*DropTableArgs, error) { + return getOrDecodeArgs[*DropTableArgs](&DropTableArgs{}, job) +} + +// GetFinishedDropTableArgs gets the drop-table args after the job is finished. +func GetFinishedDropTableArgs(job *Job) (*DropTableArgs, error) { + if job.Version == JobVersion1 { + var ( + startKey []byte + oldPartitionIDs []int64 + oldRuleIDs []string + ) + if err := job.decodeArgs(&startKey, &oldPartitionIDs, &oldRuleIDs); err != nil { + return nil, errors.Trace(err) + } + return &DropTableArgs{ + StartKey: startKey, + OldPartitionIDs: oldPartitionIDs, + OldRuleIDs: oldRuleIDs, + }, nil + } + return getOrDecodeArgsV2[*DropTableArgs](job) +} + +// TruncateTableArgs is the arguments for truncate table/partition job. +type TruncateTableArgs struct { + FKCheck bool `json:"fk_check,omitempty"` + NewTableID int64 `json:"new_table_id,omitempty"` + NewPartitionIDs []int64 `json:"new_partition_ids,omitempty"` + OldPartitionIDs []int64 `json:"old_partition_ids,omitempty"` + + // context vars + NewPartIDsWithPolicy []int64 `json:"-"` + OldPartIDsWithPolicy []int64 `json:"-"` + ShouldUpdateAffectedPartitions bool `json:"-"` +} + +func (a *TruncateTableArgs) getArgsV1(job *Job) []any { + if job.Type == ActionTruncateTable { + // Args[0] is the new table ID, args[2] is the ids for table partitions, we + // add a placeholder here, they will be filled by job submitter. + // the last param is not required for execution, we need it to calculate + // number of new IDs to generate. + return []any{a.NewTableID, a.FKCheck, a.NewPartitionIDs, len(a.OldPartitionIDs)} + } + return []any{a.OldPartitionIDs, a.NewPartitionIDs} +} + +func (a *TruncateTableArgs) decodeV1(job *Job) error { + if job.Type == ActionTruncateTable { + return errors.Trace(job.decodeArgs(&a.NewTableID, &a.FKCheck, &a.NewPartitionIDs)) + } + return errors.Trace(job.decodeArgs(&a.OldPartitionIDs, &a.NewPartitionIDs)) +} + +func (a *TruncateTableArgs) getFinishedArgsV1(job *Job) []any { + if job.Type == ActionTruncateTable { + // the first param is the start key of the old table, it's not used anywhere + // now, so we fill an empty byte slice here. + // we can call tablecodec.EncodeTablePrefix(tableID) to get it. + return []any{[]byte{}, a.OldPartitionIDs} + } + return []any{a.OldPartitionIDs} +} + +// GetTruncateTableArgs gets the truncate table args. +func GetTruncateTableArgs(job *Job) (*TruncateTableArgs, error) { + return getOrDecodeArgs[*TruncateTableArgs](&TruncateTableArgs{}, job) +} + +// GetFinishedTruncateTableArgs gets the truncate table args after the job is finished. +func GetFinishedTruncateTableArgs(job *Job) (*TruncateTableArgs, error) { + if job.Version == JobVersion1 { + if job.Type == ActionTruncateTable { + var startKey []byte + var oldPartitionIDs []int64 + if err := job.decodeArgs(&startKey, &oldPartitionIDs); err != nil { + return nil, errors.Trace(err) + } + return &TruncateTableArgs{OldPartitionIDs: oldPartitionIDs}, nil + } + var oldPartitionIDs []int64 + if err := job.decodeArgs(&oldPartitionIDs); err != nil { + return nil, errors.Trace(err) + } + return &TruncateTableArgs{OldPartitionIDs: oldPartitionIDs}, nil + } + + return getOrDecodeArgsV2[*TruncateTableArgs](job) +} + +// TableIDIndexID contains TableID+IndexID of index ranges to be deleted +type TableIDIndexID struct { + TableID int64 + IndexID int64 +} + +// TablePartitionArgs is the arguments for table partition related jobs, including: +// - ActionAlterTablePartitioning +// - ActionRemovePartitioning +// - ActionReorganizePartition +// - ActionAddTablePartition: don't have finished args if success. +// - ActionDropTablePartition +// +// when rolling back, args of ActionAddTablePartition will be changed to be the same +// as ActionDropTablePartition, and it will have finished args, but not used anywhere, +// for other types, their args will be decoded as if its args is the same of ActionDropTablePartition. +type TablePartitionArgs struct { + PartNames []string `json:"part_names,omitempty"` + PartInfo *PartitionInfo `json:"part_info,omitempty"` + + // set on finished + OldPhysicalTblIDs []int64 `json:"old_physical_tbl_ids,omitempty"` + OldGlobalIndexes []TableIDIndexID `json:"old_global_indexes,omitempty"` + + // runtime info + NewPartitionIDs []int64 `json:"-"` +} + +func (a *TablePartitionArgs) getArgsV1(job *Job) []any { + if job.Type == ActionAddTablePartition { + return []any{a.PartInfo} + } else if job.Type == ActionDropTablePartition { + return []any{a.PartNames} + } + return []any{a.PartNames, a.PartInfo} +} + +func (a *TablePartitionArgs) getFinishedArgsV1(job *Job) []any { + intest.Assert(job.Type != ActionAddTablePartition || job.State == JobStateRollbackDone, + "add table partition job should not call getFinishedArgsV1 if not rollback") + return []any{a.OldPhysicalTblIDs, a.OldGlobalIndexes} +} + +func (a *TablePartitionArgs) decodeV1(job *Job) error { + var ( + partNames []string + partInfo = &PartitionInfo{} + ) + if job.Type == ActionAddTablePartition { + if job.State == JobStateRollingback { + if err := job.decodeArgs(&partNames); err != nil { + return err + } + } else { + if err := job.decodeArgs(partInfo); err != nil { + return err + } + } + } else if job.Type == ActionDropTablePartition { + if err := job.decodeArgs(&partNames); err != nil { + return err + } + } else { + if err := job.decodeArgs(&partNames, partInfo); err != nil { + return err + } + } + a.PartNames = partNames + a.PartInfo = partInfo + return nil +} + +// GetTablePartitionArgs gets the table partition args. +func GetTablePartitionArgs(job *Job) (*TablePartitionArgs, error) { + args, err := getOrDecodeArgs[*TablePartitionArgs](&TablePartitionArgs{}, job) + if err != nil { + return nil, errors.Trace(err) + } + // when it's ActionDropTablePartition job, or roll-backing a ActionAddTablePartition + // job, our execution part expect a non-nil PartInfo. + if args.PartInfo == nil { + args.PartInfo = &PartitionInfo{} + } + return args, nil +} + +// GetFinishedTablePartitionArgs gets the table partition args after the job is finished. +func GetFinishedTablePartitionArgs(job *Job) (*TablePartitionArgs, error) { + if job.Version == JobVersion1 { + var oldPhysicalTblIDs []int64 + var oldIndexes []TableIDIndexID + if err := job.decodeArgs(&oldPhysicalTblIDs, &oldIndexes); err != nil { + return nil, errors.Trace(err) + } + return &TablePartitionArgs{OldPhysicalTblIDs: oldPhysicalTblIDs, OldGlobalIndexes: oldIndexes}, nil + } + return getOrDecodeArgsV2[*TablePartitionArgs](job) +} + +// FillRollbackArgsForAddPartition fills the rollback args for add partition job. +// see details in TablePartitionArgs. +func FillRollbackArgsForAddPartition(job *Job, args *TablePartitionArgs) { + intest.Assert(job.Type == ActionAddTablePartition, "only for add partition job") + fake := &Job{ + Version: job.Version, + Type: ActionDropTablePartition, + } + // PartInfo cannot be saved, onDropTablePartition expects that PartInfo is empty + // in this case + fake.FillArgs(&TablePartitionArgs{ + PartNames: args.PartNames, + }) + job.args = fake.args +} + +// ExchangeTablePartitionArgs is the arguments for exchange table partition job. +// pt: the partition table to exchange +// nt: the non-partition table to exchange with +type ExchangeTablePartitionArgs struct { + PartitionID int64 `json:"partition_id,omitempty"` + PTSchemaID int64 `json:"pt_schema_id,omitempty"` + PTTableID int64 `json:"pt_table_id,omitempty"` + PartitionName string `json:"partition_name,omitempty"` + WithValidation bool `json:"with_validation,omitempty"` +} + +func (a *ExchangeTablePartitionArgs) getArgsV1(*Job) []any { + return []any{a.PartitionID, a.PTSchemaID, a.PTTableID, a.PartitionName, a.WithValidation} +} + +func (a *ExchangeTablePartitionArgs) decodeV1(job *Job) error { + return errors.Trace(job.decodeArgs(&a.PartitionID, &a.PTSchemaID, &a.PTTableID, &a.PartitionName, &a.WithValidation)) +} + +// GetExchangeTablePartitionArgs gets the exchange table partition args. +func GetExchangeTablePartitionArgs(job *Job) (*ExchangeTablePartitionArgs, error) { + return getOrDecodeArgs[*ExchangeTablePartitionArgs](&ExchangeTablePartitionArgs{}, job) +} + +// AlterTablePartitionArgs is the arguments for alter table partition job. +// it's used for: +// - ActionAlterTablePartitionAttributes +// - ActionAlterTablePartitionPlacement +type AlterTablePartitionArgs struct { + PartitionID int64 `json:"partition_id,omitempty"` + LabelRule *pdhttp.LabelRule `json:"label_rule,omitempty"` + PolicyRefInfo *PolicyRefInfo `json:"policy_ref_info,omitempty"` +} + +func (a *AlterTablePartitionArgs) getArgsV1(job *Job) []any { + if job.Type == ActionAlterTablePartitionAttributes { + return []any{a.PartitionID, a.LabelRule} + } + return []any{a.PartitionID, a.PolicyRefInfo} +} + +func (a *AlterTablePartitionArgs) decodeV1(job *Job) error { + if job.Type == ActionAlterTablePartitionAttributes { + return errors.Trace(job.decodeArgs(&a.PartitionID, &a.LabelRule)) + } + return errors.Trace(job.decodeArgs(&a.PartitionID, &a.PolicyRefInfo)) +} + +// GetAlterTablePartitionArgs gets the alter table partition args. +func GetAlterTablePartitionArgs(job *Job) (*AlterTablePartitionArgs, error) { + return getOrDecodeArgs[*AlterTablePartitionArgs](&AlterTablePartitionArgs{}, job) +} + +// RenameTableArgs is the arguments for rename table DDL job. +// It's also used for rename tables. +type RenameTableArgs struct { + // for Args + OldSchemaID int64 `json:"old_schema_id,omitempty"` + OldSchemaName pmodel.CIStr `json:"old_schema_name,omitempty"` + NewTableName pmodel.CIStr `json:"new_table_name,omitempty"` + + // for rename tables + OldTableName pmodel.CIStr `json:"old_table_name,omitempty"` + NewSchemaID int64 `json:"new_schema_id,omitempty"` + TableID int64 `json:"table_id,omitempty"` + + // runtime info + OldSchemaIDForSchemaDiff int64 `json:"-"` +} + +func (rt *RenameTableArgs) getArgsV1(*Job) []any { + return []any{rt.OldSchemaID, rt.NewTableName, rt.OldSchemaName} +} + +func (rt *RenameTableArgs) decodeV1(job *Job) error { + return errors.Trace(job.decodeArgs(&rt.OldSchemaID, &rt.NewTableName, &rt.OldSchemaName)) +} + +// GetRenameTableArgs get the arguments from job. +func GetRenameTableArgs(job *Job) (*RenameTableArgs, error) { + args, err := getOrDecodeArgs[*RenameTableArgs](&RenameTableArgs{}, job) + if err != nil { + return nil, errors.Trace(err) + } + // NewSchemaID is used for checkAndRenameTables, which is not set for rename table. + args.NewSchemaID = job.SchemaID + return args, nil +} + +// ResourceGroupArgs is the arguments for resource group job. +type ResourceGroupArgs struct { + // for DropResourceGroup we only use it to store the name, other fields are invalid. + RGInfo *ResourceGroupInfo `json:"rg_info,omitempty"` +} + +func (a *ResourceGroupArgs) getArgsV1(job *Job) []any { + if job.Type == ActionCreateResourceGroup { + // what's the second parameter for? we keep it for compatibility. + return []any{a.RGInfo, false} + } else if job.Type == ActionAlterResourceGroup { + return []any{a.RGInfo} + } else if job.Type == ActionDropResourceGroup { + // it's not used anywhere. + return []any{a.RGInfo.Name} + } + return nil +} + +func (a *ResourceGroupArgs) decodeV1(job *Job) error { + a.RGInfo = &ResourceGroupInfo{} + if job.Type == ActionCreateResourceGroup || job.Type == ActionAlterResourceGroup { + return errors.Trace(job.decodeArgs(a.RGInfo)) + } else if job.Type == ActionDropResourceGroup { + return errors.Trace(job.decodeArgs(&a.RGInfo.Name)) + } + return nil +} + +// GetResourceGroupArgs gets the resource group args. +func GetResourceGroupArgs(job *Job) (*ResourceGroupArgs, error) { + return getOrDecodeArgs[*ResourceGroupArgs](&ResourceGroupArgs{}, job) +} + +// RebaseAutoIDArgs is the arguments for ActionRebaseAutoID DDL. +// It is also for ActionRebaseAutoRandomBase. +type RebaseAutoIDArgs struct { + NewBase int64 `json:"new_base,omitempty"` + Force bool `json:"force,omitempty"` +} + +func (a *RebaseAutoIDArgs) getArgsV1(*Job) []any { + return []any{a.NewBase, a.Force} +} + +func (a *RebaseAutoIDArgs) decodeV1(job *Job) error { + return errors.Trace(job.decodeArgs(&a.NewBase, &a.Force)) +} + +// GetRebaseAutoIDArgs the args for ActionRebaseAutoID/ActionRebaseAutoRandomBase ddl. +func GetRebaseAutoIDArgs(job *Job) (*RebaseAutoIDArgs, error) { + return getOrDecodeArgs[*RebaseAutoIDArgs](&RebaseAutoIDArgs{}, job) +} + +// ModifyTableCommentArgs is the arguments for ActionModifyTableComment ddl. +type ModifyTableCommentArgs struct { + Comment string `json:"comment,omitempty"` +} + +func (a *ModifyTableCommentArgs) getArgsV1(*Job) []any { + return []any{a.Comment} +} + +func (a *ModifyTableCommentArgs) decodeV1(job *Job) error { + return errors.Trace(job.decodeArgs(&a.Comment)) +} + +// GetModifyTableCommentArgs gets the args for ActionModifyTableComment. +func GetModifyTableCommentArgs(job *Job) (*ModifyTableCommentArgs, error) { + return getOrDecodeArgs[*ModifyTableCommentArgs](&ModifyTableCommentArgs{}, job) +} + +// ModifyTableCharsetAndCollateArgs is the arguments for ActionModifyTableCharsetAndCollate ddl. +type ModifyTableCharsetAndCollateArgs struct { + ToCharset string `json:"to_charset,omitempty"` + ToCollate string `json:"to_collate,omitempty"` + NeedsOverwriteCols bool `json:"needs_overwrite_cols,omitempty"` +} + +func (a *ModifyTableCharsetAndCollateArgs) getArgsV1(*Job) []any { + return []any{a.ToCharset, a.ToCollate, a.NeedsOverwriteCols} +} + +func (a *ModifyTableCharsetAndCollateArgs) decodeV1(job *Job) error { + return errors.Trace(job.decodeArgs(&a.ToCharset, &a.ToCollate, &a.NeedsOverwriteCols)) +} + +// GetModifyTableCharsetAndCollateArgs gets the args for ActionModifyTableCharsetAndCollate ddl. +func GetModifyTableCharsetAndCollateArgs(job *Job) (*ModifyTableCharsetAndCollateArgs, error) { + return getOrDecodeArgs[*ModifyTableCharsetAndCollateArgs](&ModifyTableCharsetAndCollateArgs{}, job) +} + +// AlterIndexVisibilityArgs is the arguments for ActionAlterIndexVisibility ddl. +type AlterIndexVisibilityArgs struct { + IndexName pmodel.CIStr `json:"index_name,omitempty"` + Invisible bool `json:"invisible,omitempty"` +} + +func (a *AlterIndexVisibilityArgs) getArgsV1(*Job) []any { + return []any{a.IndexName, a.Invisible} +} + +func (a *AlterIndexVisibilityArgs) decodeV1(job *Job) error { + return errors.Trace(job.decodeArgs(&a.IndexName, &a.Invisible)) +} + +// GetAlterIndexVisibilityArgs gets the args for AlterIndexVisibility ddl. +func GetAlterIndexVisibilityArgs(job *Job) (*AlterIndexVisibilityArgs, error) { + return getOrDecodeArgs[*AlterIndexVisibilityArgs](&AlterIndexVisibilityArgs{}, job) +} + +// AddForeignKeyArgs is the arguments for ActionAddForeignKey ddl. +type AddForeignKeyArgs struct { + FkInfo *FKInfo `json:"fk_info,omitempty"` + FkCheck bool `json:"fk_check,omitempty"` +} + +func (a *AddForeignKeyArgs) getArgsV1(*Job) []any { + return []any{a.FkInfo, a.FkCheck} +} + +func (a *AddForeignKeyArgs) decodeV1(job *Job) error { + return errors.Trace(job.decodeArgs(&a.FkInfo, &a.FkCheck)) +} + +// GetAddForeignKeyArgs get the args for AddForeignKey ddl. +func GetAddForeignKeyArgs(job *Job) (*AddForeignKeyArgs, error) { + return getOrDecodeArgs[*AddForeignKeyArgs](&AddForeignKeyArgs{}, job) +} + +// DropForeignKeyArgs is the arguments for DropForeignKey ddl. +type DropForeignKeyArgs struct { + FkName pmodel.CIStr `json:"fk_name,omitempty"` +} + +func (a *DropForeignKeyArgs) getArgsV1(*Job) []any { + return []any{a.FkName} +} + +func (a *DropForeignKeyArgs) decodeV1(job *Job) error { + return errors.Trace(job.decodeArgs(&a.FkName)) +} + +// GetDropForeignKeyArgs gets the args for DropForeignKey ddl. +func GetDropForeignKeyArgs(job *Job) (*DropForeignKeyArgs, error) { + return getOrDecodeArgs[*DropForeignKeyArgs](&DropForeignKeyArgs{}, job) +} + +// TableColumnArgs is the arguments for dropping column ddl or Adding column ddl. +type TableColumnArgs struct { + // follow items for add column. + Col *ColumnInfo `json:"column_info,omitempty"` + Pos *ast.ColumnPosition `json:"position,omitempty"` + Offset int `json:"offset,omitempty"` + // it's shared by add/drop column. + IgnoreExistenceErr bool `json:"ignore_existence_err,omitempty"` + + // for drop column. + // below 2 fields are filled during running. + IndexIDs []int64 `json:"index_ids,omitempty"` + PartitionIDs []int64 `json:"partition_ids,omitempty"` +} + +func (a *TableColumnArgs) getArgsV1(job *Job) []any { + if job.Type == ActionDropColumn { + return []any{a.Col.Name, a.IgnoreExistenceErr, a.IndexIDs, a.PartitionIDs} + } + return []any{a.Col, a.Pos, a.Offset, a.IgnoreExistenceErr} +} + +func (a *TableColumnArgs) decodeV1(job *Job) error { + a.Col = &ColumnInfo{} + a.Pos = &ast.ColumnPosition{} + + // when rollbacking add-columm, it's arguments is same as drop-column + if job.Type == ActionDropColumn || job.State == JobStateRollingback { + return errors.Trace(job.decodeArgs(&a.Col.Name, &a.IgnoreExistenceErr, &a.IndexIDs, &a.PartitionIDs)) + } + // for add column ddl. + return errors.Trace(job.decodeArgs(a.Col, a.Pos, &a.Offset, &a.IgnoreExistenceErr)) +} + +// FillRollBackArgsForAddColumn fills the args for rollback add column ddl. +func FillRollBackArgsForAddColumn(job *Job, args *TableColumnArgs) { + intest.Assert(job.Type == ActionAddColumn, "only for add column job") + fakeJob := &Job{ + Version: job.Version, + Type: ActionDropColumn, + } + fakeJob.FillArgs(args) + job.args = fakeJob.args +} + +// GetTableColumnArgs gets the args for dropping column ddl or Adding column ddl. +func GetTableColumnArgs(job *Job) (*TableColumnArgs, error) { + return getOrDecodeArgs[*TableColumnArgs](&TableColumnArgs{}, job) +} + +// RenameTablesArgs is the arguments for rename tables job. +type RenameTablesArgs struct { + RenameTableInfos []*RenameTableArgs `json:"rename_table_infos,omitempty"` +} + +func (a *RenameTablesArgs) getArgsV1(*Job) []any { + n := len(a.RenameTableInfos) + oldSchemaIDs := make([]int64, n) + oldSchemaNames := make([]pmodel.CIStr, n) + oldTableNames := make([]pmodel.CIStr, n) + newSchemaIDs := make([]int64, n) + newTableNames := make([]pmodel.CIStr, n) + tableIDs := make([]int64, n) + + for i, info := range a.RenameTableInfos { + oldSchemaIDs[i] = info.OldSchemaID + oldSchemaNames[i] = info.OldSchemaName + oldTableNames[i] = info.OldTableName + newSchemaIDs[i] = info.NewSchemaID + newTableNames[i] = info.NewTableName + tableIDs[i] = info.TableID + } + + // To make it compatible with previous create metas + return []any{oldSchemaIDs, newSchemaIDs, newTableNames, tableIDs, oldSchemaNames, oldTableNames} +} + +func (a *RenameTablesArgs) decodeV1(job *Job) error { + var ( + oldSchemaIDs []int64 + oldSchemaNames []pmodel.CIStr + oldTableNames []pmodel.CIStr + newSchemaIDs []int64 + newTableNames []pmodel.CIStr + tableIDs []int64 + ) + if err := job.decodeArgs( + &oldSchemaIDs, &newSchemaIDs, &newTableNames, + &tableIDs, &oldSchemaNames, &oldTableNames); err != nil { + return errors.Trace(err) + } + + a.RenameTableInfos = GetRenameTablesArgsFromV1( + oldSchemaIDs, oldSchemaNames, oldTableNames, + newSchemaIDs, newTableNames, tableIDs, + ) + return nil +} + +// GetRenameTablesArgsFromV1 get v2 args from v1 +func GetRenameTablesArgsFromV1( + oldSchemaIDs []int64, + oldSchemaNames []pmodel.CIStr, + oldTableNames []pmodel.CIStr, + newSchemaIDs []int64, + newTableNames []pmodel.CIStr, + tableIDs []int64, +) []*RenameTableArgs { + infos := make([]*RenameTableArgs, 0, len(oldSchemaIDs)) + for i, oldSchemaID := range oldSchemaIDs { + infos = append(infos, &RenameTableArgs{ + OldSchemaID: oldSchemaID, + OldSchemaName: oldSchemaNames[i], + OldTableName: oldTableNames[i], + NewSchemaID: newSchemaIDs[i], + NewTableName: newTableNames[i], + TableID: tableIDs[i], + }) + } + + return infos +} + +// GetRenameTablesArgs gets the rename-tables args. +func GetRenameTablesArgs(job *Job) (*RenameTablesArgs, error) { + return getOrDecodeArgs[*RenameTablesArgs](&RenameTablesArgs{}, job) +} + +// AlterSequenceArgs is the arguments for alter sequence ddl job. +type AlterSequenceArgs struct { + Ident ast.Ident `json:"ident,omitempty"` + SeqOptions []*ast.SequenceOption `json:"seq_options,omitempty"` +} + +func (a *AlterSequenceArgs) getArgsV1(*Job) []any { + return []any{a.Ident, a.SeqOptions} +} + +func (a *AlterSequenceArgs) decodeV1(job *Job) error { + return errors.Trace(job.decodeArgs(&a.Ident, &a.SeqOptions)) +} + +// GetAlterSequenceArgs gets the args for alter Sequence ddl job. +func GetAlterSequenceArgs(job *Job) (*AlterSequenceArgs, error) { + return getOrDecodeArgs[*AlterSequenceArgs](&AlterSequenceArgs{}, job) +} + +// ModifyTableAutoIDCacheArgs is the arguments for Modify Table AutoID Cache ddl job. +type ModifyTableAutoIDCacheArgs struct { + NewCache int64 `json:"new_cache,omitempty"` +} + +func (a *ModifyTableAutoIDCacheArgs) getArgsV1(*Job) []any { + return []any{a.NewCache} +} + +func (a *ModifyTableAutoIDCacheArgs) decodeV1(job *Job) error { + return errors.Trace(job.decodeArgs(&a.NewCache)) +} + +// GetModifyTableAutoIDCacheArgs gets the args for modify table autoID cache ddl job. +func GetModifyTableAutoIDCacheArgs(job *Job) (*ModifyTableAutoIDCacheArgs, error) { + return getOrDecodeArgs[*ModifyTableAutoIDCacheArgs](&ModifyTableAutoIDCacheArgs{}, job) +} + +// ShardRowIDArgs is the arguments for shard row ID ddl job. +type ShardRowIDArgs struct { + ShardRowIDBits uint64 `json:"shard_row_id_bits,omitempty"` +} + +func (a *ShardRowIDArgs) getArgsV1(*Job) []any { + return []any{a.ShardRowIDBits} +} + +func (a *ShardRowIDArgs) decodeV1(job *Job) error { + return errors.Trace(job.decodeArgs(&a.ShardRowIDBits)) +} + +// GetShardRowIDArgs gets the args for shard row ID ddl job. +func GetShardRowIDArgs(job *Job) (*ShardRowIDArgs, error) { + return getOrDecodeArgs[*ShardRowIDArgs](&ShardRowIDArgs{}, job) +} + +// AlterTTLInfoArgs is the arguments for alter ttl info job. +type AlterTTLInfoArgs struct { + TTLInfo *TTLInfo `json:"ttl_info,omitempty"` + TTLEnable *bool `json:"ttl_enable,omitempty"` + TTLCronJobSchedule *string `json:"ttl_cron_job_schedule,omitempty"` +} + +func (a *AlterTTLInfoArgs) getArgsV1(*Job) []any { + return []any{a.TTLInfo, a.TTLEnable, a.TTLCronJobSchedule} +} + +func (a *AlterTTLInfoArgs) decodeV1(job *Job) error { + return errors.Trace(job.decodeArgs(&a.TTLInfo, &a.TTLEnable, &a.TTLCronJobSchedule)) +} + +// GetAlterTTLInfoArgs gets the args for alter ttl info job. +func GetAlterTTLInfoArgs(job *Job) (*AlterTTLInfoArgs, error) { + return getOrDecodeArgs[*AlterTTLInfoArgs](&AlterTTLInfoArgs{}, job) +} + +// CheckConstraintArgs is the arguments for both AlterCheckConstraint and DropCheckConstraint job. +type CheckConstraintArgs struct { + ConstraintName pmodel.CIStr `json:"constraint_name,omitempty"` + Enforced bool `json:"enforced,omitempty"` +} + +func (a *CheckConstraintArgs) getArgsV1(*Job) []any { + return []any{a.ConstraintName, a.Enforced} +} + +func (a *CheckConstraintArgs) decodeV1(job *Job) error { + return errors.Trace(job.decodeArgs(&a.ConstraintName, &a.Enforced)) +} + +// GetCheckConstraintArgs gets the AlterCheckConstraint args. +func GetCheckConstraintArgs(job *Job) (*CheckConstraintArgs, error) { + return getOrDecodeArgs[*CheckConstraintArgs](&CheckConstraintArgs{}, job) +} + +// AddCheckConstraintArgs is the args for add check constraint +type AddCheckConstraintArgs struct { + Constraint *ConstraintInfo `json:"constraint_info"` +} + +func (a *AddCheckConstraintArgs) getArgsV1(*Job) []any { + return []any{a.Constraint} +} + +func (a *AddCheckConstraintArgs) decodeV1(job *Job) error { + a.Constraint = &ConstraintInfo{} + return errors.Trace(job.decodeArgs(&a.Constraint)) +} + +// GetAddCheckConstraintArgs gets the AddCheckConstraint args. +func GetAddCheckConstraintArgs(job *Job) (*AddCheckConstraintArgs, error) { + return getOrDecodeArgs[*AddCheckConstraintArgs](&AddCheckConstraintArgs{}, job) +} + +// AlterTablePlacementArgs is the arguments for alter table placements ddl job. +type AlterTablePlacementArgs struct { + PlacementPolicyRef *PolicyRefInfo `json:"placement_policy_ref,omitempty"` +} + +func (a *AlterTablePlacementArgs) getArgsV1(*Job) []any { + return []any{a.PlacementPolicyRef} +} + +func (a *AlterTablePlacementArgs) decodeV1(job *Job) error { + // when the target policy is 'default', policy info is nil + return errors.Trace(job.decodeArgs(&a.PlacementPolicyRef)) +} + +// GetAlterTablePlacementArgs gets the args for alter table placements ddl job. +func GetAlterTablePlacementArgs(job *Job) (*AlterTablePlacementArgs, error) { + return getOrDecodeArgs[*AlterTablePlacementArgs](&AlterTablePlacementArgs{}, job) +} + +// SetTiFlashReplicaArgs is the arguments for setting TiFlash replica ddl. +type SetTiFlashReplicaArgs struct { + TiflashReplica ast.TiFlashReplicaSpec `json:"tiflash_replica,omitempty"` +} + +func (a *SetTiFlashReplicaArgs) getArgsV1(*Job) []any { + return []any{a.TiflashReplica} +} + +func (a *SetTiFlashReplicaArgs) decodeV1(job *Job) error { + return errors.Trace(job.decodeArgs(&a.TiflashReplica)) +} + +// GetSetTiFlashReplicaArgs gets the args for setting TiFlash replica ddl. +func GetSetTiFlashReplicaArgs(job *Job) (*SetTiFlashReplicaArgs, error) { + return getOrDecodeArgs[*SetTiFlashReplicaArgs](&SetTiFlashReplicaArgs{}, job) +} + +// UpdateTiFlashReplicaStatusArgs is the arguments for updating TiFlash replica status ddl. +type UpdateTiFlashReplicaStatusArgs struct { + Available bool `json:"available,omitempty"` + PhysicalID int64 `json:"physical_id,omitempty"` +} + +func (a *UpdateTiFlashReplicaStatusArgs) getArgsV1(*Job) []any { + return []any{a.Available, a.PhysicalID} +} + +func (a *UpdateTiFlashReplicaStatusArgs) decodeV1(job *Job) error { + return errors.Trace(job.decodeArgs(&a.Available, &a.PhysicalID)) +} + +// GetUpdateTiFlashReplicaStatusArgs gets the args for updating TiFlash replica status ddl. +func GetUpdateTiFlashReplicaStatusArgs(job *Job) (*UpdateTiFlashReplicaStatusArgs, error) { + return getOrDecodeArgs[*UpdateTiFlashReplicaStatusArgs](&UpdateTiFlashReplicaStatusArgs{}, job) +} + +// LockTablesArgs is the argument for LockTables. +type LockTablesArgs struct { + LockTables []TableLockTpInfo `json:"lock_tables,omitempty"` + IndexOfLock int `json:"index_of_lock,omitempty"` + UnlockTables []TableLockTpInfo `json:"unlock_tables,omitempty"` + IndexOfUnlock int `json:"index_of_unlock,omitempty"` + SessionInfo SessionInfo `json:"session_info,omitempty"` + IsCleanup bool `json:"is_cleanup:omitempty"` +} + +func (a *LockTablesArgs) getArgsV1(*Job) []any { + return []any{a} +} + +func (a *LockTablesArgs) decodeV1(job *Job) error { + return errors.Trace(job.decodeArgs(a)) +} + +// GetLockTablesArgs get the LockTablesArgs argument. +func GetLockTablesArgs(job *Job) (*LockTablesArgs, error) { + return getOrDecodeArgs[*LockTablesArgs](&LockTablesArgs{}, job) +} + +// RepairTableArgs is the argument for repair table +type RepairTableArgs struct { + TableInfo *TableInfo `json:"table_info"` +} + +func (a *RepairTableArgs) getArgsV1(*Job) []any { + return []any{a.TableInfo} +} + +func (a *RepairTableArgs) decodeV1(job *Job) error { + return errors.Trace(job.decodeArgs(&a.TableInfo)) +} + +// GetRepairTableArgs get the repair table args. +func GetRepairTableArgs(job *Job) (*RepairTableArgs, error) { + return getOrDecodeArgs[*RepairTableArgs](&RepairTableArgs{}, job) +} + +// AlterTableAttributesArgs is the argument for alter table attributes +type AlterTableAttributesArgs struct { + LabelRule *pdhttp.LabelRule `json:"label_rule,omitempty"` +} + +func (a *AlterTableAttributesArgs) getArgsV1(*Job) []any { + return []any{a.LabelRule} +} + +func (a *AlterTableAttributesArgs) decodeV1(job *Job) error { + a.LabelRule = &pdhttp.LabelRule{} + return errors.Trace(job.decodeArgs(a.LabelRule)) +} + +// GetAlterTableAttributesArgs get alter table attribute args from job. +func GetAlterTableAttributesArgs(job *Job) (*AlterTableAttributesArgs, error) { + return getOrDecodeArgs[*AlterTableAttributesArgs](&AlterTableAttributesArgs{}, job) +} + +// RecoverArgs is the argument for recover table/schema. +type RecoverArgs struct { + RecoverInfo *RecoverSchemaInfo `json:"recover_info,omitempty"` + CheckFlag int64 `json:"check_flag,omitempty"` + + // used during runtime + AffectedPhysicalIDs []int64 `json:"-"` +} + +func (a *RecoverArgs) getArgsV1(job *Job) []any { + if job.Type == ActionRecoverTable { + return []any{a.RecoverTableInfos()[0], a.CheckFlag} + } + return []any{a.RecoverInfo, a.CheckFlag} +} + +func (a *RecoverArgs) decodeV1(job *Job) error { + var ( + recoverTableInfo *RecoverTableInfo + recoverSchemaInfo = &RecoverSchemaInfo{} + recoverCheckFlag int64 + ) + if job.Type == ActionRecoverTable { + err := job.decodeArgs(&recoverTableInfo, &recoverCheckFlag) + if err != nil { + return errors.Trace(err) + } + recoverSchemaInfo.RecoverTableInfos = []*RecoverTableInfo{recoverTableInfo} + } else { + err := job.decodeArgs(recoverSchemaInfo, &recoverCheckFlag) + if err != nil { + return errors.Trace(err) + } + } + a.RecoverInfo = recoverSchemaInfo + a.CheckFlag = recoverCheckFlag + return nil +} + +// RecoverTableInfos get all the recover infos. +func (a *RecoverArgs) RecoverTableInfos() []*RecoverTableInfo { + return a.RecoverInfo.RecoverTableInfos +} + +// GetRecoverArgs get the recover table/schema args. +func GetRecoverArgs(job *Job) (*RecoverArgs, error) { + return getOrDecodeArgs[*RecoverArgs](&RecoverArgs{}, job) +} + +// PlacementPolicyArgs is the argument for create/alter/drop placement policy +type PlacementPolicyArgs struct { + Policy *PolicyInfo `json:"policy,omitempty"` + ReplaceOnExist bool `json:"replace_on_exist,omitempty"` + PolicyName pmodel.CIStr `json:"policy_name,omitempty"` + + // it's set for alter/drop policy in v2 + PolicyID int64 `json:"policy_id"` +} + +func (a *PlacementPolicyArgs) getArgsV1(job *Job) []any { + if job.Type == ActionCreatePlacementPolicy { + return []any{a.Policy, a.ReplaceOnExist} + } else if job.Type == ActionAlterPlacementPolicy { + return []any{a.Policy} + } + return []any{a.PolicyName} +} + +func (a *PlacementPolicyArgs) decodeV1(job *Job) error { + a.PolicyID = job.SchemaID + + if job.Type == ActionCreatePlacementPolicy { + return errors.Trace(job.decodeArgs(&a.Policy, &a.ReplaceOnExist)) + } else if job.Type == ActionAlterPlacementPolicy { + return errors.Trace(job.decodeArgs(&a.Policy)) + } + return errors.Trace(job.decodeArgs(&a.PolicyName)) +} + +// GetPlacementPolicyArgs gets the placement policy args. +func GetPlacementPolicyArgs(job *Job) (*PlacementPolicyArgs, error) { + return getOrDecodeArgs[*PlacementPolicyArgs](&PlacementPolicyArgs{}, job) +} + +// SetDefaultValueArgs is the argument for setting default value ddl. +type SetDefaultValueArgs struct { + Col *ColumnInfo `json:"column_info,omitempty"` +} + +func (a *SetDefaultValueArgs) getArgsV1(*Job) []any { + return []any{a.Col} +} + +func (a *SetDefaultValueArgs) decodeV1(job *Job) error { + a.Col = &ColumnInfo{} + return errors.Trace(job.decodeArgs(a.Col)) +} + +// GetSetDefaultValueArgs get the args for setting default value ddl. +func GetSetDefaultValueArgs(job *Job) (*SetDefaultValueArgs, error) { + return getOrDecodeArgs[*SetDefaultValueArgs](&SetDefaultValueArgs{}, job) +} + +// KeyRange is copied from kv.KeyRange to avoid cycle import. +// Unused fields are removed. +type KeyRange struct { + StartKey []byte + EndKey []byte +} + +// FlashbackClusterArgs is the argument for flashback cluster. +type FlashbackClusterArgs struct { + FlashbackTS uint64 `json:"flashback_ts,omitempty"` + PDScheduleValue map[string]any `json:"pd_schedule_value,omitempty"` + EnableGC bool `json:"enable_gc,omitempty"` + EnableAutoAnalyze bool `json:"enable_auto_analyze,omitempty"` + EnableTTLJob bool `json:"enable_ttl_job,omitempty"` + SuperReadOnly bool `json:"super_read_only,omitempty"` + LockedRegionCnt uint64 `json:"locked_region_cnt,omitempty"` + StartTS uint64 `json:"start_ts,omitempty"` + CommitTS uint64 `json:"commit_ts,omitempty"` + FlashbackKeyRanges []KeyRange `json:"key_ranges,omitempty"` +} + +func (a *FlashbackClusterArgs) getArgsV1(*Job) []any { + enableAutoAnalyze := "ON" + superReadOnly := "ON" + enableTTLJob := "ON" + if !a.EnableAutoAnalyze { + enableAutoAnalyze = "OFF" + } + if !a.SuperReadOnly { + superReadOnly = "OFF" + } + if !a.EnableTTLJob { + enableTTLJob = "OFF" + } + + return []any{ + a.FlashbackTS, a.PDScheduleValue, a.EnableGC, + enableAutoAnalyze, superReadOnly, a.LockedRegionCnt, + a.StartTS, a.CommitTS, enableTTLJob, a.FlashbackKeyRanges, + } +} + +func (a *FlashbackClusterArgs) decodeV1(job *Job) error { + var autoAnalyzeValue, readOnlyValue, ttlJobEnableValue string + + if err := job.decodeArgs( + &a.FlashbackTS, &a.PDScheduleValue, &a.EnableGC, + &autoAnalyzeValue, &readOnlyValue, &a.LockedRegionCnt, + &a.StartTS, &a.CommitTS, &ttlJobEnableValue, &a.FlashbackKeyRanges, + ); err != nil { + return errors.Trace(err) + } + + if autoAnalyzeValue == "ON" { + a.EnableAutoAnalyze = true + } + if readOnlyValue == "ON" { + a.SuperReadOnly = true + } + if ttlJobEnableValue == "ON" { + a.EnableTTLJob = true + } + + return nil +} + +// GetFlashbackClusterArgs get the flashback cluster argument from job. +func GetFlashbackClusterArgs(job *Job) (*FlashbackClusterArgs, error) { + return getOrDecodeArgs[*FlashbackClusterArgs](&FlashbackClusterArgs{}, job) +} + +// IndexOp is used to identify arguemnt type, which is only used for v1 index args. +// TODO(joechenrh): remove this type after totally switched to v2 +type IndexOp byte + +// List op types. +const ( + OpAddIndex = iota + OpDropIndex + OpRollbackAddIndex +) + +// IndexArg is the argument for single add/drop/rename index operation. +// Different types of job use different fields. +// Below lists used fields for each type (listed in order of the layout in v1) +// +// Adding NonPK: Unique, IndexName, IndexPartSpecifications, IndexOption, SQLMode, Warning(not stored, always nil), Global +// Adding PK: Unique, IndexName, IndexPartSpecifications, IndexOptions, HiddelCols, Global +// Adding vector index: IndexName, IndexPartSpecifications, IndexOption, FuncExpr +// Drop index: IndexName, IfExist, IndexID +// Rollback add index: IndexName, IfExist, IsVector +// Rename index: IndexName +type IndexArg struct { + // Global is never used, we only use Global in IndexOption. Can be deprecated later. + Global bool `json:"-"` + Unique bool `json:"unique,omitempty"` + IndexName pmodel.CIStr `json:"index_name,omitempty"` + IndexPartSpecifications []*ast.IndexPartSpecification `json:"index_part_specifications"` + IndexOption *ast.IndexOption `json:"index_option,omitempty"` + HiddenCols []*ColumnInfo `json:"hidden_cols,omitempty"` + + // For vector index + FuncExpr string `json:"func_expr,omitempty"` + IsVector bool `json:"is_vector,omitempty"` + + // For PK + IsPK bool `json:"is_pk,omitempty"` + SQLMode mysql.SQLMode `json:"sql_mode,omitempty"` + + // IfExist will be used in onDropIndex. + IndexID int64 `json:"index_id,omitempty"` + IfExist bool `json:"if_exist,omitempty"` + IsGlobal bool `json:"is_global,omitempty"` +} + +// ModifyIndexArgs is the argument for add/drop/rename index jobs, +// which includes PK and vector index. +type ModifyIndexArgs struct { + IndexArgs []*IndexArg `json:"index_args,omitempty"` + + // Belows is used for finished args. + PartitionIDs []int64 `json:"partition_ids,omitempty"` + + // This is only used for getFinishedArgsV1 to distinguish different type of job in v1, + // since they need different arguments layout. + // TODO(joechenrh): remove this flag after totally switched to v2 + OpType IndexOp `json:"-"` +} + +func (a *ModifyIndexArgs) getArgsV1(job *Job) []any { + if job.Type == ActionRenameIndex { + return []any{a.IndexArgs[0].IndexName, a.IndexArgs[1].IndexName} + } + + // Drop index + if job.Type == ActionDropIndex || job.Type == ActionDropPrimaryKey { + if len(a.IndexArgs) == 1 { + return []any{a.IndexArgs[0].IndexName, a.IndexArgs[0].IfExist} + } + indexNames := make([]pmodel.CIStr, len(a.IndexArgs)) + ifExists := make([]bool, len(a.IndexArgs)) + for i, idxArg := range a.IndexArgs { + indexNames[i] = idxArg.IndexName + ifExists[i] = idxArg.IfExist + } + return []any{indexNames, ifExists} + } + + // Add vector index + if job.Type == ActionAddVectorIndex { + arg := a.IndexArgs[0] + return []any{arg.IndexName, arg.IndexPartSpecifications[0], arg.IndexOption, arg.FuncExpr} + } + + // Add primary key + if job.Type == ActionAddPrimaryKey { + arg := a.IndexArgs[0] + + // The sixth argument is set and never used. + // Leave it as nil to make it compatible with history job. + return []any{ + arg.Unique, arg.IndexName, arg.IndexPartSpecifications, + arg.IndexOption, arg.SQLMode, nil, arg.Global, + } + } + + // Add index + n := len(a.IndexArgs) + unique := make([]bool, n) + indexName := make([]pmodel.CIStr, n) + indexPartSpecification := make([][]*ast.IndexPartSpecification, n) + indexOption := make([]*ast.IndexOption, n) + hiddenCols := make([][]*ColumnInfo, n) + global := make([]bool, n) + + for i, arg := range a.IndexArgs { + unique[i] = arg.Unique + indexName[i] = arg.IndexName + indexPartSpecification[i] = arg.IndexPartSpecifications + indexOption[i] = arg.IndexOption + hiddenCols[i] = arg.HiddenCols + global[i] = arg.Global + } + + // This is to make the args compatible with old logic + if n == 1 { + return []any{unique[0], indexName[0], indexPartSpecification[0], indexOption[0], hiddenCols[0], global[0]} + } + + return []any{unique, indexName, indexPartSpecification, indexOption, hiddenCols, global} +} + +func (a *ModifyIndexArgs) decodeV1(job *Job) error { + var err error + switch job.Type { + case ActionRenameIndex: + err = a.decodeRenameIndexV1(job) + case ActionAddIndex: + err = a.decodeAddIndexV1(job) + case ActionAddVectorIndex: + err = a.decodeAddVectorIndexV1(job) + case ActionAddPrimaryKey: + err = a.decodeAddPrimaryKeyV1(job) + default: + err = errors.Errorf("Invalid job type for decoding %d", job.Type) + } + return errors.Trace(err) +} + +func (a *ModifyIndexArgs) decodeRenameIndexV1(job *Job) error { + var from, to pmodel.CIStr + if err := job.decodeArgs(&from, &to); err != nil { + return errors.Trace(err) + } + a.IndexArgs = []*IndexArg{ + {IndexName: from}, + {IndexName: to}, + } + return nil +} + +func (a *ModifyIndexArgs) decodeDropIndexV1(job *Job) error { + indexNames := make([]pmodel.CIStr, 1) + ifExists := make([]bool, 1) + if err := job.decodeArgs(&indexNames[0], &ifExists[0]); err != nil { + if err = job.decodeArgs(&indexNames, &ifExists); err != nil { + return errors.Trace(err) + } + } + + a.IndexArgs = make([]*IndexArg, len(indexNames)) + for i, indexName := range indexNames { + a.IndexArgs[i] = &IndexArg{ + IndexName: indexName, + IfExist: ifExists[i], + } + } + return nil +} + +func (a *ModifyIndexArgs) decodeAddIndexV1(job *Job) error { + uniques := make([]bool, 1) + indexNames := make([]pmodel.CIStr, 1) + indexPartSpecifications := make([][]*ast.IndexPartSpecification, 1) + indexOptions := make([]*ast.IndexOption, 1) + hiddenCols := make([][]*ColumnInfo, 1) + globals := make([]bool, 1) + + if err := job.decodeArgs( + &uniques, &indexNames, &indexPartSpecifications, + &indexOptions, &hiddenCols, &globals); err != nil { + if err = job.decodeArgs( + &uniques[0], &indexNames[0], &indexPartSpecifications[0], + &indexOptions[0], &hiddenCols[0], &globals[0]); err != nil { + return errors.Trace(err) + } + } + + for i, unique := range uniques { + a.IndexArgs = append(a.IndexArgs, &IndexArg{ + Unique: unique, + IndexName: indexNames[i], + IndexPartSpecifications: indexPartSpecifications[i], + IndexOption: indexOptions[i], + HiddenCols: hiddenCols[i], + Global: globals[i], + }) + } + return nil +} + +func (a *ModifyIndexArgs) decodeAddPrimaryKeyV1(job *Job) error { + a.IndexArgs = []*IndexArg{{IsPK: true}} + var unused any + if err := job.decodeArgs( + &a.IndexArgs[0].Unique, &a.IndexArgs[0].IndexName, &a.IndexArgs[0].IndexPartSpecifications, + &a.IndexArgs[0].IndexOption, &a.IndexArgs[0].SQLMode, + &unused, &a.IndexArgs[0].Global); err != nil { + return errors.Trace(err) + } + return nil +} + +func (a *ModifyIndexArgs) decodeAddVectorIndexV1(job *Job) error { + var ( + indexName pmodel.CIStr + indexPartSpecification *ast.IndexPartSpecification + indexOption *ast.IndexOption + funcExpr string + ) + + if err := job.decodeArgs( + &indexName, &indexPartSpecification, &indexOption, &funcExpr); err != nil { + return errors.Trace(err) + } + + a.IndexArgs = []*IndexArg{{ + IndexName: indexName, + IndexPartSpecifications: []*ast.IndexPartSpecification{indexPartSpecification}, + IndexOption: indexOption, + FuncExpr: funcExpr, + IsVector: true, + }} + return nil +} + +func (a *ModifyIndexArgs) getFinishedArgsV1(job *Job) []any { + // Add index + if a.OpType == OpAddIndex { + if job.Type == ActionAddVectorIndex { + return []any{a.IndexArgs[0].IndexID, a.IndexArgs[0].IfExist, a.PartitionIDs, a.IndexArgs[0].IsGlobal} + } + + n := len(a.IndexArgs) + indexIDs := make([]int64, n) + ifExists := make([]bool, n) + isGlobals := make([]bool, n) + for i, arg := range a.IndexArgs { + indexIDs[i] = arg.IndexID + ifExists[i] = arg.IfExist + isGlobals[i] = arg.Global + } + return []any{indexIDs, ifExists, a.PartitionIDs, isGlobals} + } + + // Below is to make the args compatible with old logic: + // 1. For drop index, arguments are [CIStr, bool, int64, []int64, bool]. + // 3. For rollback add index, arguments are [[]CIStr, []bool, []int64]. + if a.OpType == OpRollbackAddIndex { + indexNames := make([]pmodel.CIStr, len(a.IndexArgs)) + ifExists := make([]bool, len(a.IndexArgs)) + for i, idxArg := range a.IndexArgs { + indexNames[i] = idxArg.IndexName + ifExists[i] = idxArg.IfExist + } + return []any{indexNames, ifExists, a.PartitionIDs} + } + + idxArg := a.IndexArgs[0] + return []any{idxArg.IndexName, idxArg.IfExist, idxArg.IndexID, a.PartitionIDs, idxArg.IsVector} +} + +// GetRenameIndexes get name of renamed index. +func (a *ModifyIndexArgs) GetRenameIndexes() (from, to pmodel.CIStr) { + from, to = a.IndexArgs[0].IndexName, a.IndexArgs[1].IndexName + return +} + +// GetModifyIndexArgs gets the add/rename index args. +func GetModifyIndexArgs(job *Job) (*ModifyIndexArgs, error) { + return getOrDecodeArgs(&ModifyIndexArgs{}, job) +} + +// GetDropIndexArgs is only used to get drop index arg. +// The logic is separated from ModifyIndexArgs.decodeV1. +// TODO(joechenrh): replace this function with GetModifyIndexArgs after totally switched to v2. +func GetDropIndexArgs(job *Job) (*ModifyIndexArgs, error) { + if job.Version == JobVersion2 { + return getOrDecodeArgsV2[*ModifyIndexArgs](job) + } + + // For add index jobs(ActionAddIndex, etc.) in v1, it can store both drop index arguments and add index arguments. + // The logic in ModifyIndexArgs.decodeV1 maybe: + // Decode rename index args if type == ActionRenameIndex + // Try decode drop index args + // Try decode add index args if failed + // So we separate this from decodeV1 to avoid unnecessary "try decode" logic. + a := &ModifyIndexArgs{} + err := a.decodeDropIndexV1(job) + return a, errors.Trace(err) +} + +// GetFinishedModifyIndexArgs gets the add/drop index args. +func GetFinishedModifyIndexArgs(job *Job) (*ModifyIndexArgs, error) { + if job.Version == JobVersion2 { + return getOrDecodeArgsV2[*ModifyIndexArgs](job) + } + + if job.IsRollingback() || job.Type == ActionDropIndex || job.Type == ActionDropPrimaryKey { + indexNames := make([]pmodel.CIStr, 1) + ifExists := make([]bool, 1) + indexIDs := make([]int64, 1) + var partitionIDs []int64 + isVector := false + var err error + + if job.IsRollingback() { + // Rollback add indexes + err = job.decodeArgs(&indexNames, &ifExists, &partitionIDs, &isVector) + } else { + // Finish drop index + err = job.decodeArgs(&indexNames[0], &ifExists[0], &indexIDs[0], &partitionIDs, &isVector) + } + if err != nil { + return nil, errors.Trace(err) + } + + a := &ModifyIndexArgs{ + PartitionIDs: partitionIDs, + } + a.IndexArgs = make([]*IndexArg, len(indexNames)) + for i, indexName := range indexNames { + a.IndexArgs[i] = &IndexArg{ + IndexName: indexName, + IfExist: ifExists[i], + IsVector: isVector, + } + } + // For drop index, store index id in IndexArgs, no impact on other situations. + // Currently, there is only one indexID since drop index is not supported in multi schema change. + // TODO(joechenrh): modify this and corresponding logic if we need support drop multi indexes in V1. + a.IndexArgs[0].IndexID = indexIDs[0] + + return a, nil + } + + // Add index/vector index/PK + addIndexIDs := make([]int64, 1) + ifExists := make([]bool, 1) + isGlobals := make([]bool, 1) + var partitionIDs []int64 + + // add vector index args doesn't store slice. + if err := job.decodeArgs(&addIndexIDs[0], &ifExists[0], &partitionIDs, &isGlobals[0]); err != nil { + if err = job.decodeArgs(&addIndexIDs, &ifExists, &partitionIDs, &isGlobals); err != nil { + return nil, errors.Errorf("Failed to decode finished arguments from job version 1") + } + } + a := &ModifyIndexArgs{PartitionIDs: partitionIDs} + for i, indexID := range addIndexIDs { + a.IndexArgs = append(a.IndexArgs, &IndexArg{ + IndexID: indexID, + IfExist: ifExists[i], + IsGlobal: isGlobals[i], + }) + } + return a, nil +} + +// ModifyColumnArgs is the argument for modify column. +type ModifyColumnArgs struct { + Column *ColumnInfo `json:"column,omitempty"` + OldColumnName pmodel.CIStr `json:"old_column_name,omitempty"` + Position *ast.ColumnPosition `json:"position,omitempty"` + ModifyColumnType byte `json:"modify_column_type,omitempty"` + NewShardBits uint64 `json:"new_shard_bits,omitempty"` + // ChangingColumn is the temporary column derived from OldColumn + ChangingColumn *ColumnInfo `json:"changing_column,omitempty"` + // ChangingIdxs is only used in test, so don't persist it + ChangingIdxs []*IndexInfo `json:"-"` + // RedundantIdxs stores newly-created temp indexes which can be overwritten by other temp indexes. + // These idxs will be added to finished args after job done. + RedundantIdxs []int64 `json:"removed_idxs,omitempty"` + + // Finished args + // IndexIDs stores index ids to be added to gc table. + IndexIDs []int64 `json:"index_ids,omitempty"` + PartitionIDs []int64 `json:"partition_ids,omitempty"` +} + +func (a *ModifyColumnArgs) getArgsV1(*Job) []any { + return []any{ + a.Column, a.OldColumnName, a.Position, a.ModifyColumnType, + a.NewShardBits, a.ChangingColumn, a.ChangingIdxs, a.RedundantIdxs, + } +} + +func (a *ModifyColumnArgs) decodeV1(job *Job) error { + return job.decodeArgs( + &a.Column, &a.OldColumnName, &a.Position, &a.ModifyColumnType, + &a.NewShardBits, &a.ChangingColumn, &a.ChangingIdxs, &a.RedundantIdxs, + ) +} + +func (a *ModifyColumnArgs) getFinishedArgsV1(*Job) []any { + return []any{a.IndexIDs, a.PartitionIDs} +} + +// GetModifyColumnArgs get the modify column argument from job. +func GetModifyColumnArgs(job *Job) (*ModifyColumnArgs, error) { + return getOrDecodeArgs(&ModifyColumnArgs{}, job) +} + +// GetFinishedModifyColumnArgs get the finished modify column argument from job. +func GetFinishedModifyColumnArgs(job *Job) (*ModifyColumnArgs, error) { + if job.Version == JobVersion1 { + var ( + indexIDs []int64 + partitionIDs []int64 + ) + if err := job.decodeArgs(&indexIDs, &partitionIDs); err != nil { + return nil, errors.Trace(err) + } + return &ModifyColumnArgs{ + IndexIDs: indexIDs, + PartitionIDs: partitionIDs, + }, nil + } + return getOrDecodeArgsV2[*ModifyColumnArgs](job) +} diff --git a/pkg/table/tables/partition.go b/pkg/table/tables/partition.go index a7af6b55b5239..3674b0cc87def 100644 --- a/pkg/table/tables/partition.go +++ b/pkg/table/tables/partition.go @@ -1518,7 +1518,9 @@ func GetReorganizedPartitionedTable(t table.Table) (table.PartitionedTable, erro pi.Type = pi.DDLType pi.Expr = pi.DDLExpr pi.Columns = pi.DDLColumns - tblInfo.ID = pi.NewTableID + if pi.NewTableID != 0 { + tblInfo.ID = pi.NewTableID + } constraints, err := table.LoadCheckConstraint(tblInfo) if err != nil { @@ -1626,8 +1628,17 @@ func partitionedTableAddRecord(ctx sessionctx.Context, t *partitionedTable, r [] if err != nil { return nil, errors.Trace(err) } +<<<<<<< HEAD tbl = t.GetPartition(pid) recordID, err = tbl.AddRecord(ctx, r, opts...) +======= + tbl = t.getPartition(pid) + if !tbl.Meta().PKIsHandle && !tbl.Meta().IsCommonHandle { + // Preserve the _tidb_rowid also in the new partition! + r = append(r, types.NewIntDatum(recordID.IntValue())) + } + recordID, err = tbl.addRecord(ctx, txn, r, opt) +>>>>>>> b6025b97877 (*: Reorg partition fix delete ranges and handling non-clustered tables with concurrent DML (#57114)) if err != nil { return } diff --git a/tests/integrationtest/r/ddl/db_partition.result b/tests/integrationtest/r/ddl/db_partition.result index 1c37b2bcfeb5f..24ecd739d3656 100644 --- a/tests/integrationtest/r/ddl/db_partition.result +++ b/tests/integrationtest/r/ddl/db_partition.result @@ -3306,7 +3306,7 @@ id store_id 1 1 select *,_tidb_rowid from t; id store_id _tidb_rowid -0 18 30257 +0 18 30002 1 1 30001 drop table t, t1; create table t (id int not null, store_id int not null ) partition by range (store_id) (partition p0 values less than (6), partition p1 values less than (11), partition p2 values less than (16), partition p3 values less than (21)); @@ -3329,6 +3329,6 @@ id store_id 1 1 select *,_tidb_rowid from t; id store_id _tidb_rowid -0 18 30257 +0 18 30002 1 1 30001 drop table t, t1;