Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
133 commits
Select commit Hold shift + click to select a range
cfa9bde
Initial test with two domains for different SchemaVersions
mjonss May 31, 2024
311e71f
Added a test for AddPartition
mjonss Jun 3, 2024
4236768
Updated test
mjonss Jun 3, 2024
2154259
Merge remote-tracking branch 'pingcap/master' into multi-domain-parti…
mjonss Aug 7, 2024
b4388a7
Moved tests to a new file
mjonss Aug 8, 2024
7d4dd7e
Stabilized tests
mjonss Aug 9, 2024
8adc5ac
Merge remote-tracking branch 'pingcap/master' into multi-domain-parti…
mjonss Aug 20, 2024
849825d
Updated with GLOBAL IndexOption and failpoint instead of hooks
mjonss Aug 20, 2024
607118d
Renamed variables
mjonss Aug 20, 2024
d7217ed
WIP test now with drop partition, needs more work, but passes
mjonss Aug 20, 2024
1ea37d7
Updated tests
mjonss Aug 27, 2024
f51b542
Added test to see how drop unique index behaves
mjonss Aug 27, 2024
61f7668
Merge remote-tracking branch 'pingcap/master' into multi-domain-parti…
mjonss Aug 30, 2024
d0e5b10
Updated test
mjonss Aug 30, 2024
cd1f500
Refactor test and added utility function to skip on disabled failpoints
mjonss Sep 2, 2024
dc709d4
Merge branch 'multi-domain-partition-tests' into truncate-partition-g…
mjonss Sep 3, 2024
7e36c59
Cleaned up non-needed tests
mjonss Sep 3, 2024
10aac89
Added test case for #55819
mjonss Sep 3, 2024
234a636
Delete entries during INSERT if TRUNCATE PARTITIONING is in DeleteReorg
mjonss Sep 3, 2024
4542f06
Linting and more tests
mjonss Sep 3, 2024
6a119d7
Updated tests and linting
mjonss Sep 3, 2024
389dd63
Linting
mjonss Sep 3, 2024
d162004
More tests
mjonss Sep 3, 2024
5e06972
Test updates
mjonss Sep 3, 2024
c5917c1
WIP check all global indexes before insert and if duplicates, delete
mjonss Sep 4, 2024
043d366
New test passes \o/
mjonss Sep 4, 2024
4dbf35f
Linting
mjonss Sep 4, 2024
d312f0a
Merge remote-tracking branch 'pingcap/master' into truncate-partition…
mjonss Sep 4, 2024
4351bb3
Added one more state in Truncate partition to avoid inconsistencies
mjonss Sep 4, 2024
789c633
Test updates and linting
mjonss Sep 4, 2024
ed7d4ea
Updated test
mjonss Sep 4, 2024
ac48047
Test update
mjonss Sep 4, 2024
777f54d
Third way to implement it, this time with index.create instead...
mjonss Sep 5, 2024
de984bf
Cleanup / linting
mjonss Sep 5, 2024
f9c9a27
Third change, only check and update assertion for overwriting in inde…
mjonss Sep 6, 2024
48a1515
Drop Partition updates
mjonss Sep 7, 2024
2a486ca
Added comment to test.
mjonss Sep 7, 2024
be65b3a
code refactoring to decrease code duplication
mjonss Sep 9, 2024
4a20061
reverted name changes etc. to make the pr smaller
mjonss Sep 9, 2024
14548f7
Merge remote-tracking branch 'pingcap/master' into truncate-partition…
mjonss Sep 9, 2024
9520c12
Removed non-neccessary comments
mjonss Sep 9, 2024
7e1221e
Linting
mjonss Sep 10, 2024
53188f0
Linting
mjonss Sep 10, 2024
ae80860
Merge remote-tracking branch 'pingcap/master' into truncate-partition…
mjonss Sep 11, 2024
c37bf0e
Merge remote-tracking branch 'pingcap/master' into truncate-partition…
mjonss Sep 11, 2024
8d30146
Minor refactor for passing Label rules test
mjonss Sep 11, 2024
0d9b93d
Removed outdated comment
mjonss Sep 11, 2024
7d4568d
Added error injection tests
mjonss Sep 11, 2024
287394d
Merge branch 'truncate-partition-global-index' into truncate-partitio…
mjonss Sep 11, 2024
c98f605
Added JobStateCancelled in the right places
mjonss Sep 11, 2024
878e727
Merge branch 'truncate-partition-global-index' into truncate-partitio…
mjonss Sep 11, 2024
a805cb2
WIP, state where delete_range is used when it should not.
mjonss Sep 12, 2024
67bf137
Truncate partition rollback/cancel fix
mjonss Sep 12, 2024
5da49a3
Merge remote-tracking branch 'pingcap/master' into truncate-partition…
mjonss Sep 12, 2024
eaeb0f7
Added a comment
mjonss Sep 12, 2024
9461c30
Merge branch 'truncate-partition-global-index' into truncate-partitio…
mjonss Sep 12, 2024
8f023a3
Merge remote-tracking branch 'pingcap/master' into truncate-partition…
mjonss Sep 17, 2024
9a809bf
Merge branch 'truncate-partition-global-index' into truncate-partitio…
mjonss Sep 17, 2024
cc56350
updated test
mjonss Sep 17, 2024
4f60a18
Removed SkipIfFailpointDisabled
mjonss Sep 19, 2024
aa8e2bf
Merge remote-tracking branch 'pingcap/master' into truncate-partition…
mjonss Sep 26, 2024
82ed64a
*: Allow Point_Get during DDL with Global Index
mjonss Sep 27, 2024
ace4c13
Added comment
mjonss Sep 27, 2024
25d56a7
Added back check for empty set of partitions after pruning
mjonss Sep 27, 2024
e9424d8
reshuffled code
mjonss Sep 27, 2024
c4f0a13
Added todo comment
mjonss Sep 27, 2024
3a1271a
Clarified comment
mjonss Sep 27, 2024
c34dd49
Merge branch 'global-index-point-get-during-ddl' into truncate-partit…
mjonss Sep 28, 2024
57e140f
Fix for job args and cancellation
mjonss Sep 28, 2024
09daeee
Added back FillFinishedArgs
mjonss Sep 28, 2024
20aabb5
Merge remote-tracking branch 'pingcap/master' into global-index-point…
mjonss Sep 28, 2024
8ed1ce2
Merge branch 'global-index-point-get-during-ddl' into truncate-partit…
mjonss Sep 28, 2024
1a15158
Reverted Point_Get during DDL state
mjonss Oct 4, 2024
2d2d8c1
Merge remote-tracking branch 'pingcap/master' into truncate-partition…
mjonss Oct 4, 2024
338ea36
Restricting PointGet for StateWriteOnly in Truncate Table
mjonss Oct 7, 2024
5c67076
Merge branch 'truncate-partition-global-index' into truncate-partitio…
mjonss Oct 7, 2024
74fdc7e
Merge remote-tracking branch 'pingcap/master' into truncate-partition…
mjonss Oct 9, 2024
afb5e07
Keep truncated partitions in the map of pid->partition
mjonss Oct 9, 2024
8b3c04a
Merge branch 'truncate-partition-global-index' into truncate-partitio…
mjonss Oct 9, 2024
f37f199
Merge remote-tracking branch 'pingcap/master' into truncate-partition…
mjonss Oct 24, 2024
28a029b
Linting
mjonss Oct 24, 2024
2b41559
Merge branch 'truncate-partition-global-index' into truncate-partitio…
mjonss Oct 24, 2024
e41f845
Merge branch 'truncate-partition-global-index' into truncate-partitio…
mjonss Oct 24, 2024
6c13ede
Linting
mjonss Oct 24, 2024
aa181c0
Cleanups + fixed a missing merge line
mjonss Oct 25, 2024
5392e12
Merge branch 'truncate-partition-global-index' into truncate-partitio…
mjonss Oct 25, 2024
81ff60f
Merge remote-tracking branch 'pingcap/master' into truncate-partition…
mjonss Oct 27, 2024
fad0824
Updated comments, and removed optimization for non-global index in tr…
mjonss Oct 28, 2024
a06eca9
Merge branch 'truncate-partition-global-index' into truncate-partitio…
mjonss Oct 28, 2024
02401b1
Merge remote-tracking branch 'pingcap/master' into truncate-partition…
mjonss Oct 29, 2024
9340f08
Merge branch 'truncate-partition-global-index' into truncate-partitio…
mjonss Oct 29, 2024
973f2e1
Merge remote-tracking branch 'pingcap/master' into truncate-partition…
mjonss Oct 30, 2024
fe345d1
Merge branch 'truncate-partition-global-index' into truncate-partitio…
mjonss Oct 30, 2024
7ecf919
Merge remote-tracking branch 'pingcap/master' into truncate-partition…
mjonss Oct 30, 2024
c641a07
Merge branch 'truncate-partition-global-index' into truncate-partitio…
mjonss Oct 30, 2024
8863b8a
Cleaned up unused function.
mjonss Oct 30, 2024
512511b
Fixed tests and optimized/simplified global index overwrite.
mjonss Oct 30, 2024
8e09c8a
Merge remote-tracking branch 'pingcap/master' into truncate-partition…
mjonss Oct 30, 2024
2ab129a
Merge branch 'truncate-partition-global-index' into truncate-partitio…
mjonss Oct 30, 2024
7c29728
Cleanup from non-needed change.
mjonss Oct 30, 2024
2d6d7b2
Cleaunup of non-used function
mjonss Oct 30, 2024
ba3df60
Updated tests, including adding a test for Modify Column anomaly.
mjonss Oct 30, 2024
4391a67
Added test
mjonss Oct 31, 2024
08f27ad
Merge branch 'truncate-partition-global-index' into truncate-partitio…
mjonss Oct 31, 2024
3df45fb
Updated test and simplified rollback logic
mjonss Nov 1, 2024
8470761
Updated test, to be fixed in follow up PR #56029
mjonss Nov 1, 2024
ace2802
Merge remote-tracking branch 'pingcap/master' into truncate-partition…
mjonss Nov 1, 2024
ce93a80
Merge branch 'truncate-partition-global-index' into truncate-partitio…
mjonss Nov 1, 2024
b186402
Updated test.
mjonss Nov 1, 2024
5b14000
Simplified code, reused similar function.
mjonss Nov 4, 2024
4ee1f5e
Merge branch 'truncate-partition-global-index' into truncate-partitio…
mjonss Nov 4, 2024
10a6af5
Test tweak
mjonss Nov 6, 2024
af4bb23
Made test more stable
mjonss Nov 6, 2024
e6032db
moved back code after not implementing optimization
mjonss Nov 12, 2024
29c51a2
Merge remote-tracking branch 'pingcap/master' into truncate-partition…
mjonss Nov 12, 2024
c2bce67
Added a comment about IDsInDDLToIgnore().
mjonss Nov 12, 2024
73963d1
Merge remote-tracking branch 'pingcap/master' into truncate-partition…
mjonss Nov 15, 2024
5d62812
Merge branch 'truncate-partition-global-index' into truncate-partitio…
mjonss Nov 15, 2024
6fcecec
Updated test to make it more stable
mjonss Nov 15, 2024
39216a5
Merge branch 'truncate-partition-global-index' into truncate-partitio…
mjonss Nov 15, 2024
c9b5536
Made test more stable
mjonss Nov 15, 2024
fd67867
Made test more stable
mjonss Nov 15, 2024
bd1492a
Merge branch 'truncate-partition-global-index' into truncate-partitio…
mjonss Nov 15, 2024
2dba3be
Made test more stable
mjonss Nov 15, 2024
825e726
Merge remote-tracking branch 'pingcap/master' into truncate-partition…
mjonss Nov 19, 2024
c8f9c4b
Removed non-used code
mjonss Nov 19, 2024
e903907
Merge branch 'truncate-partition-global-index' into truncate-partitio…
mjonss Nov 20, 2024
8bb440b
Merge remote-tracking branch 'pingcap/master' into truncate-partition…
mjonss Nov 20, 2024
cd842aa
Fix test merge issue
mjonss Nov 20, 2024
0b17dc1
Manual fix of merge
mjonss Nov 20, 2024
dc750d2
Reverted non-important variable name change
mjonss Nov 20, 2024
8a2ce32
removed emtpy line...
mjonss Nov 20, 2024
2c5bce1
Cleanups / reviewer comments
mjonss Nov 22, 2024
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
37 changes: 34 additions & 3 deletions pkg/ddl/partition.go
Original file line number Diff line number Diff line change
Expand Up @@ -2507,6 +2507,18 @@ func (w *worker) onTruncateTablePartition(jobCtx *jobContext, job *model.Job) (i
return ver, errors.Trace(dbterror.ErrPartitionMgmtOnNonpartitioned)
}

if job.IsRollingback() {
return convertTruncateTablePartitionJob2RollbackJob(jobCtx, job, dbterror.ErrCancelledDDLJob, tblInfo)
}

failpoint.Inject("truncatePartCancel1", func(val failpoint.Value) {
if val.(bool) {
job.State = model.JobStateCancelled
err = errors.New("Injected error by truncatePartCancel1")
failpoint.Return(ver, err)
}
})

var oldDefinitions []model.PartitionDefinition
var newDefinitions []model.PartitionDefinition

Expand All @@ -2522,11 +2534,19 @@ func (w *worker) onTruncateTablePartition(jobCtx *jobContext, job *model.Job) (i
pi.DDLState = job.SchemaState
return updateVersionAndTableInfo(jobCtx, job, tblInfo, true)
case model.StateWriteOnly:
// We can still rollback here, since we have not yet started to write to the new partitions!
oldDefinitions, newDefinitions, err = replaceTruncatePartitions(job, jobCtx.metaMut, tblInfo, oldIDs, newIDs)
if err != nil {
return ver, errors.Trace(err)
}
preSplitAndScatter(w.sess.Context, jobCtx.store, tblInfo, newDefinitions)
failpoint.Inject("truncatePartFail1", func(val failpoint.Value) {
if val.(bool) {
job.ErrorCount += variable.GetDDLErrorCountLimit() / 2
err = errors.New("Injected error by truncatePartFail1")
failpoint.Return(ver, err)
}
})
// This work as a flag to ignore Global Index entries from the old partitions!
// Used in IDsInDDLToIgnore() for filtering old partitions from
// the global index
Expand Down Expand Up @@ -2554,6 +2574,13 @@ func (w *worker) onTruncateTablePartition(jobCtx *jobContext, job *model.Job) (i
if err != nil || !done {
return ver, errors.Trace(err)
}
failpoint.Inject("truncatePartFail2", func(val failpoint.Value) {
if val.(bool) {
job.ErrorCount += variable.GetDDLErrorCountLimit() / 2
err = errors.New("Injected error by truncatePartFail2")
failpoint.Return(ver, err)
}
})
// For the truncatePartitionEvent
oldDefinitions = pi.DroppingDefinitions
newDefinitions = make([]model.PartitionDefinition, 0, len(oldIDs))
Expand All @@ -2562,21 +2589,25 @@ func (w *worker) onTruncateTablePartition(jobCtx *jobContext, job *model.Job) (i
newDef.ID = newIDs[i]
newDefinitions = append(newDefinitions, newDef)
}
// TODO: Test injecting failure

pi.DroppingDefinitions = nil
pi.NewPartitionIDs = nil
pi.DDLState = model.StateNone
pi.DDLAction = model.ActionNone

failpoint.Inject("truncatePartFail3", func(val failpoint.Value) {
if val.(bool) {
job.ErrorCount += variable.GetDDLErrorCountLimit() / 2
err = errors.New("Injected error by truncatePartFail3")
failpoint.Return(ver, err)
}
})
// used by ApplyDiff in updateSchemaVersion
args.ShouldUpdateAffectedPartitions = true
ver, err = updateVersionAndTableInfo(jobCtx, job, tblInfo, true)
if err != nil {
return ver, errors.Trace(err)
}
// Finish this job.
job.FinishTableJob(model.JobStateDone, model.StateNone, ver, tblInfo)
truncatePartitionEvent := notifier.NewTruncatePartitionEvent(
tblInfo,
&model.PartitionInfo{Definitions: newDefinitions},
Expand Down
1 change: 1 addition & 0 deletions pkg/ddl/placement_policy_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1956,6 +1956,7 @@ func TestTruncateTablePartitionWithPlacement(t *testing.T) {
" PARTITION `p1` VALUES LESS THAN (1000) /*T![placement] PLACEMENT POLICY=`p2` */,\n" +
" PARTITION `p2` VALUES LESS THAN (10000) /*T![placement] PLACEMENT POLICY=`p3` */,\n" +
" PARTITION `p3` VALUES LESS THAN (100000))"))
dom.Reload()
checkExistTableBundlesInPD(t, dom, "test", "tp")
checkWaitingGCPartitionBundlesInPD(t, dom, checkOldPartitions)

Expand Down
4 changes: 2 additions & 2 deletions pkg/ddl/reorg.go
Original file line number Diff line number Diff line change
Expand Up @@ -932,8 +932,8 @@ func getReorgInfo(ctx *ReorgContext, jobCtx *jobContext, rh *reorgHandler, job *
zap.String("startKey", hex.EncodeToString(start)),
zap.String("endKey", hex.EncodeToString(end)))

failpoint.Inject("errorUpdateReorgHandle", func() (*reorgInfo, error) {
return &info, errors.New("occur an error when update reorg handle")
failpoint.Inject("errorUpdateReorgHandle", func() {
failpoint.Return(&info, errors.New("occur an error when update reorg handle"))
})
err = rh.InitDDLReorgHandle(job, start, end, pid, elements[0])
if err != nil {
Expand Down
39 changes: 38 additions & 1 deletion pkg/ddl/rollingback.go
Original file line number Diff line number Diff line change
Expand Up @@ -316,6 +316,41 @@ func rollingbackExchangeTablePartition(jobCtx *jobContext, job *model.Job) (ver
return ver, errors.Trace(err)
}

func rollingbackTruncateTablePartition(jobCtx *jobContext, job *model.Job) (ver int64, err error) {
tblInfo, err := GetTableInfoAndCancelFaultJob(jobCtx.metaMut, job, job.SchemaID)
if err != nil {
return ver, errors.Trace(err)
}

return convertTruncateTablePartitionJob2RollbackJob(jobCtx, job, dbterror.ErrCancelledDDLJob, tblInfo)
}

func convertTruncateTablePartitionJob2RollbackJob(jobCtx *jobContext, job *model.Job, otherwiseErr error, tblInfo *model.TableInfo) (ver int64, err error) {
if !job.IsRollbackable() {
// Only Original state and StateWrite can be rolled back, otherwise new partitions
// may have been used and new data would get lost.
// So we must continue to roll forward!
job.State = model.JobStateRunning
return ver, nil
}
pi := tblInfo.Partition
if len(pi.NewPartitionIDs) != 0 || pi.DDLAction != model.ActionNone || pi.DDLState != model.StateNone {
// Rollback the changes, note that no new partitions has been used yet!
// so only metadata rollback and we can cancel the DDL
tblInfo.Partition.NewPartitionIDs = nil
tblInfo.Partition.DDLAction = model.ActionNone
tblInfo.Partition.DDLState = model.StateNone
ver, err = updateVersionAndTableInfo(jobCtx, job, tblInfo, true)
if err != nil {
return ver, errors.Trace(err)
}
return ver, nil
}
// No change yet, just cancel the job.
job.State = model.JobStateCancelled
return ver, errors.Trace(otherwiseErr)
}

func convertAddTablePartitionJob2RollbackJob(jobCtx *jobContext, job *model.Job, otherwiseErr error, tblInfo *model.TableInfo) (ver int64, err error) {
addingDefinitions := tblInfo.Partition.AddingDefinitions
partNames := make([]string, 0, len(addingDefinitions))
Expand Down Expand Up @@ -590,8 +625,10 @@ func convertJob2RollbackJob(w *worker, jobCtx *jobContext, job *model.Job) (ver
ver, err = rollingbackTruncateTable(jobCtx, job)
case model.ActionModifyColumn:
ver, err = rollingbackModifyColumn(jobCtx, job)
case model.ActionDropForeignKey, model.ActionTruncateTablePartition:
case model.ActionDropForeignKey:
ver, err = cancelOnlyNotHandledJob(job, model.StatePublic)
case model.ActionTruncateTablePartition:
ver, err = rollingbackTruncateTablePartition(jobCtx, job)
case model.ActionRebaseAutoID, model.ActionShardRowID, model.ActionAddForeignKey,
model.ActionRenameTable, model.ActionRenameTables,
model.ActionModifyTableCharsetAndCollate,
Expand Down
1 change: 1 addition & 0 deletions pkg/ddl/tests/partition/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -5,6 +5,7 @@ go_test(
timeout = "short",
srcs = [
"db_partition_test.go",
"error_injection_test.go",
"main_test.go",
"multi_domain_test.go",
"placement_test.go",
Expand Down
5 changes: 4 additions & 1 deletion pkg/ddl/tests/partition/db_partition_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -3679,15 +3679,18 @@ func TestTruncateNumberOfPhases(t *testing.T) {
tk.MustExec(`create table t (a int primary key , b varchar(255)) partition by hash(a) partitions 3`)
ctx := tk.Session()
dom := domain.GetDomain(ctx)
dom.Reload()
schemaVersion := dom.InfoSchema().SchemaMetaVersion()
tk.MustExec(`insert into t values (1,1),(2,2),(3,3)`)
tk.MustExec(`alter table t truncate partition p1`)
// Without global index, truncate partition should be a single state change
dom.Reload()
// Without global index, truncate partition could be a single state change
require.Equal(t, int64(4), dom.InfoSchema().SchemaMetaVersion()-schemaVersion)
tk.MustExec(`drop table t`)
tk.MustExec(`create table t (a int primary key , b varchar(255), unique key (b) global) partition by hash(a) partitions 3`)
schemaVersion = dom.InfoSchema().SchemaMetaVersion()
tk.MustExec(`insert into t values (1,1),(2,2),(3,3)`)
tk.MustExec(`alter table t truncate partition p1`)
dom.Reload()
require.Equal(t, int64(4), dom.InfoSchema().SchemaMetaVersion()-schemaVersion)
}
211 changes: 211 additions & 0 deletions pkg/ddl/tests/partition/error_injection_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,211 @@
// 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 (
"testing"

"github.com/pingcap/failpoint"
"github.com/pingcap/tidb/pkg/ddl"
"github.com/pingcap/tidb/pkg/testkit"
"github.com/pingcap/tidb/pkg/testkit/external"
"github.com/stretchr/testify/require"
)

type InjectedTest struct {
Name string
Recoverable bool
Rollback bool
}

type FailureTest struct {
FailpointPrefix string
Tests []InjectedTest
}

var truncateTests = FailureTest{
FailpointPrefix: "truncatePart",
Tests: []InjectedTest{
{
Name: "Cancel1",
Recoverable: false,
Rollback: true,
},
{
Name: "Fail1",
Recoverable: true,
Rollback: true,
},
{
Name: "Fail2",
Recoverable: true,
Rollback: false,
},
{
Name: "Fail3",
Recoverable: true,
Rollback: false,
},
},
}

func TestTruncatePartitionListFailuresWithGlobalIndex(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 truncate partition p0,p2`
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")
afterRecover := testkit.Rows("1 1 1", "2 2 2", "8 8 8")
testDDLWithInjectedErrors(t, truncateTests, create, alter, beforeDML, beforeResult, afterDML, afterResult, afterRecover, "Cancel2")
}

func TestTruncatePartitionListFailures(t *testing.T) {
create := `create table t (a int unsigned primary key, b int not null, c varchar(255)) partition by list(a) (
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 truncate partition p0,p2`
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, a = 3 where c = 4`,
`delete from t where a = 8`,
`delete from t where b = 2`,
}
beforeResult := testkit.Rows("3 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, c = 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 6", "7 7 7", "8 8 8")
afterRecover := testkit.Rows("1 1 1", "2 2 2", "8 8 8")
testDDLWithInjectedErrors(t, truncateTests, create, alter, beforeDML, beforeResult, afterDML, afterResult, afterRecover, "Fail1", "Fail2", "Fail3")
}

func testDDLWithInjectedErrors(t *testing.T, tests FailureTest, createSQL, alterSQL string, beforeDML []string, beforeResult [][]any, afterDML []string, afterRollback, afterRecover [][]any, skipTests ...string) {
TEST:
for _, test := range tests.Tests {
for _, skip := range skipTests {
if test.Name == skip {
continue TEST
}
}
if test.Recoverable {
runOneTest(t, test, true, tests.FailpointPrefix, createSQL, alterSQL, beforeDML, beforeResult, afterDML, afterRecover)
}
if test.Rollback {
runOneTest(t, test, false, tests.FailpointPrefix, createSQL, alterSQL, beforeDML, beforeResult, afterDML, afterRollback)
}
}
}

func runOneTest(t *testing.T, test InjectedTest, recoverable bool, failpointName, createSQL, alterSQL string, beforeDML []string, beforeResult [][]any, afterDML []string, afterResult [][]any) {
name := failpointName + test.Name
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")
}()
oldWaitTimeWhenErrorOccurred := ddl.WaitTimeWhenErrorOccurred
defer func() {
ddl.WaitTimeWhenErrorOccurred = oldWaitTimeWhenErrorOccurred
}()
ddl.WaitTimeWhenErrorOccurred = 0
tk.MustExec(createSQL)
for _, sql := range beforeDML {
tk.MustExec(sql + ` /* ` + name + ` */`)
}
tk.MustQuery(`select * from t /* ` + name + ` */`).Sort().Check(beforeResult)
tOrg := external.GetTableByName(t, tk, "test", "t")
idxIDs := make([]int64, 0, len(tOrg.Meta().Indices))
for _, idx := range tOrg.Meta().Indices {
idxIDs = append(idxIDs, idx.ID)
}
pids := make([]int64, 0, len(tOrg.Meta().Partition.Definitions))
for _, def := range tOrg.Meta().Partition.Definitions {
pids = append(pids, def.ID)
}
oldCreate := tk.MustQuery(`show create table t`).Rows()
fullName := "github.com/pingcap/tidb/pkg/ddl/" + name
term := "return(true)"
if recoverable {
// test that it should handle recover/retry on error
term = "1*return(true)"
}
require.NoError(t, failpoint.Enable(fullName, term))
err := tk.ExecToErr(alterSQL + " /* " + name + " */")
require.NoError(t, failpoint.Disable(fullName))
tt := external.GetTableByName(t, tk, "test", "t")
pi := tt.Meta().Partition
if recoverable {
require.NoError(t, err)
equal := true
for i, pid := range pids {
equal = equal && pid == pi.Definitions[i].ID
}
require.False(t, equal, name)
return
}
require.Error(t, err, "failpoint "+name)
require.ErrorContains(t, err, "Injected error by "+name)
tk.MustQuery(`show create table t /* ` + name + ` */`).Check(oldCreate)
require.Equal(t, len(tOrg.Meta().Partition.Definitions), len(pi.Definitions), name)
require.Equal(t, 0, len(pi.AddingDefinitions), name)
require.Equal(t, 0, len(pi.DroppingDefinitions), name)
require.Equal(t, 0, len(pi.NewPartitionIDs), name)
require.Equal(t, len(tOrg.Meta().Indices), len(tt.Meta().Indices), name)
for i := range tOrg.Meta().Indices {
require.Equal(t, idxIDs[i], tt.Meta().Indices[i].ID, name)
}
for i, pid := range pids {
require.Equal(t, pid, tt.Meta().Partition.Definitions[i].ID, name)
}
tk.MustExec(`admin check table t /* ` + name + ` */`)
tk.MustExec(`update t set b = 7 where a = 9 /* ` + name + ` */`)
for _, sql := range afterDML {
tk.MustExec(sql + " /* " + name + " */")
}
tk.MustQuery(`select * from t /* ` + name + ` */`).Sort().Check(afterResult)
tk.MustExec(`drop table t /* ` + name + ` */`)
// TODO: Check no rows on new partitions
// TODO: Check TiFlash replicas
// TODO: Check Label rules
// TODO: Check bundles
// TODO: Check autoIDs
// TODO: Check delete_range tables, so no delete request for old partitions in failed alters!
}
Loading