-
Notifications
You must be signed in to change notification settings - Fork 5.9k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
DDL: support drop partition on the partitioned table with global indexes #19222
DDL: support drop partition on the partitioned table with global indexes #19222
Conversation
Please resolve lock @ldeng-ustc |
n := len(w.indexes) | ||
for i, idxRecord := range idxRecords { | ||
taskCtx.scanCount++ | ||
err := w.indexes[i%n].Delete(w.sessCtx.GetSessionVars().StmtCtx, txn, idxRecord.vals, idxRecord.handle) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
What does i%n
mean here?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
For example, when dropping 3 indexes, records in idxRecords
are belong to index1, index2, index3, index1, index2……
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This is a good way to add/clean index records for multi-index reorg work.
For now, tidb will handle one element(index) every time, do them one by one, that's what the current element used for. If so we need some change in the reorgInfo
, maybe we can keep running it as the old way and file an issue to accelerate all other reorg work like this. I will follow your pr continuously.
ddl/index.go
Outdated
if len(idxWorkers) > int(workerCnt) { | ||
workers := idxWorkers[workerCnt:] | ||
idxWorkers = idxWorkers[:workerCnt] | ||
closeReorgIndexWorkers(workers) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
There is a defer closeReorgIndexWorkers(idxWorkers)
at the begining of the function, will the worker be closed twice?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
No, here idxWorkers
is set to workers not closed.
if err != nil { | ||
return false, errors.Trace(err) | ||
} | ||
reorg.StartHandle, reorg.EndHandle, reorg.PhysicalTableID = start, end, pid |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Can add/drop index and drop partition DDL be run at the same time? if so, store this reorg information is insufficient, the DDL job will overwrite each other.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
UpdateReorgMeta
also save the DDL job ID, so it seems OK.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
OK, the stored meta info would not overwrite each other.
But how about those two operations, I guess let them run at the same time is not safe.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@ldeng-ustc, please update your pull request. |
@ldeng-ustc, please update your pull request. |
@tiancaiamao, @AilinKid, PTAL. |
ddl/partition.go
Outdated
return nt | ||
} | ||
|
||
func buildPlacementDropRules(schemaID, tableID int64, partitionIDs []int64) []*placement.RuleOp { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Where is it used?
ddl/partition.go
Outdated
if d.infoHandle != nil { | ||
bundles := make([]*placement.Bundle, 0, len(physicalTableIDs)) | ||
for _, ID := range physicalTableIDs { | ||
oldBundle, ok := d.infoHandle.Get().BundleByName(placement.GroupID(ID)) | ||
if ok && !oldBundle.IsEmpty() { | ||
bundles = append(bundles, buildPlacementDropBundle(ID)) | ||
} | ||
} | ||
|
||
err = infosync.PutRuleBundles(nil, bundles) | ||
if err != nil { | ||
job.State = model.JobStateCancelled | ||
return ver, errors.Wrapf(err, "failed to notify PD the placement rules") | ||
} | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Duplicate code, it can be simplified.
ddl/index.go
Outdated
rowDecoder: rowDecoder, | ||
defaultVals: make([]types.Datum, len(t.WritableCols())), | ||
rowMap: make(map[int64]types.Datum, len(decodeColMap)), | ||
metricCounter: metrics.BackfillTotalCounter.WithLabelValues("add_idx_speed"), |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
maybe this metric is not suitable.
if err != nil { | ||
return errors.Trace(err) | ||
} | ||
taskCtx.addedCount++ |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
may the added count is not suitable here too
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Maybe we should replace all "backfilling" in backfilling.go
to "reorganize", because cleaning up index is not "backfilling", but it a big project, so I leave it unchanged now.
ddl/partition.go
Outdated
// If an error occurs, it returns that it cannot delete all partitions or that the partition doesn't exist. | ||
err = checkDropTablePartition(tblInfo, partNames) | ||
if err != nil { | ||
job.State = model.JobStateCancelled | ||
return ver, errors.Trace(err) | ||
} | ||
physicalTableIDs = removePartitionInfo(tblInfo, partNames) | ||
} | ||
updateDroppingPartitionInfo(tblInfo, partNames) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
how about move this after the placement bundle check below.
ddl/reorg.go
Outdated
if job.SnapshotVer == 0 { | ||
// For the case of the old TiDB version(do not exist the element information) is upgraded to the new TiDB version. | ||
// Third step, we need to remove the element information to make sure we can save the reorganized information to storage. | ||
failpoint.Inject("MockGetIndexRecordErr", func(val failpoint.Value) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
this failpoint is useless
ddl/reorg.go
Outdated
job.SnapshotVer = ver.Ver | ||
element = elements[0] | ||
} else { | ||
failpoint.Inject("MockGetIndexRecordErr", func(val failpoint.Value) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
ditto
n := len(w.indexes) | ||
for i, idxRecord := range idxRecords { | ||
taskCtx.scanCount++ | ||
err := w.indexes[i%n].Delete(w.sessCtx.GetSessionVars().StmtCtx, txn, idxRecord.vals, idxRecord.handle) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This is a good way to add/clean index records for multi-index reorg work.
For now, tidb will handle one element(index) every time, do them one by one, that's what the current element used for. If so we need some change in the reorgInfo
, maybe we can keep running it as the old way and file an issue to accelerate all other reorg work like this. I will follow your pr continuously.
@tiancaiamao, @AilinKid, @djshow832, PTAL. |
…idb into global_index_drop_partition
Thank you. In fact, I create this pr before element is added, so I did not use it. Maybe I will try to modify other reorg logic after finished works on global index. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Rest LGTM
cc937d7
to
b3d58c0
Compare
Co-authored-by: djshow832 <zhangming@pingcap.com>
b3d58c0
to
d1ebeec
Compare
@tiancaiamao, @AilinKid, @djshow832, PTAL. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
LGTM
/merge |
/run-all-tests |
What problem does this PR solve?
Support drop partition operation on the table which has global index.
Proposal: #18982
This PR is based on #18024 and parser #970.
What is changed and how it works?
What's Changed:
Drop partition is change to 4 state:
public
,delete only
,delete reorg
andnone
.New
ReorgIndexWorker
interface to support different types of reorganization. Implement this interface indropIndexWorker
, use to clean up global indexes when dropping partitions.Test for the new features.
How it Works:
From
public
todelete only
, move the definitions of dropping partitions toDroppingDefinitions
, then do nothing in statedelete only
. When ddl owner is inreorg
state, all servers can not see the dropping partitions, so that owner can do reorganization safely.In reorganization, interface
ReorgIndexWorker
receive task and return results.Implement
dropIndexWorker
to clean up indexes. Move some code ofaddIndexWorker
tobaseIndexWorker
, and modify it to support reorganizing multiple indexes to reuse the code.Process of cleaning up the indexes is similar as adding index.
cleanupGlobalIndexes
clean the indexes partition by partition,cleanupPhysicalTableIndex
split the handle range, and send task todropIndexWorker
, worker clean up the index entries related to handles in a txn, an return the result.Potential problems
In previous TiDB, only
add index
has a realreorg
state. So all reorganizing code is write for adding index. This PR modify these code to support cleaning up the global index indrop partition
, it may cause some potential problems. It is hard to identify all problems, but I still list some of them here.metric
3 metric variables (
AddIndexProgress
,AddIndexTotalCounter
andBatchAddIdxHistogram
) are used in reorganizing ofadd index
. I leave them unmodified for the moment. So it may show wrong information when running reorganization ofdrop partition
. These variables should be modified to support all reorganization works or we should add other 3 variables to monitor thedrop partition
process.ddl job queue
Now,
add index
jobs use an independent ddl job queue so that long reorganizing time will not block other ddl jobs (seemeta.AddIndexJobListKey
). Maybedrop partion
in partitioned table with global indexes should use the same queue or another new queue, but I am not sure.Check List
Tests
Release note