From 6df378f472a3d5c1d76add1de44c61109da73e6c Mon Sep 17 00:00:00 2001 From: Mattias Jonsson Date: Tue, 22 Oct 2024 16:40:57 +0200 Subject: [PATCH 01/25] Better index management during rollback of partitioning DDLs --- pkg/ddl/partition.go | 21 +++-- pkg/ddl/rollingback.go | 29 +++++-- pkg/ddl/tests/partition/multi_domain_test.go | 84 +++++++++++++++++-- .../tests/partition/reorg_partition_test.go | 45 ++++++++-- 4 files changed, 148 insertions(+), 31 deletions(-) diff --git a/pkg/ddl/partition.go b/pkg/ddl/partition.go index edd1a2e7d5f6f..dac075b81a174 100644 --- a/pkg/ddl/partition.go +++ b/pkg/ddl/partition.go @@ -2176,8 +2176,8 @@ func (w *worker) rollbackLikeDropPartition(jobCtx *jobContext, job *model.Job) ( var dropIndices []*model.IndexInfo for _, indexInfo := range tblInfo.Indices { if indexInfo.Unique && - indexInfo.State == model.StateDeleteReorganization && - tblInfo.Partition.DDLState == model.StateDeleteReorganization { + indexInfo.State == model.StateWriteOnly && + tblInfo.Partition.DDLState == model.StateWriteReorganization { dropIndices = append(dropIndices, indexInfo) } } @@ -3452,17 +3452,17 @@ func (w *worker) onReorganizePartition(jobCtx *jobContext, job *model.Job) (ver index.State = model.StatePublic case model.StatePublic: if index.Global { - // Mark the old global index as non-readable, and to be dropped - index.State = model.StateDeleteReorganization + // Mark the old global index as non-readable, but still writable, and to be dropped + index.State = model.StateWriteOnly } else { inAllPartitionColumns, err := checkPartitionKeysConstraint(partInfo, index.Columns, tblInfo) if err != nil { return rollbackReorganizePartitionWithErr(jobCtx, job, err) } if !inAllPartitionColumns { - // Mark the old unique index as non-readable, and to be dropped, + // Mark the old unique index as non-readable, but still writable, and to be dropped, // since it is replaced by a global index - index.State = model.StateDeleteReorganization + index.State = model.StateWriteOnly } } } @@ -3493,9 +3493,9 @@ func (w *worker) onReorganizePartition(jobCtx *jobContext, job *model.Job) (ver // Now all the data copying is done, but we cannot simply remove the droppingDefinitions // since they are a part of the normal Definitions that other nodes with // the current schema version. So we need to double write for one more schema version - tblInfo.Partition.DDLState = model.StateDeleteReorganization - ver, err = updateVersionAndTableInfo(jobCtx, job, tblInfo, true) job.SchemaState = model.StateDeleteReorganization + tblInfo.Partition.DDLState = job.SchemaState + ver, err = updateVersionAndTableInfo(jobCtx, job, tblInfo, true) case model.StateDeleteReorganization: // Drop the droppingDefinitions and finish the DDL @@ -3521,7 +3521,7 @@ func (w *worker) onReorganizePartition(jobCtx *jobContext, job *model.Job) (ver var dropIndices []*model.IndexInfo for _, indexInfo := range tblInfo.Indices { - if indexInfo.Unique && indexInfo.State == model.StateDeleteReorganization { + if indexInfo.Unique && indexInfo.State == model.StateWriteOnly { // Drop the old unique (possible global) index, see onDropIndex indexInfo.State = model.StateNone DropIndexColumnFlag(tblInfo, indexInfo) @@ -3591,6 +3591,9 @@ func (w *worker) onReorganizePartition(jobCtx *jobContext, job *model.Job) (ver } }) job.CtxVars = []any{physicalTableIDs, newIDs} + failpoint.Inject("updateVersionAndTableInfoErrInStateDeleteReorganization", func() { + failpoint.Return(ver, errors.New("Injected error in StateDeleteReorganization")) + }) ver, err = updateVersionAndTableInfo(jobCtx, job, tblInfo, true) if err != nil { return ver, errors.Trace(err) diff --git a/pkg/ddl/rollingback.go b/pkg/ddl/rollingback.go index d20999b8f8d46..66e713697e60b 100644 --- a/pkg/ddl/rollingback.go +++ b/pkg/ddl/rollingback.go @@ -354,8 +354,8 @@ func convertAddTablePartitionJob2RollbackJob(jobCtx *jobContext, job *model.Job, if err != nil { return ver, errors.Trace(err) } - tblInfo.Partition.DDLState = model.StateNone - tblInfo.Partition.DDLAction = model.ActionNone + //tblInfo.Partition.DDLState = model.StateNone + //tblInfo.Partition.DDLAction = model.ActionNone job.State = model.JobStateRollingback return ver, errors.Trace(otherwiseErr) } @@ -384,15 +384,22 @@ func convertReorgPartitionJob2RollbackJob(jobCtx *jobContext, job *model.Job, ot } var dropIndices []*model.IndexInfo // When Global Index is duplicated to a non Global, we later need - // to know if if it was Global before (marked to be dropped) or not. + // to know if it was Global before (marked to be dropped) or not. globalToUniqueDupMap := make(map[string]int64) for _, indexInfo := range tblInfo.Indices { if !indexInfo.Unique { continue } switch indexInfo.State { - case model.StateWriteReorganization, model.StateDeleteOnly, - model.StateWriteOnly: + case model.StateWriteOnly: + if pi.DDLState == model.StateDeleteReorganization { + // Revert the non-public state + indexInfo.State = model.StatePublic + globalToUniqueDupMap[indexInfo.Name.L] = indexInfo.ID + } else { + dropIndices = append(dropIndices, indexInfo) + } + case model.StateDeleteOnly, model.StateWriteReorganization: dropIndices = append(dropIndices, indexInfo) case model.StateDeleteReorganization: if pi.DDLState != model.StateDeleteReorganization { @@ -416,7 +423,7 @@ func convertReorgPartitionJob2RollbackJob(jobCtx *jobContext, job *model.Job, ot // New index that became public in this state, // mark it to be dropped in next schema version if indexInfo.Global { - indexInfo.State = model.StateDeleteReorganization + indexInfo.State = model.StateWriteOnly } else { // How to know if this index was created as a duplicate or not? if id, ok := globalToUniqueDupMap[indexInfo.Name.L]; ok { @@ -424,7 +431,7 @@ func convertReorgPartitionJob2RollbackJob(jobCtx *jobContext, job *model.Job, ot if id >= indexInfo.ID { return ver, errors.NewNoStackErrorf("Indexes in wrong order during rollback, '%s', %d >= %d", indexInfo.Name.O, id, indexInfo.ID) } - indexInfo.State = model.StateDeleteReorganization + indexInfo.State = model.StateWriteOnly } else { globalToUniqueDupMap[indexInfo.Name.L] = indexInfo.ID } @@ -478,13 +485,19 @@ func convertReorgPartitionJob2RollbackJob(jobCtx *jobContext, job *model.Job, ot return ver, errors.Trace(errors.New("Internal error, failed to find original partition definitions")) } pi.Definitions = newDefs - pi.Num = uint64(len(pi.Definitions)) } else { + // Move back to StateWriteReorganization, i.e. use the original table + // (non-partitioned or differently partitioned) as the main table to use. + // Otherwise, the Type does not match the expression. pi.Type, pi.DDLType = pi.DDLType, pi.Type pi.Expr, pi.DDLExpr = pi.DDLExpr, pi.Expr pi.Columns, pi.DDLColumns = pi.DDLColumns, pi.Columns pi.Definitions = pi.DroppingDefinitions } + pi.Num = uint64(len(pi.Definitions)) + // We should move back one state, since there might be other sessions seeing the new partitions. + job.SchemaState = model.StateWriteReorganization + pi.DDLState = job.SchemaState } args, err := model.GetTablePartitionArgs(job) diff --git a/pkg/ddl/tests/partition/multi_domain_test.go b/pkg/ddl/tests/partition/multi_domain_test.go index 1c567c9a4afaf..9867847523b02 100644 --- a/pkg/ddl/tests/partition/multi_domain_test.go +++ b/pkg/ddl/tests/partition/multi_domain_test.go @@ -18,10 +18,10 @@ import ( "testing" "time" - "github.com/pingcap/failpoint" "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/session" "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" @@ -249,16 +249,81 @@ func TestMultiSchemaDropListColumnsDefaultPartition(t *testing.T) { runMultiSchemaTest(t, createSQL, alterSQL, initFn, func(kit *testkit.TestKit) {}, loopFn) } +func TestMultiSchemaReorganizePartition(t *testing.T) { + createSQL := `create table t (a int primary key, b varchar(255)) partition by range (a) (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 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 "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)`, "[kv:1062]Duplicate entry '1' for key 't.PRIMARY'") + tkNO.MustContainErrMsg(`insert into t values (1,1)`, "[kv:1062]Duplicate entry '1' for key 't.PRIMARY'") + tkO.MustContainErrMsg(`insert into t values (101,101)`, "[kv:1062]Duplicate entry '101' for key 't.PRIMARY'") + tkNO.MustContainErrMsg(`insert into t values (101,101)`, "[kv:1062]Duplicate entry '101' for key 't.PRIMARY'") + 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("1 1", "101 101", "102 102", "2 2")) + 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.MustContainErrMsg(`insert into t values (1,2)`, "[kv:1062]Duplicate entry '1' for key 't.PRIMARY'") + tkNO.MustContainErrMsg(`insert into t values (1,2)`, "[kv:1062]Duplicate entry '1' for key 't.PRIMARY'") + tkO.MustContainErrMsg(`insert into t values (101,101)`, "[kv:1062]Duplicate entry '101' for key 't.PRIMARY'") + tkNO.MustContainErrMsg(`insert into t values (101,101)`, "[kv:1062]Duplicate entry '101' for key 't.PRIMARY'") + 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")) + case "write reorganization": + // just to not fail :) + case "delete reorganization": + // TODO: Test Global/Unique Key updates! May need more indexes? + // 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" + + ") 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" + + ") 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 "none": + // just to not fail :) + default: + require.Failf(t, "unhandled schema state '%s'", schemaState) + } + } + runMultiSchemaTest(t, createSQL, alterSQL, initFn, func(kit *testkit.TestKit) {}, loopFn) +} + func runMultiSchemaTest(t *testing.T, createSQL, alterSQL string, initFn, postFn func(*testkit.TestKit), loopFn func(tO, tNO *testkit.TestKit)) { distCtx := testkit.NewDistExecutionContextWithLease(t, 2, 15*time.Second) store := distCtx.Store domOwner := distCtx.GetDomain(0) domNonOwner := distCtx.GetDomain(1) - defer func() { - domOwner.Close() - domNonOwner.Close() - store.Close() - }() if !domOwner.DDL().OwnerManager().IsOwner() { domOwner, domNonOwner = domNonOwner, domOwner @@ -290,8 +355,7 @@ func runMultiSchemaTest(t *testing.T, createSQL, alterSQL string, initFn, postFn <-hookChan logutil.BgLogger().Info("XXXXXXXXXXX Hook released", zap.String("job.State", job.State.String()), zap.String("job.SchemaStage", job.SchemaState.String())) } - failpoint.EnableCall("github.com/pingcap/tidb/pkg/ddl/onJobRunAfter", hookFunc) - defer failpoint.Disable("github.com/pingcap/tidb/pkg/ddl/onJobRunAfter") + testfailpoint.EnableCall(t, "github.com/pingcap/tidb/pkg/ddl/onJobRunAfter", hookFunc) alterChan := make(chan struct{}) go func() { tkDDLOwner.MustExec(alterSQL) @@ -339,4 +403,8 @@ func runMultiSchemaTest(t *testing.T, createSQL, alterSQL string, initFn, postFn } logutil.BgLogger().Info("XXXXXXXXXXX states loop done") postFn(tkO) + // NOT deferring this, since it might hang on test failures... + domOwner.Close() + domNonOwner.Close() + store.Close() } diff --git a/pkg/ddl/tests/partition/reorg_partition_test.go b/pkg/ddl/tests/partition/reorg_partition_test.go index ca5f8e07e3493..fa27c25e9039b 100644 --- a/pkg/ddl/tests/partition/reorg_partition_test.go +++ b/pkg/ddl/tests/partition/reorg_partition_test.go @@ -237,7 +237,7 @@ func TestPartitionByFailures(t *testing.T) { `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) + testReorganizePartitionFailures(t, create, alter, beforeDML, beforeResult, afterDML, afterResult, "Cancel1", "Fail1", "Fail2") } func TestReorganizePartitionListFailures(t *testing.T) { @@ -289,6 +289,7 @@ func TestPartitionByListFailures(t *testing.T) { } 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) { @@ -337,6 +338,14 @@ func TestCoalesceKeyPartitionFailures(t *testing.T) { testReorganizePartitionFailures(t, create, alter, beforeDML, beforeResult, afterDML, afterResult, "Fail4") } +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, "Fail4") +} + 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) @@ -386,7 +395,10 @@ func testReorganizePartitionFailures(t *testing.T, createSQL, alterSQL string, b } tk.MustQuery(`select * from t /* ` + suffix + ` */`).Sort().Check(beforeResult) tOrg := external.GetTableByName(t, tk, "test", "t") - idxID := tOrg.Meta().Indices[0].ID + var idxID int64 + if len(tOrg.Meta().Indices) > 0 { + idxID = tOrg.Meta().Indices[0].ID + } oldCreate := tk.MustQuery(`show create table t`).Rows() name := "github.com/pingcap/tidb/pkg/ddl/reorgPart" + suffix testfailpoint.Enable(t, name, `return(true)`) @@ -397,11 +409,17 @@ func testReorganizePartitionFailures(t *testing.T, createSQL, alterSQL string, b tk.MustQuery(`show create table t /* ` + suffix + ` */`).Check(oldCreate) tt := external.GetTableByName(t, tk, "test", "t") partition := tt.Meta().Partition - 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) + 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) + } require.Equal(t, len(tOrg.Meta().Indices), len(tt.Meta().Indices), suffix) - require.Equal(t, idxID, tt.Meta().Indices[0].ID, suffix) + if idxID != 0 { + require.Equal(t, idxID, tt.Meta().Indices[0].ID, suffix) + } noNewTablesAfter(t, tk, tk.Session(), tOrg, suffix) tk.MustExec(`admin check table t /* ` + suffix + ` */`) for _, sql := range afterDML { @@ -918,3 +936,18 @@ func TestPartitionByColumnChecks(t *testing.T) { 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", `return(1)`) + defer func() { + testfailpoint.Disable(t, "github.com/pingcap/tidb/pkg/ddl/updateVersionAndTableInfoErrInStateDeleteReorganization") + }() + + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("set global tidb_ddl_error_count_limit = 3") + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t (a int)") + tk.MustContainErrMsg("alter table t partition by range(a) (partition p1 values less than (20))", "[ddl:-1]DDL job rollback, error msg: Injected error in StateDeleteReorganization") // should NOT panic +} From 27978e0c30a7f5b3c36410cd6875ad982e4a0272 Mon Sep 17 00:00:00 2001 From: Mattias Jonsson Date: Tue, 22 Oct 2024 16:53:21 +0200 Subject: [PATCH 02/25] Linting --- pkg/ddl/rollingback.go | 2 -- pkg/ddl/tests/partition/reorg_partition_test.go | 6 +----- 2 files changed, 1 insertion(+), 7 deletions(-) diff --git a/pkg/ddl/rollingback.go b/pkg/ddl/rollingback.go index 66e713697e60b..02ddf7b0b8506 100644 --- a/pkg/ddl/rollingback.go +++ b/pkg/ddl/rollingback.go @@ -354,8 +354,6 @@ func convertAddTablePartitionJob2RollbackJob(jobCtx *jobContext, job *model.Job, if err != nil { return ver, errors.Trace(err) } - //tblInfo.Partition.DDLState = model.StateNone - //tblInfo.Partition.DDLAction = model.ActionNone job.State = model.JobStateRollingback return ver, errors.Trace(otherwiseErr) } diff --git a/pkg/ddl/tests/partition/reorg_partition_test.go b/pkg/ddl/tests/partition/reorg_partition_test.go index fa27c25e9039b..ea6ac72f75480 100644 --- a/pkg/ddl/tests/partition/reorg_partition_test.go +++ b/pkg/ddl/tests/partition/reorg_partition_test.go @@ -237,7 +237,7 @@ func TestPartitionByFailures(t *testing.T) { `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, "Cancel1", "Fail1", "Fail2") + testReorganizePartitionFailures(t, create, alter, beforeDML, beforeResult, afterDML, afterResult) } func TestReorganizePartitionListFailures(t *testing.T) { @@ -289,7 +289,6 @@ func TestPartitionByListFailures(t *testing.T) { } 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) { @@ -939,9 +938,6 @@ func TestPartitionByColumnChecks(t *testing.T) { func TestPartitionIssue56634(t *testing.T) { testfailpoint.Enable(t, "github.com/pingcap/tidb/pkg/ddl/updateVersionAndTableInfoErrInStateDeleteReorganization", `return(1)`) - defer func() { - testfailpoint.Disable(t, "github.com/pingcap/tidb/pkg/ddl/updateVersionAndTableInfoErrInStateDeleteReorganization") - }() store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) From 10f5f61f5af975a82cc5e0250296734ffaa7c9ef Mon Sep 17 00:00:00 2001 From: Mattias Jonsson Date: Wed, 23 Oct 2024 23:34:51 +0200 Subject: [PATCH 03/25] Reworked index handling in Reorganize Partition. Still some debug and cleanup to do. --- pkg/ddl/partition.go | 36 ++-- pkg/ddl/tests/partition/multi_domain_test.go | 198 ++++++++++++++++--- pkg/meta/model/table.go | 16 +- pkg/table/tables/index.go | 3 + pkg/table/tables/partition.go | 72 ++++--- 5 files changed, 252 insertions(+), 73 deletions(-) diff --git a/pkg/ddl/partition.go b/pkg/ddl/partition.go index bbbfa07f76df3..fa9186dff366e 100644 --- a/pkg/ddl/partition.go +++ b/pkg/ddl/partition.go @@ -3142,6 +3142,9 @@ func getReorgPartitionInfo(t *meta.Mutator, job *model.Job, args *model.TablePar // // Everything now looks as it should, no memory of old partitions/indexes, // and no more double writing, since the previous state is only reading the new partitions/indexes. +// +// Note: Special handling is also required in tables.newPartitionedTable(), +// to get per partition indexes in the right state. func (w *worker) onReorganizePartition(jobCtx *jobContext, job *model.Job) (ver int64, _ error) { args, err := model.GetTablePartitionArgs(job) if err != nil { @@ -3253,6 +3256,9 @@ func (w *worker) onReorganizePartition(jobCtx *jobContext, job *model.Job) (ver if err != nil { return ver, errors.Trace(err) } + if tblInfo.Partition.DDLChangedIndex == nil { + tblInfo.Partition.DDLChangedIndex = make(map[int64]bool) + } if !inAllPartitionColumns { // Currently only support Explicit Global indexes. if !newGlobal { @@ -3270,6 +3276,8 @@ func (w *worker) onReorganizePartition(jobCtx *jobContext, job *model.Job) (ver newIndex := index.Clone() newIndex.State = model.StateDeleteOnly newIndex.ID = AllocateIndexID(tblInfo) + tblInfo.Partition.DDLChangedIndex[index.ID] = false + tblInfo.Partition.DDLChangedIndex[newIndex.ID] = true newIndex.Global = true tblInfo.Indices = append(tblInfo.Indices, newIndex) } else { @@ -3282,6 +3290,8 @@ func (w *worker) onReorganizePartition(jobCtx *jobContext, job *model.Job) (ver newIndex := index.Clone() newIndex.State = model.StateDeleteOnly newIndex.ID = AllocateIndexID(tblInfo) + tblInfo.Partition.DDLChangedIndex[index.ID] = false + tblInfo.Partition.DDLChangedIndex[newIndex.ID] = true newIndex.Global = false tblInfo.Indices = append(tblInfo.Indices, newIndex) } @@ -3477,26 +3487,18 @@ func (w *worker) onReorganizePartition(jobCtx *jobContext, job *model.Job) (ver if !index.Unique { continue } - switch index.State { - case model.StateWriteReorganization: + isNew, ok := tblInfo.Partition.DDLChangedIndex[index.ID] + if !ok { + continue + } + if isNew { // Newly created index, replacing old unique/global index index.State = model.StatePublic - case model.StatePublic: - if index.Global { - // Mark the old global index as non-readable, but still writable, and to be dropped - index.State = model.StateWriteOnly - } else { - inAllPartitionColumns, err := checkPartitionKeysConstraint(partInfo, index.Columns, tblInfo) - if err != nil { - return rollbackReorganizePartitionWithErr(jobCtx, job, err) - } - if !inAllPartitionColumns { - // Mark the old unique index as non-readable, but still writable, and to be dropped, - // since it is replaced by a global index - index.State = model.StateWriteOnly - } - } + continue } + // Old index, should not be visible any longer, + // but needs to be kept up-to-date in case rollback happens. + index.State = model.StateWriteOnly } firstPartIdx, lastPartIdx, idMap, err2 := getReplacedPartitionIDs(partNames, tblInfo.Partition) if err2 != nil { diff --git a/pkg/ddl/tests/partition/multi_domain_test.go b/pkg/ddl/tests/partition/multi_domain_test.go index 9867847523b02..e04463efa9b3d 100644 --- a/pkg/ddl/tests/partition/multi_domain_test.go +++ b/pkg/ddl/tests/partition/multi_domain_test.go @@ -250,7 +250,7 @@ func TestMultiSchemaDropListColumnsDefaultPartition(t *testing.T) { } func TestMultiSchemaReorganizePartition(t *testing.T) { - createSQL := `create table t (a int primary key, b varchar(255)) partition by range (a) (partition p1 values less than (200))` + 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))` initFn := func(tkO *testkit.TestKit) { tkO.MustExec(`insert into t values (1,1),(2,2),(101,101),(102,102)`) } @@ -259,24 +259,14 @@ func TestMultiSchemaReorganizePartition(t *testing.T) { 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 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)`, "[kv:1062]Duplicate entry '1' for key 't.PRIMARY'") - tkNO.MustContainErrMsg(`insert into t values (1,1)`, "[kv:1062]Duplicate entry '1' for key 't.PRIMARY'") - tkO.MustContainErrMsg(`insert into t values (101,101)`, "[kv:1062]Duplicate entry '101' for key 't.PRIMARY'") - tkNO.MustContainErrMsg(`insert into t values (101,101)`, "[kv:1062]Duplicate entry '101' for key 't.PRIMARY'") - 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("1 1", "101 101", "102 102", "2 2")) - 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.MustContainErrMsg(`insert into t values (1,2)`, "[kv:1062]Duplicate entry '1' for key 't.PRIMARY'") - tkNO.MustContainErrMsg(`insert into t values (1,2)`, "[kv:1062]Duplicate entry '1' for key 't.PRIMARY'") - tkO.MustContainErrMsg(`insert into t values (101,101)`, "[kv:1062]Duplicate entry '101' for key 't.PRIMARY'") - tkNO.MustContainErrMsg(`insert into t values (101,101)`, "[kv:1062]Duplicate entry '101' for key 't.PRIMARY'") + 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_global'") + tkNO.MustContainErrMsg(`insert into t values (1,2)`, "[kv:1062]Duplicate entry '2' for key 't.idx_b_global'") + tkO.MustContainErrMsg(`insert into t values (101,101)`, "[kv:1062]Duplicate entry '101' for key 't.idx_b_global'") + tkNO.MustContainErrMsg(`insert into t values (101,101)`, "[kv:1062]Duplicate entry '101' for key 't.idx_b_global'") 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")) @@ -286,17 +276,52 @@ func TestMultiSchemaReorganizePartition(t *testing.T) { 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")) - case "write reorganization": - // just to not fail :) - case "delete reorganization": - // TODO: Test Global/Unique Key updates! May need more indexes? + 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_global'") + tkNO.MustContainErrMsg(`insert into t values (1,1)`, "[kv:1062]Duplicate entry '1' for key 't.idx_b_global'") + tkO.MustContainErrMsg(`insert into t values (101,101)`, "[kv:1062]Duplicate entry '101' for key 't.idx_b_global'") + tkNO.MustContainErrMsg(`insert into t values (101,101)`, "[kv:1062]Duplicate entry '101' for key 't.idx_b_global'") + 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" + + " 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))")) @@ -304,21 +329,136 @@ func TestMultiSchemaReorganizePartition(t *testing.T) { "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" + + " 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))")) - tkO.MustExec(`insert into t values (3,3)`) - case "none": - // just to not fail :) + case model.StateNone.String(): + tkO.MustExec(`insert into t values (11,11)`) + tkNO.MustExec(`insert into t values (12,12)`) + tkNO.MustQuery(`select * from t where b = 11`).Check(testkit.Rows("11 11")) + tkO.MustQuery(`select * from t where b = 12`).Check(testkit.Rows("12 12")) default: require.Failf(t, "unhandled schema state '%s'", schemaState) } } - runMultiSchemaTest(t, createSQL, alterSQL, initFn, func(kit *testkit.TestKit) {}, loopFn) + 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`) + tkO.MustQuery(`select * from t`).Sort().Check(testkit.Rows("1 1", "10 10", "101 101", "102 102", "12 12", "2 2", "3 3", "4 4", "5 5", "6 6", "7 7", "8 8", "9 9")) + } + runMultiSchemaTest(t, createSQL, alterSQL, initFn, postFn, 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, postFn func(*testkit.TestKit), loopFn func(tO, tNO *testkit.TestKit)) { distCtx := testkit.NewDistExecutionContextWithLease(t, 2, 15*time.Second) store := distCtx.Store diff --git a/pkg/meta/model/table.go b/pkg/meta/model/table.go index 0404895d92a14..15a520904db7f 100644 --- a/pkg/meta/model/table.go +++ b/pkg/meta/model/table.go @@ -744,14 +744,19 @@ type PartitionInfo struct { // like if there is a global index or going between non-partitioned // and partitioned table, to make the data dropping / range delete // optimized. - NewTableID int64 `json:"new_table_id"` + NewTableID int64 `json:"new_table_id,omitempty"` // Set during ALTER TABLE ... PARTITION BY ... // First as the new partition scheme, then in StateDeleteReorg as the old - DDLType model.PartitionType `json:"ddl_type"` - DDLExpr string `json:"ddl_expr"` - DDLColumns []model.CIStr `json:"ddl_columns"` + DDLType model.PartitionType `json:"ddl_type,omitempty"` + DDLExpr string `json:"ddl_expr,omitempty"` + DDLColumns []model.CIStr `json:"ddl_columns,omitempty"` // For ActionAlterTablePartitioning, UPDATE INDEXES - DDLUpdateIndexes []UpdateIndexInfo `json:"ddl_update_indexes"` + DDLUpdateIndexes []UpdateIndexInfo `json:"ddl_update_indexes,omitempty"` + // Simplified way to handle Global Index changes, instead of calculating + // it every time, keep track of the changes here. + // if index.ID exists in map, then it has changed, true for new copy, + // false for old copy (to be removed). + DDLChangedIndex map[int64]bool `json:"ddl_changed_index,omitempty"` } // Clone clones itself. @@ -856,6 +861,7 @@ func (pi *PartitionInfo) ClearReorgIntermediateInfo() { pi.DDLExpr = "" pi.DDLColumns = nil pi.NewTableID = 0 + pi.DDLChangedIndex = nil } // FindPartitionDefinitionByName finds PartitionDefinition by name. diff --git a/pkg/table/tables/index.go b/pkg/table/tables/index.go index dda35219affc5..c3674fcdd7fbf 100644 --- a/pkg/table/tables/index.go +++ b/pkg/table/tables/index.go @@ -16,6 +16,8 @@ package tables import ( "context" + "github.com/pingcap/tidb/pkg/util/logutil" + "go.uber.org/zap" "sync" "time" @@ -280,6 +282,7 @@ func (c *index) create(sctx table.MutateContext, txn kv.Transaction, indexedValu continue } + logutil.BgLogger().Info("create key", zap.Binary("key", key), zap.String("handle as string", h.String()), zap.Int64("tableID", c.tblInfo.ID), zap.Int64("physID", c.phyTblID), zap.Int64("indexID", c.idxInfo.ID), zap.Bool("global", c.idxInfo.Global)) var value []byte if c.tblInfo.TempTableType != model.TempTableNone { // Always check key for temporary table because it does not write to TiKV diff --git a/pkg/table/tables/partition.go b/pkg/table/tables/partition.go index 5da631be92d20..8344950957210 100644 --- a/pkg/table/tables/partition.go +++ b/pkg/table/tables/partition.go @@ -99,9 +99,9 @@ type partitionedTable struct { // Only used during Reorganize partition // reorganizePartitions is the currently used partitions that are reorganized - reorganizePartitions map[int64]any + reorganizePartitions map[int64]struct{} // doubleWritePartitions are the partitions not visible, but we should double write to - doubleWritePartitions map[int64]any + doubleWritePartitions map[int64]struct{} reorgPartitionExpr *PartitionExpr } @@ -112,7 +112,8 @@ func newPartitionedTable(tbl *TableCommon, tblInfo *model.TableInfo) (table.Part if pi == nil || len(pi.Definitions) == 0 { return nil, table.ErrUnknownPartition } - ret := &partitionedTable{TableCommon: tbl.Copy()} + tblCommon := tbl.Copy() + ret := &partitionedTable{TableCommon: tblCommon} partitionExpr, err := newPartitionExpr(tblInfo, pi.Type, pi.Expr, pi.Columns, pi.Definitions) if err != nil { return nil, errors.Trace(err) @@ -127,6 +128,20 @@ func newPartitionedTable(tbl *TableCommon, tblInfo *model.TableInfo) (table.Part if err := initTableIndices(&ret.TableCommon); err != nil { return nil, errors.Trace(err) } + origIndices := ret.meta.Indices + currIndices := make([]*model.IndexInfo, 0, len(origIndices)) + useNew := false + if pi.DDLState == model.StateDeleteReorganization { + useNew = true + } + for _, idx := range origIndices { + // Filter out so only the old OR new indexes are used. + if newIdx, ok := pi.DDLChangedIndex[idx.ID]; !ok || newIdx == useNew { + currIndices = append(currIndices, idx) + } + } + tblInfo.Indices = currIndices + defer func() { ret.meta.Indices = origIndices }() partitions := make(map[int64]*partition, len(pi.Definitions)) for _, p := range pi.Definitions { var t partition @@ -149,10 +164,18 @@ func newPartitionedTable(tbl *TableCommon, tblInfo *model.TableInfo) (table.Part // In StateDeleteReorganization we are using the 'new' partition definitions // and if any new change happens in AddingDefinitions, it needs to be done // also in DroppingDefinitions (since session running on schema version -1) - // should also see the changes + // should also see the changes. + useNew = !useNew + currIndices = currIndices[:0] + for _, idx := range origIndices { + // Filter out the opposite changed indexes from above, + // so the reorganized partition (not visible) have the matching indices. + if newIdx, ok := pi.DDLChangedIndex[idx.ID]; !ok || newIdx == useNew { + currIndices = append(currIndices, idx) + } + } + ret.meta.Indices = currIndices if pi.DDLState == model.StateDeleteReorganization { - origIdx := setIndexesState(ret, pi.DDLState) - defer unsetIndexesState(ret, origIdx) // TODO: Explicitly explain the different DDL/New fields! if pi.NewTableID != 0 { ret.reorgPartitionExpr, err = newPartitionExpr(tblInfo, pi.DDLType, pi.DDLExpr, pi.DDLColumns, pi.DroppingDefinitions) @@ -162,23 +185,25 @@ func newPartitionedTable(tbl *TableCommon, tblInfo *model.TableInfo) (table.Part if err != nil { return nil, errors.Trace(err) } - ret.reorganizePartitions = make(map[int64]any, len(pi.AddingDefinitions)) + ret.reorganizePartitions = make(map[int64]struct{}, len(pi.AddingDefinitions)) for _, def := range pi.AddingDefinitions { - ret.reorganizePartitions[def.ID] = nil + ret.reorganizePartitions[def.ID] = struct{}{} } - ret.doubleWritePartitions = make(map[int64]any, len(pi.DroppingDefinitions)) - for _, def := range pi.DroppingDefinitions { - p, err := initPartition(ret, def) - if err != nil { - return nil, err + if len(pi.DroppingDefinitions) > 0 { + ret.doubleWritePartitions = make(map[int64]struct{}, len(pi.DroppingDefinitions)) + for _, def := range pi.DroppingDefinitions { + p, err := initPartition(ret, def) + if err != nil { + return nil, err + } + partitions[def.ID] = p + ret.doubleWritePartitions[def.ID] = struct{}{} } - partitions[def.ID] = p - ret.doubleWritePartitions[def.ID] = nil } } else { if len(pi.AddingDefinitions) > 0 { - origIdx := setIndexesState(ret, pi.DDLState) - defer unsetIndexesState(ret, origIdx) + //origIdx := setIndexesState(ret, pi.DDLState) + //defer unsetIndexesState(ret, origIdx) if pi.NewTableID != 0 { // REMOVE PARTITIONING or PARTITION BY ret.reorgPartitionExpr, err = newPartitionExpr(tblInfo, pi.DDLType, pi.DDLExpr, pi.DDLColumns, pi.AddingDefinitions) @@ -189,9 +214,9 @@ func newPartitionedTable(tbl *TableCommon, tblInfo *model.TableInfo) (table.Part if err != nil { return nil, errors.Trace(err) } - ret.doubleWritePartitions = make(map[int64]any, len(pi.AddingDefinitions)) + ret.doubleWritePartitions = make(map[int64]struct{}, len(pi.AddingDefinitions)) for _, def := range pi.AddingDefinitions { - ret.doubleWritePartitions[def.ID] = nil + ret.doubleWritePartitions[def.ID] = struct{}{} p, err := initPartition(ret, def) if err != nil { return nil, err @@ -200,21 +225,24 @@ func newPartitionedTable(tbl *TableCommon, tblInfo *model.TableInfo) (table.Part } } if len(pi.DroppingDefinitions) > 0 { - ret.reorganizePartitions = make(map[int64]any, len(pi.DroppingDefinitions)) + ret.reorganizePartitions = make(map[int64]struct{}, len(pi.DroppingDefinitions)) for _, def := range pi.DroppingDefinitions { - ret.reorganizePartitions[def.ID] = nil + ret.reorganizePartitions[def.ID] = struct{}{} } } } return ret, nil } +// setIndexesState is used to alter the index schema state for non-public partitions, +// during reorganize partition. func setIndexesState(t *partitionedTable, state model.SchemaState) []*model.IndexInfo { orig := t.meta.Indices t.meta.Indices = make([]*model.IndexInfo, 0, len(orig)) for i := range orig { t.meta.Indices = append(t.meta.Indices, orig[i].Clone()) - if t.meta.Indices[i].State == model.StatePublic { + switch t.meta.Indices[i].State { + case model.StatePublic: switch state { case model.StateDeleteOnly, model.StateNone: t.meta.Indices[i].State = model.StateDeleteOnly From cc4f5a8dee8f080abb7b8d19e47350958e9d1b7f Mon Sep 17 00:00:00 2001 From: Mattias Jonsson Date: Thu, 24 Oct 2024 00:37:50 +0200 Subject: [PATCH 04/25] Fixed assertion issues. --- pkg/ddl/partition.go | 4 +++ pkg/ddl/rollingback.go | 51 +++++++---------------------------- pkg/table/tables/index.go | 3 --- pkg/table/tables/partition.go | 11 ++++---- 4 files changed, 19 insertions(+), 50 deletions(-) diff --git a/pkg/ddl/partition.go b/pkg/ddl/partition.go index fa9186dff366e..4c07eb1536c6c 100644 --- a/pkg/ddl/partition.go +++ b/pkg/ddl/partition.go @@ -3562,6 +3562,10 @@ func (w *worker) onReorganizePartition(jobCtx *jobContext, job *model.Job) (ver dropIndices = append(dropIndices, indexInfo) } } + // TODO: verify that the indexes are dropped, + // and that StateDeleteOnly+StateDeleteReorganization is not needed. + // 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) } diff --git a/pkg/ddl/rollingback.go b/pkg/ddl/rollingback.go index 5855e6b3d68f4..5270437a9b857 100644 --- a/pkg/ddl/rollingback.go +++ b/pkg/ddl/rollingback.go @@ -369,58 +369,25 @@ func convertReorgPartitionJob2RollbackJob(jobCtx *jobContext, job *model.Job, ot partNames = append(partNames, pd.Name.L) } var dropIndices []*model.IndexInfo - // When Global Index is duplicated to a non Global, we later need - // to know if it was Global before (marked to be dropped) or not. - globalToUniqueDupMap := make(map[string]int64) for _, indexInfo := range tblInfo.Indices { if !indexInfo.Unique { continue } - switch indexInfo.State { - case model.StateWriteOnly: + isNew, ok := pi.DDLChangedIndex[indexInfo.ID] + if !ok { + // non-changed index + continue + } + if !isNew { if pi.DDLState == model.StateDeleteReorganization { // Revert the non-public state indexInfo.State = model.StatePublic - globalToUniqueDupMap[indexInfo.Name.L] = indexInfo.ID - } else { - dropIndices = append(dropIndices, indexInfo) - } - case model.StateDeleteOnly, model.StateWriteReorganization: - dropIndices = append(dropIndices, indexInfo) - case model.StateDeleteReorganization: - if pi.DDLState != model.StateDeleteReorganization { - continue - } - // Old index marked to be dropped, rollback by making it public again - indexInfo.State = model.StatePublic - if indexInfo.Global { - if id, ok := globalToUniqueDupMap[indexInfo.Name.L]; ok { - return ver, errors.NewNoStackErrorf("Duplicate global index names '%s', %d != %d", indexInfo.Name.O, indexInfo.ID, id) - } - globalToUniqueDupMap[indexInfo.Name.L] = indexInfo.ID } - case model.StatePublic: - if pi.DDLState != model.StateDeleteReorganization { - continue - } - // We cannot drop the index here, we need to wait until - // the next schema version - // i.e. rollback in rollbackLikeDropPartition - // New index that became public in this state, - // mark it to be dropped in next schema version - if indexInfo.Global { + } else { + if pi.DDLState == model.StateDeleteReorganization { indexInfo.State = model.StateWriteOnly } else { - // How to know if this index was created as a duplicate or not? - if id, ok := globalToUniqueDupMap[indexInfo.Name.L]; ok { - // The original index - if id >= indexInfo.ID { - return ver, errors.NewNoStackErrorf("Indexes in wrong order during rollback, '%s', %d >= %d", indexInfo.Name.O, id, indexInfo.ID) - } - indexInfo.State = model.StateWriteOnly - } else { - globalToUniqueDupMap[indexInfo.Name.L] = indexInfo.ID - } + dropIndices = append(dropIndices, indexInfo) } } } diff --git a/pkg/table/tables/index.go b/pkg/table/tables/index.go index c3674fcdd7fbf..dda35219affc5 100644 --- a/pkg/table/tables/index.go +++ b/pkg/table/tables/index.go @@ -16,8 +16,6 @@ package tables import ( "context" - "github.com/pingcap/tidb/pkg/util/logutil" - "go.uber.org/zap" "sync" "time" @@ -282,7 +280,6 @@ func (c *index) create(sctx table.MutateContext, txn kv.Transaction, indexedValu continue } - logutil.BgLogger().Info("create key", zap.Binary("key", key), zap.String("handle as string", h.String()), zap.Int64("tableID", c.tblInfo.ID), zap.Int64("physID", c.phyTblID), zap.Int64("indexID", c.idxInfo.ID), zap.Bool("global", c.idxInfo.Global)) var value []byte if c.tblInfo.TempTableType != model.TempTableNone { // Always check key for temporary table because it does not write to TiKV diff --git a/pkg/table/tables/partition.go b/pkg/table/tables/partition.go index 8344950957210..e0175d9f3970c 100644 --- a/pkg/table/tables/partition.go +++ b/pkg/table/tables/partition.go @@ -176,6 +176,8 @@ func newPartitionedTable(tbl *TableCommon, tblInfo *model.TableInfo) (table.Part } ret.meta.Indices = currIndices if pi.DDLState == model.StateDeleteReorganization { + origIdx := setIndexesState(ret, pi.DDLState) + defer unsetIndexesState(ret, origIdx) // TODO: Explicitly explain the different DDL/New fields! if pi.NewTableID != 0 { ret.reorgPartitionExpr, err = newPartitionExpr(tblInfo, pi.DDLType, pi.DDLExpr, pi.DDLColumns, pi.DroppingDefinitions) @@ -202,8 +204,8 @@ func newPartitionedTable(tbl *TableCommon, tblInfo *model.TableInfo) (table.Part } } else { if len(pi.AddingDefinitions) > 0 { - //origIdx := setIndexesState(ret, pi.DDLState) - //defer unsetIndexesState(ret, origIdx) + origIdx := setIndexesState(ret, pi.DDLState) + defer unsetIndexesState(ret, origIdx) if pi.NewTableID != 0 { // REMOVE PARTITIONING or PARTITION BY ret.reorgPartitionExpr, err = newPartitionExpr(tblInfo, pi.DDLType, pi.DDLExpr, pi.DDLColumns, pi.AddingDefinitions) @@ -235,14 +237,13 @@ func newPartitionedTable(tbl *TableCommon, tblInfo *model.TableInfo) (table.Part } // setIndexesState is used to alter the index schema state for non-public partitions, -// during reorganize partition. +// during reorganize partition, so they don't trigger assertions during partition DDL state DeleteOnly. func setIndexesState(t *partitionedTable, state model.SchemaState) []*model.IndexInfo { orig := t.meta.Indices t.meta.Indices = make([]*model.IndexInfo, 0, len(orig)) for i := range orig { t.meta.Indices = append(t.meta.Indices, orig[i].Clone()) - switch t.meta.Indices[i].State { - case model.StatePublic: + if t.meta.Indices[i].State == model.StatePublic { switch state { case model.StateDeleteOnly, model.StateNone: t.meta.Indices[i].State = model.StateDeleteOnly From ee82920afeb248421873d5f307b1207a98e804e2 Mon Sep 17 00:00:00 2001 From: Mattias Jonsson Date: Thu, 24 Oct 2024 00:45:35 +0200 Subject: [PATCH 05/25] reverted non-needed changes. --- pkg/table/tables/partition.go | 19 ++++++++----------- 1 file changed, 8 insertions(+), 11 deletions(-) diff --git a/pkg/table/tables/partition.go b/pkg/table/tables/partition.go index e0175d9f3970c..bb6592e550bca 100644 --- a/pkg/table/tables/partition.go +++ b/pkg/table/tables/partition.go @@ -112,8 +112,7 @@ func newPartitionedTable(tbl *TableCommon, tblInfo *model.TableInfo) (table.Part if pi == nil || len(pi.Definitions) == 0 { return nil, table.ErrUnknownPartition } - tblCommon := tbl.Copy() - ret := &partitionedTable{TableCommon: tblCommon} + ret := &partitionedTable{TableCommon: tbl.Copy()} partitionExpr, err := newPartitionExpr(tblInfo, pi.Type, pi.Expr, pi.Columns, pi.Definitions) if err != nil { return nil, errors.Trace(err) @@ -191,16 +190,14 @@ func newPartitionedTable(tbl *TableCommon, tblInfo *model.TableInfo) (table.Part for _, def := range pi.AddingDefinitions { ret.reorganizePartitions[def.ID] = struct{}{} } - if len(pi.DroppingDefinitions) > 0 { - ret.doubleWritePartitions = make(map[int64]struct{}, len(pi.DroppingDefinitions)) - for _, def := range pi.DroppingDefinitions { - p, err := initPartition(ret, def) - if err != nil { - return nil, err - } - partitions[def.ID] = p - ret.doubleWritePartitions[def.ID] = struct{}{} + ret.doubleWritePartitions = make(map[int64]struct{}, len(pi.DroppingDefinitions)) + for _, def := range pi.DroppingDefinitions { + p, err := initPartition(ret, def) + if err != nil { + return nil, err } + partitions[def.ID] = p + ret.doubleWritePartitions[def.ID] = struct{}{} } } else { if len(pi.AddingDefinitions) > 0 { From c45e3e9f319b6e6e8b19cb0162d5049d77cb8c1f Mon Sep 17 00:00:00 2001 From: Mattias Jonsson Date: Thu, 24 Oct 2024 00:51:22 +0200 Subject: [PATCH 06/25] Reverted non-needed changes. --- pkg/table/tables/partition.go | 20 ++++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/pkg/table/tables/partition.go b/pkg/table/tables/partition.go index bb6592e550bca..c5f2ae3e8c139 100644 --- a/pkg/table/tables/partition.go +++ b/pkg/table/tables/partition.go @@ -99,9 +99,9 @@ type partitionedTable struct { // Only used during Reorganize partition // reorganizePartitions is the currently used partitions that are reorganized - reorganizePartitions map[int64]struct{} + reorganizePartitions map[int64]any // doubleWritePartitions are the partitions not visible, but we should double write to - doubleWritePartitions map[int64]struct{} + doubleWritePartitions map[int64]any reorgPartitionExpr *PartitionExpr } @@ -186,18 +186,18 @@ func newPartitionedTable(tbl *TableCommon, tblInfo *model.TableInfo) (table.Part if err != nil { return nil, errors.Trace(err) } - ret.reorganizePartitions = make(map[int64]struct{}, len(pi.AddingDefinitions)) + ret.reorganizePartitions = make(map[int64]any, len(pi.AddingDefinitions)) for _, def := range pi.AddingDefinitions { - ret.reorganizePartitions[def.ID] = struct{}{} + ret.reorganizePartitions[def.ID] = nil } - ret.doubleWritePartitions = make(map[int64]struct{}, len(pi.DroppingDefinitions)) + ret.doubleWritePartitions = make(map[int64]any, len(pi.DroppingDefinitions)) for _, def := range pi.DroppingDefinitions { p, err := initPartition(ret, def) if err != nil { return nil, err } partitions[def.ID] = p - ret.doubleWritePartitions[def.ID] = struct{}{} + ret.doubleWritePartitions[def.ID] = nil } } else { if len(pi.AddingDefinitions) > 0 { @@ -213,9 +213,9 @@ func newPartitionedTable(tbl *TableCommon, tblInfo *model.TableInfo) (table.Part if err != nil { return nil, errors.Trace(err) } - ret.doubleWritePartitions = make(map[int64]struct{}, len(pi.AddingDefinitions)) + ret.doubleWritePartitions = make(map[int64]any, len(pi.AddingDefinitions)) for _, def := range pi.AddingDefinitions { - ret.doubleWritePartitions[def.ID] = struct{}{} + ret.doubleWritePartitions[def.ID] = nil p, err := initPartition(ret, def) if err != nil { return nil, err @@ -224,9 +224,9 @@ func newPartitionedTable(tbl *TableCommon, tblInfo *model.TableInfo) (table.Part } } if len(pi.DroppingDefinitions) > 0 { - ret.reorganizePartitions = make(map[int64]struct{}, len(pi.DroppingDefinitions)) + ret.reorganizePartitions = make(map[int64]any, len(pi.DroppingDefinitions)) for _, def := range pi.DroppingDefinitions { - ret.reorganizePartitions[def.ID] = struct{}{} + ret.reorganizePartitions[def.ID] = nil } } } From 16d16286073ee6f8e02b0fe62f288d1c7f457cf7 Mon Sep 17 00:00:00 2001 From: Mattias Jonsson Date: Thu, 24 Oct 2024 12:14:41 +0200 Subject: [PATCH 07/25] Simplified index management during partitioned table initialization. --- pkg/table/tables/partition.go | 42 +++++++++-------------------------- 1 file changed, 11 insertions(+), 31 deletions(-) diff --git a/pkg/table/tables/partition.go b/pkg/table/tables/partition.go index c5f2ae3e8c139..e53e78f45a9c5 100644 --- a/pkg/table/tables/partition.go +++ b/pkg/table/tables/partition.go @@ -170,13 +170,22 @@ func newPartitionedTable(tbl *TableCommon, tblInfo *model.TableInfo) (table.Part // Filter out the opposite changed indexes from above, // so the reorganized partition (not visible) have the matching indices. if newIdx, ok := pi.DDLChangedIndex[idx.ID]; !ok || newIdx == useNew { + if !ok && idx.State == model.StatePublic { + idx = idx.Clone() + if pi.DDLState == model.StateDeleteOnly { + // New partition starting, treat also the index as DeleteOnly, + // This way it will also set correct assertions on the index. + idx.State = model.StateDeleteOnly + } else { + // The partition is not visible, so OK to have the index write only. + idx.State = model.StateWriteOnly + } + } currIndices = append(currIndices, idx) } } ret.meta.Indices = currIndices if pi.DDLState == model.StateDeleteReorganization { - origIdx := setIndexesState(ret, pi.DDLState) - defer unsetIndexesState(ret, origIdx) // TODO: Explicitly explain the different DDL/New fields! if pi.NewTableID != 0 { ret.reorgPartitionExpr, err = newPartitionExpr(tblInfo, pi.DDLType, pi.DDLExpr, pi.DDLColumns, pi.DroppingDefinitions) @@ -201,8 +210,6 @@ func newPartitionedTable(tbl *TableCommon, tblInfo *model.TableInfo) (table.Part } } else { if len(pi.AddingDefinitions) > 0 { - origIdx := setIndexesState(ret, pi.DDLState) - defer unsetIndexesState(ret, origIdx) if pi.NewTableID != 0 { // REMOVE PARTITIONING or PARTITION BY ret.reorgPartitionExpr, err = newPartitionExpr(tblInfo, pi.DDLType, pi.DDLExpr, pi.DDLColumns, pi.AddingDefinitions) @@ -233,33 +240,6 @@ func newPartitionedTable(tbl *TableCommon, tblInfo *model.TableInfo) (table.Part return ret, nil } -// setIndexesState is used to alter the index schema state for non-public partitions, -// during reorganize partition, so they don't trigger assertions during partition DDL state DeleteOnly. -func setIndexesState(t *partitionedTable, state model.SchemaState) []*model.IndexInfo { - orig := t.meta.Indices - t.meta.Indices = make([]*model.IndexInfo, 0, len(orig)) - for i := range orig { - t.meta.Indices = append(t.meta.Indices, orig[i].Clone()) - if t.meta.Indices[i].State == model.StatePublic { - switch state { - case model.StateDeleteOnly, model.StateNone: - t.meta.Indices[i].State = model.StateDeleteOnly - case model.StatePublic: - // Keep as is - default: - // use the 'StateWriteReorganization' here, since StateDeleteReorganization - // would skip index writes. - t.meta.Indices[i].State = model.StateWriteReorganization - } - } - } - return orig -} - -func unsetIndexesState(t *partitionedTable, orig []*model.IndexInfo) { - t.meta.Indices = orig -} - func initPartition(t *partitionedTable, def model.PartitionDefinition) (*partition, error) { var newPart partition err := initTableCommonWithIndices(&newPart.TableCommon, t.meta, def.ID, t.Columns, t.allocs, t.Constraints) From a45bba659e41c0db58bc891ab39ed10bf382a063 Mon Sep 17 00:00:00 2001 From: Mattias Jonsson Date: Thu, 24 Oct 2024 17:23:44 +0200 Subject: [PATCH 08/25] Improved test and fixed more issues. --- pkg/ddl/tests/partition/multi_domain_test.go | 96 ++++++++++---------- pkg/planner/core/find_best_task.go | 14 ++- pkg/table/tables/partition.go | 72 +++++++++------ pkg/testkit/testkit.go | 2 +- 4 files changed, 102 insertions(+), 82 deletions(-) diff --git a/pkg/ddl/tests/partition/multi_domain_test.go b/pkg/ddl/tests/partition/multi_domain_test.go index e04463efa9b3d..5fecdb5edb34b 100644 --- a/pkg/ddl/tests/partition/multi_domain_test.go +++ b/pkg/ddl/tests/partition/multi_domain_test.go @@ -15,6 +15,7 @@ package partition import ( + "fmt" "testing" "time" @@ -250,70 +251,72 @@ func TestMultiSchemaDropListColumnsDefaultPartition(t *testing.T) { } 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))` + 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)`) + 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.idx_b_global'") + tkNO.MustContainErrMsg(`insert into t values (999,999)`+dbgStr, "[kv:1062]Duplicate entry '999' for key 't.idx_b_global'") + 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))) + + 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))) + 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_global'") - tkNO.MustContainErrMsg(`insert into t values (1,2)`, "[kv:1062]Duplicate entry '2' for key 't.idx_b_global'") - tkO.MustContainErrMsg(`insert into t values (101,101)`, "[kv:1062]Duplicate entry '101' for key 't.idx_b_global'") - tkNO.MustContainErrMsg(`insert into t values (101,101)`, "[kv:1062]Duplicate entry '101' for key 't.idx_b_global'") - 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")) + 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. - /* - tkO.MustContainErrMsg(`insert into t values (1,1)`, "[kv:1062]Duplicate entry '1' for key 't.idx_b_global'") - tkNO.MustContainErrMsg(`insert into t values (1,1)`, "[kv:1062]Duplicate entry '1' for key 't.idx_b_global'") - tkO.MustContainErrMsg(`insert into t values (101,101)`, "[kv:1062]Duplicate entry '101' for key 't.idx_b_global'") - tkNO.MustContainErrMsg(`insert into t values (101,101)`, "[kv:1062]Duplicate entry '101' for key 't.idx_b_global'") - 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("" + @@ -324,7 +327,8 @@ func TestMultiSchemaReorganizePartition(t *testing.T) { " 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))")) + "(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" + @@ -334,20 +338,18 @@ func TestMultiSchemaReorganizePartition(t *testing.T) { ") 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))")) + " PARTITION `p1` VALUES LESS THAN (200),\n" + + " PARTITION `pMax` VALUES LESS THAN (MAXVALUE))")) case model.StateNone.String(): - tkO.MustExec(`insert into t values (11,11)`) - tkNO.MustExec(`insert into t values (12,12)`) - tkNO.MustQuery(`select * from t where b = 11`).Check(testkit.Rows("11 11")) - tkO.MustQuery(`select * from t where b = 12`).Check(testkit.Rows("12 12")) 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.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", "12 12", "2 2", "3 3", "4 4", "5 5", "6 6", "7 7", "8 8", "9 9")) + 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", "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", "998 998", "999 999")) } runMultiSchemaTest(t, createSQL, alterSQL, initFn, postFn, loopFn) } diff --git a/pkg/planner/core/find_best_task.go b/pkg/planner/core/find_best_task.go index 6deb698fdc6e6..13671cf464b2a 100644 --- a/pkg/planner/core/find_best_task.go +++ b/pkg/planner/core/find_best_task.go @@ -2283,11 +2283,15 @@ func (is *PhysicalIndexScan) addSelectionConditionForGlobalIndex(p *logicalop.Da return conditions, nil } needNot = true - for _, p := range pInfo.AddingDefinitions { - args = append(args, expression.NewInt64Const(p.ID)) - } - for _, p := range pInfo.DroppingDefinitions { - args = append(args, expression.NewInt64Const(p.ID)) + // TODO: Still waiting for https://github.com/pingcap/tidb/pull/56382 + if pInfo.DDLState != model.StateDeleteReorganization { + for _, p := range pInfo.AddingDefinitions { + args = append(args, expression.NewInt64Const(p.ID)) + } + } else { + for _, p := range pInfo.DroppingDefinitions { + args = append(args, expression.NewInt64Const(p.ID)) + } } } else if len(idxArr) == 0 { // add an invalid pid as param for `IN` function diff --git a/pkg/table/tables/partition.go b/pkg/table/tables/partition.go index e53e78f45a9c5..f191449cca804 100644 --- a/pkg/table/tables/partition.go +++ b/pkg/table/tables/partition.go @@ -128,22 +128,55 @@ func newPartitionedTable(tbl *TableCommon, tblInfo *model.TableInfo) (table.Part return nil, errors.Trace(err) } origIndices := ret.meta.Indices - currIndices := make([]*model.IndexInfo, 0, len(origIndices)) - useNew := false - if pi.DDLState == model.StateDeleteReorganization { - useNew = true - } + untouchedDefinitionIndices := make([]*model.IndexInfo, 0, len(origIndices)) + DroppingDefinitionIndices := make([]*model.IndexInfo, 0, len(origIndices)) + AddingDefinitionIndices := make([]*model.IndexInfo, 0, len(origIndices)) for _, idx := range origIndices { + untouchedDefinitionIndices = append(untouchedDefinitionIndices, idx) // Filter out so only the old OR new indexes are used. - if newIdx, ok := pi.DDLChangedIndex[idx.ID]; !ok || newIdx == useNew { - currIndices = append(currIndices, idx) + newIdx, ok := pi.DDLChangedIndex[idx.ID] + if !ok { + // Untouched index + DroppingDefinitionIndices = append(DroppingDefinitionIndices, idx) + idx = idx.Clone() + if pi.DDLState == model.StateDeleteOnly { + // New partition starting, treat also the index as DeleteOnly, + // This way it will also set correct assertions on the index. + idx.State = model.StateDeleteOnly + } else { + // OK to have partition level index state as WriteOnly, + // For reading, the table level index state will be used! + idx.State = model.StateWriteOnly + } + AddingDefinitionIndices = append(AddingDefinitionIndices, idx) + continue + } + if newIdx { + AddingDefinitionIndices = append(AddingDefinitionIndices, idx) + } else { + DroppingDefinitionIndices = append(DroppingDefinitionIndices, idx) } } - tblInfo.Indices = currIndices + tblInfo.Indices = untouchedDefinitionIndices defer func() { ret.meta.Indices = origIndices }() + dropMap := make(map[int64]struct{}) + for _, def := range pi.DroppingDefinitions { + dropMap[def.ID] = struct{}{} + } + addMap := make(map[int64]struct{}) + for _, def := range pi.AddingDefinitions { + addMap[def.ID] = struct{}{} + } partitions := make(map[int64]*partition, len(pi.Definitions)) for _, p := range pi.Definitions { var t partition + if _, drop := dropMap[p.ID]; drop { + tblInfo.Indices = DroppingDefinitionIndices + } else if _, add := addMap[p.ID]; add { + tblInfo.Indices = AddingDefinitionIndices + } else { + tblInfo.Indices = untouchedDefinitionIndices + } err := initTableCommonWithIndices(&t.TableCommon, tblInfo, p.ID, tbl.Columns, tbl.allocs, tbl.Constraints) if err != nil { return nil, errors.Trace(err) @@ -164,27 +197,6 @@ func newPartitionedTable(tbl *TableCommon, tblInfo *model.TableInfo) (table.Part // and if any new change happens in AddingDefinitions, it needs to be done // also in DroppingDefinitions (since session running on schema version -1) // should also see the changes. - useNew = !useNew - currIndices = currIndices[:0] - for _, idx := range origIndices { - // Filter out the opposite changed indexes from above, - // so the reorganized partition (not visible) have the matching indices. - if newIdx, ok := pi.DDLChangedIndex[idx.ID]; !ok || newIdx == useNew { - if !ok && idx.State == model.StatePublic { - idx = idx.Clone() - if pi.DDLState == model.StateDeleteOnly { - // New partition starting, treat also the index as DeleteOnly, - // This way it will also set correct assertions on the index. - idx.State = model.StateDeleteOnly - } else { - // The partition is not visible, so OK to have the index write only. - idx.State = model.StateWriteOnly - } - } - currIndices = append(currIndices, idx) - } - } - ret.meta.Indices = currIndices if pi.DDLState == model.StateDeleteReorganization { // TODO: Explicitly explain the different DDL/New fields! if pi.NewTableID != 0 { @@ -200,6 +212,7 @@ func newPartitionedTable(tbl *TableCommon, tblInfo *model.TableInfo) (table.Part ret.reorganizePartitions[def.ID] = nil } ret.doubleWritePartitions = make(map[int64]any, len(pi.DroppingDefinitions)) + tblInfo.Indices = DroppingDefinitionIndices for _, def := range pi.DroppingDefinitions { p, err := initPartition(ret, def) if err != nil { @@ -221,6 +234,7 @@ func newPartitionedTable(tbl *TableCommon, tblInfo *model.TableInfo) (table.Part return nil, errors.Trace(err) } ret.doubleWritePartitions = make(map[int64]any, len(pi.AddingDefinitions)) + tblInfo.Indices = AddingDefinitionIndices for _, def := range pi.AddingDefinitions { ret.doubleWritePartitions[def.ID] = nil p, err := initPartition(ret, def) diff --git a/pkg/testkit/testkit.go b/pkg/testkit/testkit.go index eac57bf400518..048511e9d2866 100644 --- a/pkg/testkit/testkit.go +++ b/pkg/testkit/testkit.go @@ -530,7 +530,7 @@ func (tk *TestKit) MustGetDBError(sql string, dberr *terror.Error) { func (tk *TestKit) MustContainErrMsg(sql string, errStr any) { err := tk.ExecToErr(sql) tk.require.Error(err, "sql: %s", sql) - tk.require.Contains(err.Error(), errStr) + tk.require.Contains(err.Error(), errStr, "sql: %s", sql) } // MustMatchErrMsg executes a sql statement and assert its error message matching errRx. From f68c29d454b7886b326a93648b1dacf1906782f5 Mon Sep 17 00:00:00 2001 From: Mattias Jonsson Date: Thu, 24 Oct 2024 17:35:17 +0200 Subject: [PATCH 09/25] minor simplification and fixed failpoint leading to panic --- pkg/ddl/tests/partition/reorg_partition_test.go | 2 +- pkg/table/tables/partition.go | 6 ++---- 2 files changed, 3 insertions(+), 5 deletions(-) diff --git a/pkg/ddl/tests/partition/reorg_partition_test.go b/pkg/ddl/tests/partition/reorg_partition_test.go index ea6ac72f75480..feb82eb77045b 100644 --- a/pkg/ddl/tests/partition/reorg_partition_test.go +++ b/pkg/ddl/tests/partition/reorg_partition_test.go @@ -817,7 +817,7 @@ func TestReorgPartitionRollback(t *testing.T) { // 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(true)`) + 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") diff --git a/pkg/table/tables/partition.go b/pkg/table/tables/partition.go index f191449cca804..b0f7e6628070b 100644 --- a/pkg/table/tables/partition.go +++ b/pkg/table/tables/partition.go @@ -128,11 +128,9 @@ func newPartitionedTable(tbl *TableCommon, tblInfo *model.TableInfo) (table.Part return nil, errors.Trace(err) } origIndices := ret.meta.Indices - untouchedDefinitionIndices := make([]*model.IndexInfo, 0, len(origIndices)) DroppingDefinitionIndices := make([]*model.IndexInfo, 0, len(origIndices)) AddingDefinitionIndices := make([]*model.IndexInfo, 0, len(origIndices)) for _, idx := range origIndices { - untouchedDefinitionIndices = append(untouchedDefinitionIndices, idx) // Filter out so only the old OR new indexes are used. newIdx, ok := pi.DDLChangedIndex[idx.ID] if !ok { @@ -157,7 +155,7 @@ func newPartitionedTable(tbl *TableCommon, tblInfo *model.TableInfo) (table.Part DroppingDefinitionIndices = append(DroppingDefinitionIndices, idx) } } - tblInfo.Indices = untouchedDefinitionIndices + tblInfo.Indices = origIndices defer func() { ret.meta.Indices = origIndices }() dropMap := make(map[int64]struct{}) for _, def := range pi.DroppingDefinitions { @@ -175,7 +173,7 @@ func newPartitionedTable(tbl *TableCommon, tblInfo *model.TableInfo) (table.Part } else if _, add := addMap[p.ID]; add { tblInfo.Indices = AddingDefinitionIndices } else { - tblInfo.Indices = untouchedDefinitionIndices + tblInfo.Indices = origIndices } err := initTableCommonWithIndices(&t.TableCommon, tblInfo, p.ID, tbl.Columns, tbl.allocs, tbl.Constraints) if err != nil { From ff3d9804b2be35b003a013a10cf497ca07026072 Mon Sep 17 00:00:00 2001 From: Mattias Jonsson Date: Thu, 24 Oct 2024 23:00:07 +0200 Subject: [PATCH 10/25] Added test for verifying cleanup. --- pkg/ddl/tests/partition/multi_domain_test.go | 134 ++++++++++++++++++- 1 file changed, 128 insertions(+), 6 deletions(-) diff --git a/pkg/ddl/tests/partition/multi_domain_test.go b/pkg/ddl/tests/partition/multi_domain_test.go index 5fecdb5edb34b..3125ed76faf59 100644 --- a/pkg/ddl/tests/partition/multi_domain_test.go +++ b/pkg/ddl/tests/partition/multi_domain_test.go @@ -15,7 +15,16 @@ package partition import ( + "context" + "encoding/hex" "fmt" + "github.com/pingcap/tidb/pkg/domain" + "github.com/pingcap/tidb/pkg/kv" + pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/sessiontxn" + "github.com/pingcap/tidb/pkg/store/gcworker" + "github.com/pingcap/tidb/pkg/tablecodec" + "math" "testing" "time" @@ -97,7 +106,7 @@ func TestMultiSchemaDropRangePartition(t *testing.T) { require.Failf(t, "unhandled schema state '%s'", schemaState) } } - runMultiSchemaTest(t, createSQL, alterSQL, initFn, func(kit *testkit.TestKit) {}, loopFn) + runMultiSchemaTest(t, createSQL, alterSQL, initFn, nil, loopFn) } func TestMultiSchemaDropListDefaultPartition(t *testing.T) { @@ -170,7 +179,7 @@ func TestMultiSchemaDropListDefaultPartition(t *testing.T) { require.Failf(t, "unhandled schema state '%s'", schemaState) } } - runMultiSchemaTest(t, createSQL, alterSQL, initFn, func(kit *testkit.TestKit) {}, loopFn) + runMultiSchemaTest(t, createSQL, alterSQL, initFn, nil, loopFn) } func TestMultiSchemaDropListColumnsDefaultPartition(t *testing.T) { @@ -247,13 +256,32 @@ func TestMultiSchemaDropListColumnsDefaultPartition(t *testing.T) { require.Failf(t, "unhandled schema state '%s'", schemaState) } } - runMultiSchemaTest(t, createSQL, alterSQL, initFn, func(kit *testkit.TestKit) {}, loopFn) + 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))` + originalPartitions := make([]int64, 0, 2) + originalIndexIDs := make([]int64, 0, 1) + originalGlobalIndexIDs := make([]int64, 0, 1) + tableID := int64(0) initFn := func(tkO *testkit.TestKit) { tkO.MustExec(`insert into t values (1,1),(2,2),(101,101),(102,102),(998,998),(999,999)`) + 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 + 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) + } } alterSQL := `alter table t reorganize partition p1 into (partition p0 values less than (100), partition p1 values less than (200))` @@ -291,6 +319,8 @@ func TestMultiSchemaReorganizePartition(t *testing.T) { 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)) + 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))) @@ -345,11 +375,68 @@ func TestMultiSchemaReorganizePartition(t *testing.T) { require.Failf(t, "unhandled schema state '%s'", schemaState) } } - postFn := func(tkO *testkit.TestKit) { + 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", "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", "998 998", "999 999")) + // TODO: Verify that there are no KV entries for old partitions or old indexes!!! + gcWorker, err := gcworker.NewMockGCWorker(store) + require.NoError(t, err) + require.Nil(t, gcWorker.DeleteRanges(context.TODO(), math.MaxInt64)) + 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) + tableID = tbl.Meta().ID + // Save this for the fix of https://github.com/pingcap/tidb/issues/56822 + //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 + 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) + } } runMultiSchemaTest(t, createSQL, alterSQL, initFn, postFn, loopFn) } @@ -461,7 +548,7 @@ func TestMultiSchemaReorganizePartition(t *testing.T) { // runMultiSchemaTest(t, createSQL, alterSQL, initFn, postFn, loopFn) //} -func runMultiSchemaTest(t *testing.T, createSQL, alterSQL string, initFn, postFn func(*testkit.TestKit), loopFn func(tO, tNO *testkit.TestKit)) { +func runMultiSchemaTest(t *testing.T, createSQL, alterSQL string, initFn func(*testkit.TestKit), postFn func(*testkit.TestKit, kv.Storage), loopFn func(tO, tNO *testkit.TestKit)) { distCtx := testkit.NewDistExecutionContextWithLease(t, 2, 15*time.Second) store := distCtx.Store domOwner := distCtx.GetDomain(0) @@ -544,9 +631,44 @@ func runMultiSchemaTest(t *testing.T, createSQL, alterSQL string, initFn, postFn hookChan <- struct{}{} } logutil.BgLogger().Info("XXXXXXXXXXX states loop done") - postFn(tkO) + 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 +} From 91fb5f92ac1d089008be7a7ea2aaa60ed180c3a3 Mon Sep 17 00:00:00 2001 From: Mattias Jonsson Date: Thu, 24 Oct 2024 23:02:48 +0200 Subject: [PATCH 11/25] Disabled test due to cleanup not fixed. --- pkg/ddl/tests/partition/multi_domain_test.go | 21 ++++++++++---------- 1 file changed, 11 insertions(+), 10 deletions(-) diff --git a/pkg/ddl/tests/partition/multi_domain_test.go b/pkg/ddl/tests/partition/multi_domain_test.go index 3125ed76faf59..599f7ad3f80f4 100644 --- a/pkg/ddl/tests/partition/multi_domain_test.go +++ b/pkg/ddl/tests/partition/multi_domain_test.go @@ -427,16 +427,17 @@ func TestMultiSchemaReorganizePartition(t *testing.T) { 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) - } + // TODO: Fix cleanup issues, most likely it needs one more SchemaState in onReorganizePartition + //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) + // } } runMultiSchemaTest(t, createSQL, alterSQL, initFn, postFn, loopFn) } From bf5defcfd19e3aafbff64fdbb0823391e8d53789 Mon Sep 17 00:00:00 2001 From: Mattias Jonsson Date: Thu, 24 Oct 2024 23:52:00 +0200 Subject: [PATCH 12/25] Simplified new partitions index states. --- pkg/ddl/tests/partition/multi_domain_test.go | 15 +++++---------- pkg/table/tables/partition.go | 18 +++++++----------- 2 files changed, 12 insertions(+), 21 deletions(-) diff --git a/pkg/ddl/tests/partition/multi_domain_test.go b/pkg/ddl/tests/partition/multi_domain_test.go index 599f7ad3f80f4..b0dcc99caa314 100644 --- a/pkg/ddl/tests/partition/multi_domain_test.go +++ b/pkg/ddl/tests/partition/multi_domain_test.go @@ -18,18 +18,16 @@ import ( "context" "encoding/hex" "fmt" - "github.com/pingcap/tidb/pkg/domain" - "github.com/pingcap/tidb/pkg/kv" - pmodel "github.com/pingcap/tidb/pkg/parser/model" - "github.com/pingcap/tidb/pkg/sessiontxn" - "github.com/pingcap/tidb/pkg/store/gcworker" - "github.com/pingcap/tidb/pkg/tablecodec" - "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/tablecodec" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/testkit/testfailpoint" "github.com/pingcap/tidb/pkg/util/logutil" @@ -381,9 +379,6 @@ func TestMultiSchemaReorganizePartition(t *testing.T) { 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", "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", "998 998", "999 999")) // TODO: Verify that there are no KV entries for old partitions or old indexes!!! - gcWorker, err := gcworker.NewMockGCWorker(store) - require.NoError(t, err) - require.Nil(t, gcWorker.DeleteRanges(context.TODO(), math.MaxInt64)) delRange := tkO.MustQuery(`select * from mysql.gc_delete_range_done`).Rows() s := "" for _, row := range delRange { diff --git a/pkg/table/tables/partition.go b/pkg/table/tables/partition.go index b0f7e6628070b..199300efc9287 100644 --- a/pkg/table/tables/partition.go +++ b/pkg/table/tables/partition.go @@ -131,20 +131,16 @@ func newPartitionedTable(tbl *TableCommon, tblInfo *model.TableInfo) (table.Part DroppingDefinitionIndices := make([]*model.IndexInfo, 0, len(origIndices)) AddingDefinitionIndices := make([]*model.IndexInfo, 0, len(origIndices)) for _, idx := range origIndices { - // Filter out so only the old OR new indexes are used. newIdx, ok := pi.DDLChangedIndex[idx.ID] if !ok { // Untouched index DroppingDefinitionIndices = append(DroppingDefinitionIndices, idx) - idx = idx.Clone() - if pi.DDLState == model.StateDeleteOnly { - // New partition starting, treat also the index as DeleteOnly, - // This way it will also set correct assertions on the index. - idx.State = model.StateDeleteOnly - } else { - // OK to have partition level index state as WriteOnly, - // For reading, the table level index state will be used! - idx.State = model.StateWriteOnly + if pi.DDLState != model.StateDeleteReorganization { + // If pi.DDLState == DeleteReorg, then keep the StatePublic. + // Otherwise, set same state as DDLState. Like DeleteOnly is needed to + // set the correct assertion on the index. + idx = idx.Clone() + idx.State = pi.DDLState } AddingDefinitionIndices = append(AddingDefinitionIndices, idx) continue @@ -194,7 +190,7 @@ func newPartitionedTable(tbl *TableCommon, tblInfo *model.TableInfo) (table.Part // In StateDeleteReorganization we are using the 'new' partition definitions // and if any new change happens in AddingDefinitions, it needs to be done // also in DroppingDefinitions (since session running on schema version -1) - // should also see the changes. + // should also see the changes if pi.DDLState == model.StateDeleteReorganization { // TODO: Explicitly explain the different DDL/New fields! if pi.NewTableID != 0 { From 6103fafef1e202d0e11511781575a57b5963e97a Mon Sep 17 00:00:00 2001 From: Mattias Jonsson Date: Fri, 25 Oct 2024 00:45:54 +0200 Subject: [PATCH 13/25] Skipping global index entries changes. --- pkg/planner/core/find_best_task.go | 16 ++++++++++++++-- 1 file changed, 14 insertions(+), 2 deletions(-) diff --git a/pkg/planner/core/find_best_task.go b/pkg/planner/core/find_best_task.go index 13671cf464b2a..91a330f9fb952 100644 --- a/pkg/planner/core/find_best_task.go +++ b/pkg/planner/core/find_best_task.go @@ -2284,11 +2284,23 @@ func (is *PhysicalIndexScan) addSelectionConditionForGlobalIndex(p *logicalop.Da } needNot = true // TODO: Still waiting for https://github.com/pingcap/tidb/pull/56382 - if pInfo.DDLState != model.StateDeleteReorganization { + removeAddingDefs := true + removeDroppingDefs := true + switch pInfo.DDLAction { + case model.ActionAlterTablePartitioning, + model.ActionReorganizePartition, model.ActionRemovePartitioning: + if pInfo.DDLState == model.StateDeleteReorganization { + removeAddingDefs = false + } else { + removeDroppingDefs = false + } + } + if removeAddingDefs { for _, p := range pInfo.AddingDefinitions { args = append(args, expression.NewInt64Const(p.ID)) } - } else { + } + if removeDroppingDefs { for _, p := range pInfo.DroppingDefinitions { args = append(args, expression.NewInt64Const(p.ID)) } From 526e781a2fd1d976987cf6cb5fe7d32b58fba118 Mon Sep 17 00:00:00 2001 From: Mattias Jonsson Date: Mon, 28 Oct 2024 20:17:50 +0100 Subject: [PATCH 14/25] Added extra state in REORGANIZE PARTITION --- pkg/ddl/partition.go | 46 +++++++++++++++++--- pkg/ddl/rollingback.go | 5 +++ pkg/ddl/tests/partition/multi_domain_test.go | 25 +++++++++-- pkg/meta/model/job.go | 6 +++ pkg/planner/core/find_best_task.go | 2 +- pkg/table/tables/partition.go | 32 ++++++++------ pkg/table/tables/tables.go | 30 ++++++++++--- 7 files changed, 117 insertions(+), 29 deletions(-) diff --git a/pkg/ddl/partition.go b/pkg/ddl/partition.go index 4c07eb1536c6c..349d75113c132 100644 --- a/pkg/ddl/partition.go +++ b/pkg/ddl/partition.go @@ -3531,6 +3531,38 @@ func (w *worker) onReorganizePartition(jobCtx *jobContext, job *model.Job) (ver ver, err = updateVersionAndTableInfo(jobCtx, job, tblInfo, true) case model.StateDeleteReorganization: + // Need to have one more state before completing, due to: + // - DeleteRanges could possibly start directly after DDL causing + // inserts during previous state (DeleteReorg) could insert after the cleanup + // leaving data in dropped partitions/indexes that will not be cleaned up again. + // - Updates in previous state (DeleteReorg) could have duplicate errors, if the row + // was deleted or updated in after finish (so here we need to have DeleteOnly index state! + // And we cannot rollback in this state! + + // Stop double writing to the indexes, only do Deletes! + // so that previous could do inserts, we do delete and allow second insert for + // previous state clients! + for _, index := range tblInfo.Indices { + isNew, ok := tblInfo.Partition.DDLChangedIndex[index.ID] + if !ok || isNew { + continue + } + // Old index, should not be visible any longer, + // but needs to be deleted, in case previous state clients inserts. + index.State = model.StateDeleteOnly + //index.State = model.StateDeleteReorganization + } + failpoint.Inject("reorgPartFail3", func(val failpoint.Value) { + if val.(bool) { + job.ErrorCount += variable.GetDDLErrorCountLimit() / 2 + failpoint.Return(ver, errors.New("Injected error by reorgPartFail3")) + } + }) + job.SchemaState = model.StatePublic + tblInfo.Partition.DDLState = job.SchemaState + ver, err = updateVersionAndTableInfo(jobCtx, job, tblInfo, true) + + case model.StatePublic: // Drop the droppingDefinitions and finish the DDL // This state is needed for the case where client A sees the schema // with version of StateWriteReorg and would not see updates of @@ -3554,7 +3586,7 @@ func (w *worker) onReorganizePartition(jobCtx *jobContext, job *model.Job) (ver var dropIndices []*model.IndexInfo for _, indexInfo := range tblInfo.Indices { - if indexInfo.Unique && indexInfo.State == model.StateWriteOnly { + if indexInfo.Unique && indexInfo.State == model.StateDeleteReorganization { // Drop the old unique (possible global) index, see onDropIndex indexInfo.State = model.StateNone DropIndexColumnFlag(tblInfo, indexInfo) @@ -3569,10 +3601,10 @@ func (w *worker) onReorganizePartition(jobCtx *jobContext, job *model.Job) (ver for _, indexInfo := range dropIndices { removeIndexInfo(tblInfo, indexInfo) } - failpoint.Inject("reorgPartFail3", func(val failpoint.Value) { + failpoint.Inject("reorgPartFail4", func(val failpoint.Value) { if val.(bool) { job.ErrorCount += variable.GetDDLErrorCountLimit() / 2 - failpoint.Return(ver, errors.New("Injected error by reorgPartFail3")) + failpoint.Return(ver, errors.New("Injected error by reorgPartFail4")) } }) var oldTblID int64 @@ -3606,10 +3638,10 @@ func (w *worker) onReorganizePartition(jobCtx *jobContext, job *model.Job) (ver // ALTER TABLE ... PARTITION BY tblInfo.Partition.ClearReorgIntermediateInfo() } - failpoint.Inject("reorgPartFail4", func(val failpoint.Value) { + failpoint.Inject("reorgPartFail5", func(val failpoint.Value) { if val.(bool) { job.ErrorCount += variable.GetDDLErrorCountLimit() / 2 - failpoint.Return(ver, errors.New("Injected error by reorgPartFail4")) + failpoint.Return(ver, errors.New("Injected error by reorgPartFail5")) } }) err = metaMut.GetAutoIDAccessors(job.SchemaID, tblInfo.ID).Put(autoIDs) @@ -3621,10 +3653,10 @@ func (w *worker) onReorganizePartition(jobCtx *jobContext, job *model.Job) (ver return ver, errors.Trace(err) } } - failpoint.Inject("reorgPartFail5", func(val failpoint.Value) { + failpoint.Inject("reorgPartFail6", func(val failpoint.Value) { if val.(bool) { job.ErrorCount += variable.GetDDLErrorCountLimit() / 2 - failpoint.Return(ver, errors.New("Injected error by reorgPartFail5")) + failpoint.Return(ver, errors.New("Injected error by reorgPartFail6")) } }) failpoint.Inject("updateVersionAndTableInfoErrInStateDeleteReorganization", func() { diff --git a/pkg/ddl/rollingback.go b/pkg/ddl/rollingback.go index 5270437a9b857..adea6d4ffefa1 100644 --- a/pkg/ddl/rollingback.go +++ b/pkg/ddl/rollingback.go @@ -341,6 +341,11 @@ func onRollbackReorganizePartition(jobCtx *jobContext, job *model.Job) (ver int6 job.State = model.JobStateCancelled return ver, errors.Trace(err) } + if job.SchemaState == model.StatePublic { + // We started to destroy the old indexes, so we can no longer rollback! + job.State = model.JobStateRunning + return ver, nil + } jobCtx.jobArgs = args return rollbackReorganizePartitionWithErr(jobCtx, job, dbterror.ErrCancelledDDLJob) diff --git a/pkg/ddl/tests/partition/multi_domain_test.go b/pkg/ddl/tests/partition/multi_domain_test.go index b0dcc99caa314..9693e933b50d7 100644 --- a/pkg/ddl/tests/partition/multi_domain_test.go +++ b/pkg/ddl/tests/partition/multi_domain_test.go @@ -317,12 +317,28 @@ func TestMultiSchemaReorganizePartition(t *testing.T) { 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)) + 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)) + //if testID == 14 { + // tkO.MustQuery(fmt.Sprintf(`explain select * from t where b = "%d"`+dbgStr, testID)).Check(testkit.Rows(fmt.Sprintf("%d %d", 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 @@ -369,6 +385,8 @@ func TestMultiSchemaReorganizePartition(t *testing.T) { " PARTITION `p1` VALUES LESS THAN (200),\n" + " PARTITION `pMax` VALUES LESS THAN (MAXVALUE))")) case model.StateNone.String(): + logutil.BgLogger().Info("Can this happen?") + case model.StatePublic.String(): default: require.Failf(t, "unhandled schema state '%s'", schemaState) } @@ -377,7 +395,7 @@ func TestMultiSchemaReorganizePartition(t *testing.T) { 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", "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", "998 998", "999 999")) + 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")) // TODO: Verify that there are no KV entries for old partitions or old indexes!!! delRange := tkO.MustQuery(`select * from mysql.gc_delete_range_done`).Rows() s := "" @@ -545,7 +563,8 @@ func TestMultiSchemaReorganizePartition(t *testing.T) { //} func runMultiSchemaTest(t *testing.T, createSQL, alterSQL string, initFn func(*testkit.TestKit), postFn func(*testkit.TestKit, kv.Storage), loopFn func(tO, tNO *testkit.TestKit)) { - distCtx := testkit.NewDistExecutionContextWithLease(t, 2, 15*time.Second) + //distCtx := testkit.NewDistExecutionContextWithLease(t, 2, 15*time.Second) + distCtx := testkit.NewDistExecutionContextWithLease(t, 2, 1500*time.Second) store := distCtx.Store domOwner := distCtx.GetDomain(0) domNonOwner := distCtx.GetDomain(1) diff --git a/pkg/meta/model/job.go b/pkg/meta/model/job.go index d75a2931ba0eb..bd4a5f1c966bd 100644 --- a/pkg/meta/model/job.go +++ b/pkg/meta/model/job.go @@ -835,6 +835,12 @@ func (job *Job) IsRollbackable() bool { job.SchemaState == StateWriteOnly { return false } + case ActionReorganizePartition, ActionRemovePartitioning, ActionAlterTablePartitioning: + if job.SchemaState == StatePublic { + // We will double write until this state, here we will do DeleteOnly on indexes, + // so no-longer rollbackable. + return false + } } return true } diff --git a/pkg/planner/core/find_best_task.go b/pkg/planner/core/find_best_task.go index 91a330f9fb952..6b9296a07b6a8 100644 --- a/pkg/planner/core/find_best_task.go +++ b/pkg/planner/core/find_best_task.go @@ -2289,7 +2289,7 @@ func (is *PhysicalIndexScan) addSelectionConditionForGlobalIndex(p *logicalop.Da switch pInfo.DDLAction { case model.ActionAlterTablePartitioning, model.ActionReorganizePartition, model.ActionRemovePartitioning: - if pInfo.DDLState == model.StateDeleteReorganization { + if pInfo.DDLState == model.StateDeleteReorganization || pInfo.DDLState == model.StatePublic { removeAddingDefs = false } else { removeDroppingDefs = false diff --git a/pkg/table/tables/partition.go b/pkg/table/tables/partition.go index 199300efc9287..4f1aed5f59246 100644 --- a/pkg/table/tables/partition.go +++ b/pkg/table/tables/partition.go @@ -191,7 +191,7 @@ func newPartitionedTable(tbl *TableCommon, tblInfo *model.TableInfo) (table.Part // and if any new change happens in AddingDefinitions, it needs to be done // also in DroppingDefinitions (since session running on schema version -1) // should also see the changes - if pi.DDLState == model.StateDeleteReorganization { + if pi.DDLState == model.StateDeleteReorganization || pi.DDLState == model.StatePublic { // TODO: Explicitly explain the different DDL/New fields! if pi.NewTableID != 0 { ret.reorgPartitionExpr, err = newPartitionExpr(tblInfo, pi.DDLType, pi.DDLExpr, pi.DDLColumns, pi.DroppingDefinitions) @@ -1389,18 +1389,23 @@ func (t *partitionedTable) locateReorgPartition(ctx expression.EvalContext, r [] // all partitions will be reorganized, // so we can use the number in Dropping or AddingDefinitions, // depending on current state. - num := len(pi.AddingDefinitions) - if pi.DDLState == model.StateDeleteReorganization { - num = len(pi.DroppingDefinitions) + reorgDefs := pi.AddingDefinitions + switch pi.DDLAction { + case model.ActionReorganizePartition, model.ActionRemovePartitioning, model.ActionAlterTablePartitioning: + if pi.DDLState == model.StatePublic { + reorgDefs = pi.DroppingDefinitions + } + fallthrough + default: + if pi.DDLState == model.StateDeleteReorganization { + reorgDefs = pi.DroppingDefinitions + } } - idx, err := t.locatePartitionCommon(ctx, pi.DDLType, t.reorgPartitionExpr, uint64(num), columnsSet, r) + idx, err := t.locatePartitionCommon(ctx, pi.DDLType, t.reorgPartitionExpr, uint64(len(reorgDefs)), columnsSet, r) if err != nil { return 0, errors.Trace(err) } - if pi.DDLState == model.StateDeleteReorganization { - return pi.DroppingDefinitions[idx].ID, nil - } - return pi.AddingDefinitions[idx].ID, nil + return reorgDefs[idx].ID, nil } func (t *partitionedTable) locateRangeColumnPartition(ctx expression.EvalContext, partitionExpr *PartitionExpr, r []types.Datum) (int, error) { @@ -1676,7 +1681,7 @@ func partitionedTableAddRecord(ctx table.MutateContext, txn kv.Transaction, t *p if err != nil { return } - if t.Meta().Partition.DDLState == model.StateDeleteOnly { + if t.Meta().Partition.DDLState == model.StateDeleteOnly || t.Meta().Partition.DDLState == model.StatePublic { return } if _, ok := t.reorganizePartitions[pid]; ok { @@ -1813,6 +1818,7 @@ func partitionedTableUpdateRecord(ctx table.MutateContext, txn kv.Transaction, t sh := memBuffer.Staging() defer memBuffer.Cleanup(sh) + deleteOnly := t.Meta().Partition.DDLState == model.StateDeleteOnly || t.Meta().Partition.DDLState == model.StatePublic // The old and new data locate in different partitions. // Remove record from old partition and add record to new partition. if from != to { @@ -1858,7 +1864,7 @@ func partitionedTableUpdateRecord(ctx table.MutateContext, txn kv.Transaction, t return errors.Trace(err) } } - if newTo != 0 && t.Meta().GetPartitionInfo().DDLState != model.StateDeleteOnly { + if newTo != 0 && !deleteOnly { _, err = t.getPartition(newTo).addRecord(ctx, txn, newData, opt.GetAddRecordOpt()) if err != nil { return errors.Trace(err) @@ -1885,7 +1891,7 @@ func partitionedTableUpdateRecord(ctx table.MutateContext, txn kv.Transaction, t } if newTo == newFrom { tbl = t.getPartition(newTo) - if t.Meta().Partition.DDLState == model.StateDeleteOnly { + if deleteOnly { err = tbl.RemoveRecord(ctx, txn, h, currData) } else { err = tbl.updateRecord(ctx, txn, h, currData, newData, touched, opt) @@ -1901,7 +1907,7 @@ func partitionedTableUpdateRecord(ctx table.MutateContext, txn kv.Transaction, t if err != nil { return errors.Trace(err) } - if t.Meta().GetPartitionInfo().DDLState != model.StateDeleteOnly { + if !deleteOnly { tbl = t.getPartition(newTo) _, err = tbl.addRecord(ctx, txn, newData, opt.GetAddRecordOpt()) if err != nil { diff --git a/pkg/table/tables/tables.go b/pkg/table/tables/tables.go index ab270362bc25a..950a0846ae749 100644 --- a/pkg/table/tables/tables.go +++ b/pkg/table/tables/tables.go @@ -417,14 +417,34 @@ func (t *TableCommon) shouldAssert(level variable.AssertionLevel) bool { switch level { case variable.AssertionLevelFast: // Fast option, just skip assertion for all partitions. - if p.DDLState != model.StateNone && p.DDLState != model.StatePublic { - return false + switch p.DDLAction { + case model.ActionAlterTablePartitioning, model.ActionReorganizePartition, model.ActionRemovePartitioning: + switch p.DDLState { + case model.StateDeleteOnly, model.StateWriteOnly, model.StateDeleteReorganization, model.StatePublic: + return false + } + case model.ActionNone: + // no ddl ongoing, normal assertions. + return true + default: + if p.DDLState != model.StateNone && p.DDLState != model.StatePublic { + return false + } } case variable.AssertionLevelStrict: // Strict, only disable assertion for intermediate partitions. - // If there were an easy way to get from a TableCommon back to the partitioned table... - for i := range p.AddingDefinitions { - if t.physicalTableID == p.AddingDefinitions[i].ID { + defs := p.AddingDefinitions + switch p.DDLAction { + case model.ActionAlterTablePartitioning, model.ActionReorganizePartition, model.ActionRemovePartitioning: + switch p.DDLState { + case model.StateDeleteOnly: + defs = p.AddingDefinitions + case model.StateDeleteReorganization, model.StatePublic: + defs = p.DroppingDefinitions + } + } + for i := range defs { + if t.physicalTableID == defs[i].ID { return false } } From 1d92581e179cddd1ff559cabb76cd32334a49536 Mon Sep 17 00:00:00 2001 From: Mattias Jonsson Date: Mon, 28 Oct 2024 20:38:09 +0100 Subject: [PATCH 15/25] Simplified shouldAssert to a skipAssert flag instead. --- pkg/table/tables/partition.go | 8 +++-- pkg/table/tables/tables.go | 61 ++++++----------------------------- 2 files changed, 14 insertions(+), 55 deletions(-) diff --git a/pkg/table/tables/partition.go b/pkg/table/tables/partition.go index 4f1aed5f59246..878d6d015d526 100644 --- a/pkg/table/tables/partition.go +++ b/pkg/table/tables/partition.go @@ -184,13 +184,13 @@ func newPartitionedTable(tbl *TableCommon, tblInfo *model.TableInfo) (table.Part pi.DDLAction != model.ActionAlterTablePartitioning { return ret, nil } - // In StateWriteReorganization we are using the 'old' partition definitions + // In WriteReorganization we are using the 'old' partition definitions // and if any new change happens in DroppingDefinitions, it needs to be done // also in AddingDefinitions (with new evaluation of the new expression) - // In StateDeleteReorganization we are using the 'new' partition definitions + // In DeleteReorganization/Public we are using the 'new' partition definitions // and if any new change happens in AddingDefinitions, it needs to be done // also in DroppingDefinitions (since session running on schema version -1) - // should also see the changes + // should also see the changes. if pi.DDLState == model.StateDeleteReorganization || pi.DDLState == model.StatePublic { // TODO: Explicitly explain the different DDL/New fields! if pi.NewTableID != 0 { @@ -212,6 +212,7 @@ func newPartitionedTable(tbl *TableCommon, tblInfo *model.TableInfo) (table.Part if err != nil { return nil, err } + p.skipAssert = true partitions[def.ID] = p ret.doubleWritePartitions[def.ID] = nil } @@ -235,6 +236,7 @@ func newPartitionedTable(tbl *TableCommon, tblInfo *model.TableInfo) (table.Part if err != nil { return nil, err } + p.skipAssert = true partitions[def.ID] = p } } diff --git a/pkg/table/tables/tables.go b/pkg/table/tables/tables.go index 950a0846ae749..14ec727f6872f 100644 --- a/pkg/table/tables/tables.go +++ b/pkg/table/tables/tables.go @@ -36,7 +36,6 @@ import ( "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/sessionctx" - "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/statistics" "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/table/tblctx" @@ -83,6 +82,9 @@ type TableCommon struct { // recordPrefix and indexPrefix are generated using physicalTableID. recordPrefix kv.Key indexPrefix kv.Key + + // skipAssert is used for partitions that are in WriteOnly/DeleteOnly state. + skipAssert bool } // ResetColumnsCache implements testingKnob interface. @@ -408,51 +410,6 @@ func (t *TableCommon) RecordKey(h kv.Handle) kv.Key { return tablecodec.EncodeRecordKey(t.recordPrefix, h) } -// shouldAssert checks if the partition should be in consistent -// state and can have assertion. -func (t *TableCommon) shouldAssert(level variable.AssertionLevel) bool { - p := t.Meta().Partition - if p != nil { - // This disables asserting during Reorganize Partition. - switch level { - case variable.AssertionLevelFast: - // Fast option, just skip assertion for all partitions. - switch p.DDLAction { - case model.ActionAlterTablePartitioning, model.ActionReorganizePartition, model.ActionRemovePartitioning: - switch p.DDLState { - case model.StateDeleteOnly, model.StateWriteOnly, model.StateDeleteReorganization, model.StatePublic: - return false - } - case model.ActionNone: - // no ddl ongoing, normal assertions. - return true - default: - if p.DDLState != model.StateNone && p.DDLState != model.StatePublic { - return false - } - } - case variable.AssertionLevelStrict: - // Strict, only disable assertion for intermediate partitions. - defs := p.AddingDefinitions - switch p.DDLAction { - case model.ActionAlterTablePartitioning, model.ActionReorganizePartition, model.ActionRemovePartitioning: - switch p.DDLState { - case model.StateDeleteOnly: - defs = p.AddingDefinitions - case model.StateDeleteReorganization, model.StatePublic: - defs = p.DroppingDefinitions - } - } - for i := range defs { - if t.physicalTableID == defs[i].ID { - return false - } - } - } - } - return true -} - // UpdateRecord implements table.Table UpdateRecord interface. // `touched` means which columns are really modified, used for secondary indices. // Length of `oldData` and `newData` equals to length of `t.WritableCols()`. @@ -555,10 +512,10 @@ func (t *TableCommon) updateRecord(sctx table.MutateContext, txn kv.Transaction, } }) - if t.shouldAssert(sctx.TxnAssertionLevel()) { - err = txn.SetAssertion(key, kv.SetAssertExist) - } else { + if t.skipAssert { err = txn.SetAssertion(key, kv.SetAssertUnknown) + } else { + err = txn.SetAssertion(key, kv.SetAssertExist) } if err != nil { return err @@ -1256,10 +1213,10 @@ func (t *TableCommon) removeRowData(ctx table.MutateContext, txn kv.Transaction, } } }) - if t.shouldAssert(ctx.TxnAssertionLevel()) { - err = txn.SetAssertion(key, kv.SetAssertExist) - } else { + if t.skipAssert { err = txn.SetAssertion(key, kv.SetAssertUnknown) + } else { + err = txn.SetAssertion(key, kv.SetAssertExist) } if err != nil { return err From e2df898dc7c8e5713456665743df04539cc72324 Mon Sep 17 00:00:00 2001 From: Mattias Jonsson Date: Tue, 29 Oct 2024 21:20:17 +0100 Subject: [PATCH 16/25] Fixed schema version diffs --- pkg/ddl/partition.go | 56 ++++++--------- pkg/ddl/schema_version.go | 2 +- .../tests/partition/reorg_partition_test.go | 70 +++++++++++-------- 3 files changed, 65 insertions(+), 63 deletions(-) diff --git a/pkg/ddl/partition.go b/pkg/ddl/partition.go index 349d75113c132..59f97cccb0179 100644 --- a/pkg/ddl/partition.go +++ b/pkg/ddl/partition.go @@ -3045,36 +3045,29 @@ func getNewGlobal(partInfo *model.PartitionInfo, idx *model.IndexInfo) bool { return idx.Global } -func getReorgPartitionInfo(t *meta.Mutator, job *model.Job, args *model.TablePartitionArgs) (*model.TableInfo, []string, *model.PartitionInfo, []model.PartitionDefinition, []model.PartitionDefinition, error) { +func getReorgPartitionInfo(t *meta.Mutator, job *model.Job, args *model.TablePartitionArgs) (*model.TableInfo, []string, *model.PartitionInfo, error) { schemaID := job.SchemaID tblInfo, err := GetTableInfoAndCancelFaultJob(t, job, schemaID) if err != nil { - return nil, nil, nil, nil, nil, errors.Trace(err) + return nil, nil, nil, errors.Trace(err) } partNames, partInfo := args.PartNames, args.PartInfo - var addingDefs, droppingDefs []model.PartitionDefinition - if tblInfo.Partition != nil { - addingDefs = tblInfo.Partition.AddingDefinitions - droppingDefs = tblInfo.Partition.DroppingDefinitions - tblInfo.Partition.NewTableID = partInfo.NewTableID - tblInfo.Partition.DDLType = partInfo.Type - tblInfo.Partition.DDLExpr = partInfo.Expr - tblInfo.Partition.DDLColumns = partInfo.Columns - } else { - tblInfo.Partition = getPartitionInfoTypeNone() - tblInfo.Partition.NewTableID = partInfo.NewTableID - tblInfo.Partition.Definitions[0].ID = tblInfo.ID - tblInfo.Partition.DDLType = partInfo.Type - tblInfo.Partition.DDLExpr = partInfo.Expr - tblInfo.Partition.DDLColumns = partInfo.Columns - } - if len(addingDefs) == 0 { - addingDefs = []model.PartitionDefinition{} - } - if len(droppingDefs) == 0 { - droppingDefs = []model.PartitionDefinition{} + if job.SchemaState == model.StateNone { + if tblInfo.Partition != nil { + tblInfo.Partition.NewTableID = partInfo.NewTableID + tblInfo.Partition.DDLType = partInfo.Type + tblInfo.Partition.DDLExpr = partInfo.Expr + tblInfo.Partition.DDLColumns = partInfo.Columns + } else { + tblInfo.Partition = getPartitionInfoTypeNone() + tblInfo.Partition.NewTableID = partInfo.NewTableID + tblInfo.Partition.Definitions[0].ID = tblInfo.ID + tblInfo.Partition.DDLType = partInfo.Type + tblInfo.Partition.DDLExpr = partInfo.Expr + tblInfo.Partition.DDLColumns = partInfo.Columns + } } - return tblInfo, partNames, partInfo, droppingDefs, addingDefs, nil + return tblInfo, partNames, partInfo, nil } // onReorganizePartition reorganized the partitioning of a table including its indexes. @@ -3161,7 +3154,7 @@ func (w *worker) onReorganizePartition(jobCtx *jobContext, job *model.Job) (ver return ver, nil } - tblInfo, partNames, partInfo, _, addingDefinitions, err := getReorgPartitionInfo(jobCtx.metaMut, job, args) + tblInfo, partNames, partInfo, err := getReorgPartitionInfo(jobCtx.metaMut, job, args) if err != nil { return ver, err } @@ -3403,7 +3396,7 @@ func (w *worker) onReorganizePartition(jobCtx *jobContext, job *model.Job) (ver // For available state, the new added partition should wait its replica to // be finished, otherwise the query to this partition will be blocked. count := tblInfo.TiFlashReplica.Count - needRetry, err := checkPartitionReplica(count, addingDefinitions, jobCtx) + needRetry, err := checkPartitionReplica(count, tblInfo.Partition.AddingDefinitions, jobCtx) if err != nil { return rollbackReorganizePartitionWithErr(jobCtx, job, err) } @@ -3417,7 +3410,7 @@ func (w *worker) onReorganizePartition(jobCtx *jobContext, job *model.Job) (ver // When TiFlash Replica is ready, we must move them into `AvailablePartitionIDs`. // Since onUpdateFlashReplicaStatus cannot see the partitions yet (not public) - for _, d := range addingDefinitions { + for _, d := range tblInfo.Partition.AddingDefinitions { tblInfo.TiFlashReplica.AvailablePartitionIDs = append(tblInfo.TiFlashReplica.AvailablePartitionIDs, d.ID) } } @@ -3638,12 +3631,6 @@ func (w *worker) onReorganizePartition(jobCtx *jobContext, job *model.Job) (ver // ALTER TABLE ... PARTITION BY tblInfo.Partition.ClearReorgIntermediateInfo() } - failpoint.Inject("reorgPartFail5", func(val failpoint.Value) { - if val.(bool) { - job.ErrorCount += variable.GetDDLErrorCountLimit() / 2 - failpoint.Return(ver, errors.New("Injected error by reorgPartFail5")) - } - }) err = metaMut.GetAutoIDAccessors(job.SchemaID, tblInfo.ID).Put(autoIDs) if err != nil { return ver, errors.Trace(err) @@ -3653,7 +3640,7 @@ func (w *worker) onReorganizePartition(jobCtx *jobContext, job *model.Job) (ver return ver, errors.Trace(err) } } - failpoint.Inject("reorgPartFail6", func(val failpoint.Value) { + failpoint.Inject("reorgPartFail5", func(val failpoint.Value) { if val.(bool) { job.ErrorCount += variable.GetDDLErrorCountLimit() / 2 failpoint.Return(ver, errors.New("Injected error by reorgPartFail6")) @@ -3664,6 +3651,7 @@ func (w *worker) onReorganizePartition(jobCtx *jobContext, job *model.Job) (ver }) args.OldPhysicalTblIDs = physicalTableIDs args.NewPartitionIDs = newIDs + job.SchemaState = model.StateNone ver, err = updateVersionAndTableInfo(jobCtx, job, tblInfo, true) if err != nil { return ver, errors.Trace(err) diff --git a/pkg/ddl/schema_version.go b/pkg/ddl/schema_version.go index 12a90085d4d56..6c64634f633d0 100644 --- a/pkg/ddl/schema_version.go +++ b/pkg/ddl/schema_version.go @@ -222,7 +222,7 @@ func SetSchemaDiffForReorganizePartition(diff *model.SchemaDiff, job *model.Job, func SetSchemaDiffForPartitionModify(diff *model.SchemaDiff, job *model.Job, jobCtx *jobContext) { diff.TableID = job.TableID diff.OldTableID = job.TableID - if job.SchemaState == model.StateDeleteReorganization { + if job.SchemaState == model.StateNone { args := jobCtx.jobArgs.(*model.TablePartitionArgs) partInfo := args.PartInfo // Final part, new table id is assigned diff --git a/pkg/ddl/tests/partition/reorg_partition_test.go b/pkg/ddl/tests/partition/reorg_partition_test.go index feb82eb77045b..08e312b1b9acb 100644 --- a/pkg/ddl/tests/partition/reorg_partition_test.go +++ b/pkg/ddl/tests/partition/reorg_partition_test.go @@ -109,6 +109,7 @@ ROW: if found { tblmsg = fmt.Sprintf(" Table name: %s", tbl.Meta().Name.O) } + //tk.MustQuery(fmt.Sprintf(`select tidb_decode_key("%s") as 'Found table data after highest physical Table ID %d < %d (%s) %s'`, it.Key(), tblID, foundTblID, it.Key(), msg+tblmsg)).Check(testkit.Rows()) require.False(t, true, "Found table data after highest physical Table ID %d < %d (%s) "+msg+tblmsg, tblID, foundTblID, it.Key()) } break @@ -187,7 +188,7 @@ func TestReorgPartitionFailures(t *testing.T) { 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, "Fail4") + testReorganizePartitionFailures(t, create, alter, beforeDML, beforeResult, afterDML, afterResult) } func TestRemovePartitionFailures(t *testing.T) { @@ -212,7 +213,7 @@ func TestRemovePartitionFailures(t *testing.T) { `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, "Fail4") + testReorganizePartitionFailures(t, create, alter, beforeDML, beforeResult, afterDML, afterResult) } func TestPartitionByFailures(t *testing.T) { @@ -263,7 +264,7 @@ func TestReorganizePartitionListFailures(t *testing.T) { `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, "Fail4") + testReorganizePartitionFailures(t, create, alter, beforeDML, beforeResult, afterDML, afterResult) } func TestPartitionByListFailures(t *testing.T) { @@ -311,7 +312,7 @@ func TestAddHashPartitionFailures(t *testing.T) { `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, "Fail4") + testReorganizePartitionFailures(t, create, alter, beforeDML, beforeResult, afterDML, afterResult) } func TestCoalesceKeyPartitionFailures(t *testing.T) { @@ -334,7 +335,7 @@ func TestCoalesceKeyPartitionFailures(t *testing.T) { `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, "Fail4") + testReorganizePartitionFailures(t, create, alter, beforeDML, beforeResult, afterDML, afterResult) } func TestPartitionByNonPartitionedTable(t *testing.T) { @@ -342,7 +343,7 @@ func TestPartitionByNonPartitionedTable(t *testing.T) { 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, "Fail4") + 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) { @@ -358,20 +359,24 @@ func testReorganizePartitionFailures(t *testing.T, createSQL, alterSQL string, b // 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 + name string + count int + rollForwardFrom int }{ { "Cancel", 1, + -1, }, { "Fail", 5, + 4, }, { "Rollback", 4, + -1, }, } oldWaitTimeWhenErrorOccurred := ddl.WaitTimeWhenErrorOccurred @@ -388,44 +393,53 @@ func testReorganizePartitionFailures(t *testing.T, createSQL, alterSQL string, b continue SUBTEST } } - tk.MustExec(createSQL) + suffixComment := ` /* ` + suffix + ` */` + tk.MustExec(createSQL + suffixComment) for _, sql := range beforeDML { - tk.MustExec(sql + ` /* ` + suffix + ` */`) + tk.MustExec(sql + suffixComment) } - tk.MustQuery(`select * from t /* ` + suffix + ` */`).Sort().Check(beforeResult) + 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`).Rows() + oldCreate := tk.MustQuery(`show create table t` + suffixComment).Rows() name := "github.com/pingcap/tidb/pkg/ddl/reorgPart" + suffix - testfailpoint.Enable(t, name, `return(true)`) - err := tk.ExecToErr(alterSQL) - require.Error(t, err, "failpoint reorgPart"+suffix) - require.ErrorContains(t, err, "Injected error by reorgPart"+suffix) - testfailpoint.Disable(t, name) - tk.MustQuery(`show create table t /* ` + suffix + ` */`).Check(oldCreate) + 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 - if partition == nil { - require.Nil(t, tOrg.Meta().Partition, suffix) + if test.rollForwardFrom > 0 && test.rollForwardFrom <= i { + require.NoError(t, err) } 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) + require.Error(t, err, "failpoint reorgPart"+suffix) + 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 idxID != 0 { require.Equal(t, idxID, tt.Meta().Indices[0].ID, suffix) } - noNewTablesAfter(t, tk, tk.Session(), tOrg, suffix) - tk.MustExec(`admin check table t /* ` + suffix + ` */`) + tk.MustExec(`admin check table t` + suffixComment) for _, sql := range afterDML { - tk.MustExec(sql + " /* " + suffix + " */") + tk.MustExec(sql + suffixComment) } - tk.MustQuery(`select * from t /* ` + suffix + ` */`).Sort().Check(afterResult) - tk.MustExec(`drop table t /* ` + suffix + ` */`) + 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 From 8224d5d04bd2bcfd791b590d37b8f2630814c292 Mon Sep 17 00:00:00 2001 From: Mattias Jonsson Date: Tue, 29 Oct 2024 23:48:56 +0100 Subject: [PATCH 17/25] Fixed test failures and updated tests. --- pkg/ddl/partition.go | 7 +- .../tests/partition/reorg_partition_test.go | 82 +++++++++++++++---- 2 files changed, 66 insertions(+), 23 deletions(-) diff --git a/pkg/ddl/partition.go b/pkg/ddl/partition.go index 59f97cccb0179..c9e2dd47ea629 100644 --- a/pkg/ddl/partition.go +++ b/pkg/ddl/partition.go @@ -3579,7 +3579,7 @@ func (w *worker) onReorganizePartition(jobCtx *jobContext, job *model.Job) (ver var dropIndices []*model.IndexInfo for _, indexInfo := range tblInfo.Indices { - if indexInfo.Unique && indexInfo.State == model.StateDeleteReorganization { + if indexInfo.Unique && indexInfo.State == model.StateDeleteOnly { // Drop the old unique (possible global) index, see onDropIndex indexInfo.State = model.StateNone DropIndexColumnFlag(tblInfo, indexInfo) @@ -3643,12 +3643,9 @@ func (w *worker) onReorganizePartition(jobCtx *jobContext, job *model.Job) (ver failpoint.Inject("reorgPartFail5", func(val failpoint.Value) { if val.(bool) { job.ErrorCount += variable.GetDDLErrorCountLimit() / 2 - failpoint.Return(ver, errors.New("Injected error by reorgPartFail6")) + failpoint.Return(ver, errors.New("Injected error by reorgPartFail5")) } }) - failpoint.Inject("updateVersionAndTableInfoErrInStateDeleteReorganization", func() { - failpoint.Return(ver, errors.New("Injected error in StateDeleteReorganization")) - }) args.OldPhysicalTblIDs = physicalTableIDs args.NewPartitionIDs = newIDs job.SchemaState = model.StateNone diff --git a/pkg/ddl/tests/partition/reorg_partition_test.go b/pkg/ddl/tests/partition/reorg_partition_test.go index 08e312b1b9acb..ccba36cd7fe66 100644 --- a/pkg/ddl/tests/partition/reorg_partition_test.go +++ b/pkg/ddl/tests/partition/reorg_partition_test.go @@ -414,9 +414,11 @@ func testReorganizePartitionFailures(t *testing.T, createSQL, alterSQL string, b 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) require.ErrorContains(t, err, "Injected error by reorgPart"+suffix) tk.MustQuery(`show create table t` + suffixComment).Check(oldCreate) @@ -431,7 +433,7 @@ func testReorganizePartitionFailures(t *testing.T, createSQL, alterSQL string, b } testfailpoint.Disable(t, name) require.Equal(t, len(tOrg.Meta().Indices), len(tt.Meta().Indices), suffix) - if idxID != 0 { + if rollback && idxID != 0 { require.Equal(t, idxID, tt.Meta().Indices[0].ID, suffix) } tk.MustExec(`admin check table t` + suffixComment) @@ -477,7 +479,8 @@ func TestReorgPartitionConcurrent(t *testing.T) { (job.SchemaState == model.StateDeleteOnly || job.SchemaState == model.StateWriteOnly || job.SchemaState == model.StateWriteReorganization || - job.SchemaState == model.StateDeleteReorganization) && + job.SchemaState == model.StateDeleteReorganization || + job.SchemaState == model.StatePublic) && currState != job.SchemaState { currState = job.SchemaState <-wait @@ -573,6 +576,60 @@ func TestReorgPartitionConcurrent(t *testing.T) { "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))")) wait <- true syncOnChanged <- true // This reads the new schema (Schema update completed) @@ -581,11 +638,12 @@ func TestReorgPartitionConcurrent(t *testing.T) { "12 12b 12", "14 14 14", "15 15 15", - "16 16 16")) + "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()-deleteReorgInfoSchema.SchemaMetaVersion()) - oldTbl, err = deleteReorgInfoSchema.TableByName(context.Background(), pmodel.NewCIStr(schemaName), pmodel.NewCIStr("t")) + 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) @@ -603,7 +661,7 @@ func TestReorgPartitionConcurrent(t *testing.T) { " PARTITION `p1a` VALUES LESS THAN (15),\n" + " PARTITION `p1b` VALUES LESS THAN (20),\n" + " PARTITION `pMax` VALUES LESS THAN (MAXVALUE))")) - newTbl, err := deleteReorgInfoSchema.TableByName(context.Background(), pmodel.NewCIStr(schemaName), pmodel.NewCIStr("t")) + 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 @@ -949,15 +1007,3 @@ func TestPartitionByColumnChecks(t *testing.T) { 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", `return(1)`) - - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("set global tidb_ddl_error_count_limit = 3") - tk.MustExec("use test") - tk.MustExec("drop table if exists t") - tk.MustExec("create table t (a int)") - tk.MustContainErrMsg("alter table t partition by range(a) (partition p1 values less than (20))", "[ddl:-1]DDL job rollback, error msg: Injected error in StateDeleteReorganization") // should NOT panic -} From 677ac4aa9a3444cc6c557ee3a9248bc075bb9953 Mon Sep 17 00:00:00 2001 From: Mattias Jonsson Date: Wed, 30 Oct 2024 01:12:41 +0100 Subject: [PATCH 18/25] Linting --- pkg/ddl/tests/partition/reorg_partition_test.go | 1 + 1 file changed, 1 insertion(+) diff --git a/pkg/ddl/tests/partition/reorg_partition_test.go b/pkg/ddl/tests/partition/reorg_partition_test.go index fb1f17cff012d..d850cdbfa527b 100644 --- a/pkg/ddl/tests/partition/reorg_partition_test.go +++ b/pkg/ddl/tests/partition/reorg_partition_test.go @@ -629,6 +629,7 @@ func TestReorgPartitionConcurrent(t *testing.T) { " 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) From f21b86ee7ea65609d4e088e7f12cb9f3f76ce28e Mon Sep 17 00:00:00 2001 From: Mattias Jonsson Date: Fri, 1 Nov 2024 16:37:57 +0100 Subject: [PATCH 19/25] Removed limitation of global index cannot have all partitioning columns --- pkg/ddl/partition.go | 64 +++++++------------ .../integrationtest/r/globalindex/misc.result | 4 +- tests/integrationtest/t/globalindex/misc.test | 4 +- 3 files changed, 28 insertions(+), 44 deletions(-) diff --git a/pkg/ddl/partition.go b/pkg/ddl/partition.go index b1d67d3000955..b2c4fa9a3a729 100644 --- a/pkg/ddl/partition.go +++ b/pkg/ddl/partition.go @@ -3043,9 +3043,6 @@ func (w *worker) onExchangeTablePartition(jobCtx *jobContext, job *model.Job) (v } func getNewGlobal(partInfo *model.PartitionInfo, idx *model.IndexInfo) bool { - if len(partInfo.DDLUpdateIndexes) == 0 { - return idx.Global - } for _, newIdx := range partInfo.DDLUpdateIndexes { if strings.EqualFold(idx.Name.L, newIdx.IndexName) { return newIdx.Global @@ -3265,46 +3262,33 @@ func (w *worker) onReorganizePartition(jobCtx *jobContext, job *model.Job) (ver if err != nil { return ver, errors.Trace(err) } + // Currently only support Explicit Global indexes. + if !inAllPartitionColumns && !newGlobal { + job.State = model.JobStateCancelled + return ver, dbterror.ErrGlobalIndexNotExplicitlySet.GenWithStackByArgs(index.Name.O) + } + if !index.Global && !newGlobal { + // still local index, no need to duplicate index. + continue + } if tblInfo.Partition.DDLChangedIndex == nil { tblInfo.Partition.DDLChangedIndex = make(map[int64]bool) } - if !inAllPartitionColumns { - // Currently only support Explicit Global indexes. - if !newGlobal { - job.State = model.JobStateCancelled - return ver, dbterror.ErrGlobalIndexNotExplicitlySet.GenWithStackByArgs(index.Name.O) - } - // Duplicate the unique indexes with new index ids. - // If previously was Global or will be Global: - // it must be recreated with new index ID - // TODO: Could we allow that session in StateWriteReorganization, when StateDeleteReorganization - // has started, may not find changes through the global index that sessions in StateDeleteReorganization made? - // If so, then we could avoid copying the full Global Index if it has not changed from LOCAL! - // It might be possible to use the new, not yet public partitions to access those rows?! - // Just that it would not work with explicit partition select SELECT FROM t PARTITION (p,...) - newIndex := index.Clone() - newIndex.State = model.StateDeleteOnly - newIndex.ID = AllocateIndexID(tblInfo) - tblInfo.Partition.DDLChangedIndex[index.ID] = false - tblInfo.Partition.DDLChangedIndex[newIndex.ID] = true - newIndex.Global = true - tblInfo.Indices = append(tblInfo.Indices, newIndex) - } else { - if newGlobal { - // TODO: For the future loosen this restriction and allow global indexes for unique keys also including all partitioning columns - return ver, dbterror.ErrGeneralUnsupportedDDL.GenWithStackByArgs(fmt.Sprintf("PARTITION BY, index '%v' is unique and contains all partitioning columns, but has Global Index set", index.Name.O)) - } - if index.Global { - // Index was previously Global, now it needs to be duplicated and become a local index. - newIndex := index.Clone() - newIndex.State = model.StateDeleteOnly - newIndex.ID = AllocateIndexID(tblInfo) - tblInfo.Partition.DDLChangedIndex[index.ID] = false - tblInfo.Partition.DDLChangedIndex[newIndex.ID] = true - newIndex.Global = false - tblInfo.Indices = append(tblInfo.Indices, newIndex) - } - } + // Duplicate the unique indexes with new index ids. + // If previously was Global or will be Global: + // it must be recreated with new index ID + // TODO: Could we allow that session in StateWriteReorganization, when StateDeleteReorganization + // has started, may not find changes through the global index that sessions in StateDeleteReorganization made? + // If so, then we could avoid copying the full Global Index if it has not changed from LOCAL! + // It might be possible to use the new, not yet public partitions to access those rows?! + // Just that it would not work with explicit partition select SELECT FROM t PARTITION (p,...) + newIndex := index.Clone() + newIndex.State = model.StateDeleteOnly + newIndex.ID = AllocateIndexID(tblInfo) + tblInfo.Partition.DDLChangedIndex[index.ID] = false + tblInfo.Partition.DDLChangedIndex[newIndex.ID] = true + newIndex.Global = newGlobal + tblInfo.Indices = append(tblInfo.Indices, newIndex) } failpoint.Inject("reorgPartCancel1", func(val failpoint.Value) { if val.(bool) { diff --git a/tests/integrationtest/r/globalindex/misc.result b/tests/integrationtest/r/globalindex/misc.result index d9228942dcfc9..89694a3d76fa4 100644 --- a/tests/integrationtest/r/globalindex/misc.result +++ b/tests/integrationtest/r/globalindex/misc.result @@ -205,7 +205,8 @@ t CREATE TABLE `t` ( ) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin PARTITION BY HASH (`a`) PARTITIONS 3 alter table t partition by key (b) partitions 3; -Error 8200 (HY000): DDL job rollback, error msg: Unsupported PARTITION BY, index 'idx_b' is unique and contains all partitioning columns, but has Global Index set +alter table t partition by key (b) partitions 3 update indexes (idx_b GLOBAL); +alter table t partition by key (b) partitions 3 update indexes (idx_b LOCAL); alter table t partition by hash (a) partitions 3 update indexes (idx_b LOCAL); Error 8264 (HY000): Global Index is needed for index 'idx_b', since the unique index is not including all partitioning columns, and GLOBAL is not given as IndexOption alter table t partition by hash (a) partitions 3 update indexes (idx_b GLOBAL); @@ -218,7 +219,6 @@ t CREATE TABLE `t` ( ) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin PARTITION BY HASH (`a`) PARTITIONS 3 alter table t partition by hash (b) partitions 3 update indexes(idx_b global); -Error 8200 (HY000): DDL job rollback, error msg: Unsupported PARTITION BY, index 'idx_b' is unique and contains all partitioning columns, but has Global Index set alter table t partition by hash (b) partitions 3 update indexes(idx_b local); show create table t; Table Create Table diff --git a/tests/integrationtest/t/globalindex/misc.test b/tests/integrationtest/t/globalindex/misc.test index 6b3e52dcdd59c..cded23d78282a 100644 --- a/tests/integrationtest/t/globalindex/misc.test +++ b/tests/integrationtest/t/globalindex/misc.test @@ -139,13 +139,13 @@ CREATE TABLE `t` ( ) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin PARTITION BY HASH (`a`) PARTITIONS 3; show create table t; ---error 8200 alter table t partition by key (b) partitions 3; +alter table t partition by key (b) partitions 3 update indexes (idx_b GLOBAL); +alter table t partition by key (b) partitions 3 update indexes (idx_b LOCAL); --error 8264 alter table t partition by hash (a) partitions 3 update indexes (idx_b LOCAL); alter table t partition by hash (a) partitions 3 update indexes (idx_b GLOBAL); show create table t; ---error 8200 alter table t partition by hash (b) partitions 3 update indexes(idx_b global); alter table t partition by hash (b) partitions 3 update indexes(idx_b local); show create table t; From 03d049e9a9d5a4c3e73129ffeb7306ce98df7a04 Mon Sep 17 00:00:00 2001 From: Mattias Jonsson Date: Mon, 4 Nov 2024 11:43:18 +0100 Subject: [PATCH 20/25] review comments addressed, simplified if statement. --- pkg/ddl/partition.go | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/pkg/ddl/partition.go b/pkg/ddl/partition.go index b2c4fa9a3a729..f0f210e44d659 100644 --- a/pkg/ddl/partition.go +++ b/pkg/ddl/partition.go @@ -2185,9 +2185,7 @@ func (w *worker) rollbackLikeDropPartition(jobCtx *jobContext, job *model.Job) ( var dropIndices []*model.IndexInfo for _, indexInfo := range tblInfo.Indices { - if indexInfo.Unique && - indexInfo.State == model.StateWriteOnly && - tblInfo.Partition.DDLState == model.StateWriteReorganization { + if indexInfo.State == model.StateWriteOnly { dropIndices = append(dropIndices, indexInfo) } } From 05b13fb22178d24e2b9043a82d616195c4102039 Mon Sep 17 00:00:00 2001 From: Mattias Jonsson Date: Wed, 6 Nov 2024 00:53:48 +0100 Subject: [PATCH 21/25] Updated comments and removed dead code. --- pkg/ddl/partition.go | 19 +++++++++++++------ pkg/ddl/tests/partition/multi_domain_test.go | 3 --- .../tests/partition/reorg_partition_test.go | 1 - 3 files changed, 13 insertions(+), 10 deletions(-) diff --git a/pkg/ddl/partition.go b/pkg/ddl/partition.go index 77af77ac454c5..78b8d49152a25 100644 --- a/pkg/ddl/partition.go +++ b/pkg/ddl/partition.go @@ -3093,8 +3093,9 @@ func getReorgPartitionInfo(t *meta.Mutator, job *model.Job, args *model.TablePar // // job.SchemaState goes through the following SchemaState(s): // StateNone -> StateDeleteOnly -> StateWriteOnly -> StateWriteReorganization -// -> StateDeleteOrganization -> StatePublic +// -> StateDeleteOrganization -> StatePublic -> Done // There are more details embedded in the implementation, but the high level changes are: +// // StateNone -> StateDeleteOnly: // // Various checks and validations. @@ -3120,13 +3121,20 @@ func getReorgPartitionInfo(t *meta.Mutator, job *model.Job, args *model.TablePar // and if new unique indexes are added, it also updates them with the rest of data from // the non-touched partitions. // For indexes that are to be replaced with new ones (old/new global index), -// mark the old indexes as StateDeleteReorganization and new ones as StatePublic +// mark the old indexes as StateWriteOnly and new ones as StatePublic // Finally make the table visible with the new partition definitions. // I.e. in this state clients will read from the old set of partitions, -// and will read the new set of partitions in StateDeleteReorganization. +// and next state will read the new set of partitions in StateDeleteReorganization. // // StateDeleteOrganization -> StatePublic: // +// Now we mark all replaced (old) indexes as StateDeleteOnly +// in case DeleteRange would be called directly after the DDL, +// this way there will be no orphan records inserted after DeleteRanges +// has cleaned up the old partitions and old global indexes. +// +// StatePublic -> Done: +// // Now all heavy lifting is done, and we just need to finalize and drop things, while still doing // double writes, since previous state sees the old partitions/indexes. // Remove the old indexes and old partitions from the TableInfo. @@ -3135,10 +3143,10 @@ func getReorgPartitionInfo(t *meta.Mutator, job *model.Job, args *model.TablePar // if ALTER TABLE t PARTITION BY/REMOVE PARTITIONING: // Recreate the table with the new TableID, by DropTableOrView+CreateTableOrView // -// StatePublic: +// Done: // // Everything now looks as it should, no memory of old partitions/indexes, -// and no more double writing, since the previous state is only reading the new partitions/indexes. +// and no more double writing, since the previous state is only using the new partitions/indexes. // // Note: Special handling is also required in tables.newPartitionedTable(), // to get per partition indexes in the right state. @@ -3535,7 +3543,6 @@ func (w *worker) onReorganizePartition(jobCtx *jobContext, job *model.Job) (ver // Old index, should not be visible any longer, // but needs to be deleted, in case previous state clients inserts. index.State = model.StateDeleteOnly - //index.State = model.StateDeleteReorganization } failpoint.Inject("reorgPartFail3", func(val failpoint.Value) { if val.(bool) { diff --git a/pkg/ddl/tests/partition/multi_domain_test.go b/pkg/ddl/tests/partition/multi_domain_test.go index 8e3d8e386ab6c..ac1c102b22fb0 100644 --- a/pkg/ddl/tests/partition/multi_domain_test.go +++ b/pkg/ddl/tests/partition/multi_domain_test.go @@ -343,9 +343,6 @@ func TestMultiSchemaReorganizePartition(t *testing.T) { testID++ tkNO.MustExec(fmt.Sprintf(`insert into t values (%d,%d)`+dbgStr, testID, testID)) - //if testID == 14 { - // tkO.MustQuery(fmt.Sprintf(`explain select * from t where b = "%d"`+dbgStr, testID)).Check(testkit.Rows(fmt.Sprintf("%d %d", 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. diff --git a/pkg/ddl/tests/partition/reorg_partition_test.go b/pkg/ddl/tests/partition/reorg_partition_test.go index d850cdbfa527b..8fca584cfbba9 100644 --- a/pkg/ddl/tests/partition/reorg_partition_test.go +++ b/pkg/ddl/tests/partition/reorg_partition_test.go @@ -108,7 +108,6 @@ ROW: if found { tblmsg = fmt.Sprintf(" Table name: %s", tbl.Meta().Name.O) } - //tk.MustQuery(fmt.Sprintf(`select tidb_decode_key("%s") as 'Found table data after highest physical Table ID %d < %d (%s) %s'`, it.Key(), tblID, foundTblID, it.Key(), msg+tblmsg)).Check(testkit.Rows()) require.False(t, true, "Found table data after highest physical Table ID %d < %d (%s) "+msg+tblmsg, tblID, foundTblID, it.Key()) } break From 332f235402dea79ee492506983154ef222bd7fff Mon Sep 17 00:00:00 2001 From: Mattias Jonsson Date: Fri, 15 Nov 2024 15:45:49 +0100 Subject: [PATCH 22/25] Fixed test case --- pkg/ddl/partition.go | 1 + pkg/ddl/tests/partition/reorg_partition_test.go | 14 +++----------- 2 files changed, 4 insertions(+), 11 deletions(-) diff --git a/pkg/ddl/partition.go b/pkg/ddl/partition.go index 4af4e40772734..21bcd94a8d15b 100644 --- a/pkg/ddl/partition.go +++ b/pkg/ddl/partition.go @@ -3654,6 +3654,7 @@ func (w *worker) onReorganizePartition(jobCtx *jobContext, job *model.Job) (ver job.SchemaState = model.StateNone ver, err = updateVersionAndTableInfo(jobCtx, job, tblInfo, true) if err != nil { + job.ErrorCount += variable.GetDDLErrorCountLimit() / 2 return ver, errors.Trace(err) } // How to handle this? diff --git a/pkg/ddl/tests/partition/reorg_partition_test.go b/pkg/ddl/tests/partition/reorg_partition_test.go index 4463a1eaf7b5f..4a6a030d1784b 100644 --- a/pkg/ddl/tests/partition/reorg_partition_test.go +++ b/pkg/ddl/tests/partition/reorg_partition_test.go @@ -344,14 +344,6 @@ func TestPartitionByNonPartitionedTable(t *testing.T) { testReorganizePartitionFailures(t, create, alter, nil, beforeResult, nil, 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, "Fail4") -} - 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) @@ -1016,13 +1008,13 @@ func TestPartitionByColumnChecks(t *testing.T) { } func TestPartitionIssue56634(t *testing.T) { - testfailpoint.Enable(t, "github.com/pingcap/tidb/pkg/ddl/updateVersionAndTableInfoErrInStateDeleteReorganization", `return(1)`) + testfailpoint.Enable(t, "github.com/pingcap/tidb/pkg/ddl/updateVersionAndTableInfoErrInStateDeleteReorganization", `4*return(1)`) store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) - tk.MustExec("set global tidb_ddl_error_count_limit = 3") tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t (a int)") - tk.MustContainErrMsg("alter table t partition by range(a) (partition p1 values less than (20))", "[ddl:-1]DDL job rollback, error msg: Injected error in StateDeleteReorganization") // should NOT panic + // Changed, since StatePublic can no longer rollback! + tk.MustExec("alter table t partition by range(a) (partition p1 values less than (20))") } From 3c1d49e23eb8dff26d71da52f484917264a4e7a9 Mon Sep 17 00:00:00 2001 From: Mattias Jonsson Date: Tue, 19 Nov 2024 18:25:40 +0100 Subject: [PATCH 23/25] Updated test --- pkg/ddl/tests/partition/multi_domain_test.go | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/pkg/ddl/tests/partition/multi_domain_test.go b/pkg/ddl/tests/partition/multi_domain_test.go index c03b42a2d6f64..10e898690e045 100644 --- a/pkg/ddl/tests/partition/multi_domain_test.go +++ b/pkg/ddl/tests/partition/multi_domain_test.go @@ -403,11 +403,12 @@ func TestMultiSchemaReorganizePartition(t *testing.T) { "(PARTITION `p0` VALUES LESS THAN (100),\n" + " PARTITION `p1` VALUES LESS THAN (200),\n" + " PARTITION `pMax` VALUES LESS THAN (MAXVALUE))")) - case model.StateNone.String(): - logutil.BgLogger().Info("Can this happen?") 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 '%s'", schemaState) + require.Failf(t, "unhandled schema state", "State '%s'", schemaState) } } postFn := func(tkO *testkit.TestKit, store kv.Storage) { From f486ac591eb4ade2800bfda39d06a582a060cb41 Mon Sep 17 00:00:00 2001 From: Mattias Jonsson Date: Wed, 20 Nov 2024 01:58:13 +0100 Subject: [PATCH 24/25] Enabled test, which this PR fixes --- pkg/ddl/tests/partition/multi_domain_test.go | 21 ++++++++++---------- 1 file changed, 10 insertions(+), 11 deletions(-) diff --git a/pkg/ddl/tests/partition/multi_domain_test.go b/pkg/ddl/tests/partition/multi_domain_test.go index 10e898690e045..4f8d0f1191a85 100644 --- a/pkg/ddl/tests/partition/multi_domain_test.go +++ b/pkg/ddl/tests/partition/multi_domain_test.go @@ -460,17 +460,16 @@ func TestMultiSchemaReorganizePartition(t *testing.T) { require.False(t, HaveEntriesForTableIndex(t, tkO, part.ID, locIdx), "Local index id %d for partition id %d has still entries!", locIdx, tableID) } } - // TODO: Fix cleanup issues, most likely it needs one more SchemaState in onReorganizePartition - //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) - // } + 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) + } } runMultiSchemaTest(t, createSQL, alterSQL, initFn, postFn, loopFn) } From e4f718f2e1479e53c63cc09a9fcfe5e09b71e004 Mon Sep 17 00:00:00 2001 From: Mattias Jonsson Date: Wed, 20 Nov 2024 10:37:02 +0100 Subject: [PATCH 25/25] Removed line that should have been for failpoint --- pkg/ddl/partition.go | 1 - 1 file changed, 1 deletion(-) diff --git a/pkg/ddl/partition.go b/pkg/ddl/partition.go index 21bcd94a8d15b..4af4e40772734 100644 --- a/pkg/ddl/partition.go +++ b/pkg/ddl/partition.go @@ -3654,7 +3654,6 @@ func (w *worker) onReorganizePartition(jobCtx *jobContext, job *model.Job) (ver job.SchemaState = model.StateNone ver, err = updateVersionAndTableInfo(jobCtx, job, tblInfo, true) if err != nil { - job.ErrorCount += variable.GetDDLErrorCountLimit() / 2 return ver, errors.Trace(err) } // How to handle this?