From f150d376517e126b6ca2396f423e1709ed9e151c Mon Sep 17 00:00:00 2001 From: wjHuang Date: Sat, 17 Dec 2022 15:16:53 +0800 Subject: [PATCH] *: revert 38302, forbid modify column on partition table (#39991) ref pingcap/tidb#39915, ref pingcap/tidb#39922 --- ddl/backfilling.go | 3 - ddl/column.go | 38 ++----- ddl/db_partition_test.go | 207 ----------------------------------- ddl/ddl_api.go | 3 + ddl/failtest/fail_db_test.go | 4 +- 5 files changed, 13 insertions(+), 242 deletions(-) diff --git a/ddl/backfilling.go b/ddl/backfilling.go index 7c966591016d9..d1035bad084bd 100644 --- a/ddl/backfilling.go +++ b/ddl/backfilling.go @@ -546,9 +546,6 @@ func (dc *ddlCtx) handleRangeTasks(scheduler *backfillScheduler, t table.Table, reorgInfo := scheduler.reorgInfo physicalTableID := reorgInfo.PhysicalTableID var prefix kv.Key - if tbl, ok := t.(table.PartitionedTable); ok { - t = tbl.GetPartition(physicalTableID) - } if reorgInfo.mergingTmpIdx { prefix = t.IndexPrefix() } else { diff --git a/ddl/column.go b/ddl/column.go index d9425ceabac2c..3bbe7417fd7b5 100644 --- a/ddl/column.go +++ b/ddl/column.go @@ -1042,30 +1042,9 @@ func BuildElements(changingCol *model.ColumnInfo, changingIdxs []*model.IndexInf return elements } -func (w *worker) updatePhysicalTableRow(t table.Table, reorgInfo *reorgInfo) error { +func (w *worker) updatePhysicalTableRow(t table.PhysicalTable, reorgInfo *reorgInfo) error { logutil.BgLogger().Info("[ddl] start to update table row", zap.String("job", reorgInfo.Job.String()), zap.String("reorgInfo", reorgInfo.String())) - if tbl, ok := t.(table.PartitionedTable); ok { - done := false - for !done { - p := tbl.GetPartition(reorgInfo.PhysicalTableID) - if p == nil { - return dbterror.ErrCancelledDDLJob.GenWithStack("Can not find partition id %d for table %d", reorgInfo.PhysicalTableID, t.Meta().ID) - } - err := w.writePhysicalTableRecord(w.sessPool, p, typeUpdateColumnWorker, reorgInfo) - if err != nil { - return err - } - done, err = w.updateReorgInfo(tbl, reorgInfo) - if err != nil { - return errors.Trace(err) - } - } - return nil - } - if tbl, ok := t.(table.PhysicalTable); ok { - return w.writePhysicalTableRecord(w.sessPool, tbl, typeUpdateColumnWorker, reorgInfo) - } - return dbterror.ErrCancelledDDLJob.GenWithStack("internal error for phys tbl id: %d tbl id: %d", reorgInfo.PhysicalTableID, t.Meta().ID) + return w.writePhysicalTableRecord(w.sessPool, t, typeUpdateColumnWorker, reorgInfo) } // TestReorgGoroutineRunning is only used in test to indicate the reorg goroutine has been started. @@ -1087,25 +1066,22 @@ func (w *worker) updateCurrentElement(t table.Table, reorgInfo *reorgInfo) error } } }) + // TODO: Support partition tables. if bytes.Equal(reorgInfo.currElement.TypeKey, meta.ColumnElementKey) { - err := w.updatePhysicalTableRow(t, reorgInfo) + //nolint:forcetypeassert + err := w.updatePhysicalTableRow(t.(table.PhysicalTable), reorgInfo) if err != nil { return errors.Trace(err) } } - var physTbl table.PhysicalTable - if tbl, ok := t.(table.PartitionedTable); ok { - physTbl = tbl.GetPartition(reorgInfo.PhysicalTableID) - } else if tbl, ok := t.(table.PhysicalTable); ok { - physTbl = tbl - } // Get the original start handle and end handle. currentVer, err := getValidCurrentVersion(reorgInfo.d.store) if err != nil { return errors.Trace(err) } - originalStartHandle, originalEndHandle, err := getTableRange(reorgInfo.d.jobContext(reorgInfo.Job), reorgInfo.d, physTbl, currentVer.Ver, reorgInfo.Job.Priority) + //nolint:forcetypeassert + originalStartHandle, originalEndHandle, err := getTableRange(reorgInfo.d.jobContext(reorgInfo.Job), reorgInfo.d, t.(table.PhysicalTable), currentVer.Ver, reorgInfo.Job.Priority) if err != nil { return errors.Trace(err) } diff --git a/ddl/db_partition_test.go b/ddl/db_partition_test.go index 67e7d6a0484d7..38aa43e4e7755 100644 --- a/ddl/db_partition_test.go +++ b/ddl/db_partition_test.go @@ -4536,136 +4536,6 @@ func TestPartitionTableWithAnsiQuotes(t *testing.T) { ` PARTITION "pMax" VALUES LESS THAN (MAXVALUE,MAXVALUE))`)) } -func TestAlterModifyColumnOnPartitionedTable(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("create database AlterPartTable") - tk.MustExec("use AlterPartTable") - tk.MustExec(`create table t (a int unsigned PRIMARY KEY, b varchar(255), key (b))`) - tk.MustExec(`insert into t values (7, "07"), (8, "08"),(23,"23"),(34,"34πŸ’₯"),(46,"46"),(57,"57")`) - 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" + - " PRIMARY KEY (`a`) /*T![clustered_index] CLUSTERED */,\n" + - " KEY `b` (`b`)\n" + - ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin")) - // TODO: Why does it allow πŸ’₯ as a latin1 character? - tk.MustQuery(`select hex(b) from t where a = 34`).Check(testkit.Rows("3334F09F92A5")) - tk.MustExec(`alter table t modify b varchar(200) charset latin1`) - tk.MustQuery(`show create table t`).Check(testkit.Rows( - "t CREATE TABLE `t` (\n" + - " `a` int(10) unsigned NOT NULL,\n" + - " `b` varchar(200) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,\n" + - " PRIMARY KEY (`a`) /*T![clustered_index] CLUSTERED */,\n" + - " KEY `b` (`b`)\n" + - ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin")) - tk.MustQuery(`select hex(b) from t where a = 34`).Check(testkit.Rows("3334F09F92A5")) - tk.MustQuery(`select * from t`).Sort().Check(testkit.Rows(""+ - "23 23", - "34 34πŸ’₯", - "46 46", - "57 57", - "7 07", - "8 08")) - tk.MustQuery(`select * from t order by b`).Check(testkit.Rows(""+ - "7 07", - "8 08", - "23 23", - "34 34πŸ’₯", - "46 46", - "57 57")) - tk.MustExec(`alter table t change b c varchar(200) charset utf8mb4`) - tk.MustExec(`drop table t`) - tk.MustExec(`create table t (a int unsigned PRIMARY KEY, b varchar(255), key (b)) partition by range (a) ` + - `(partition p0 values less than (10),` + - ` partition p1 values less than (20),` + - ` partition p2 values less than (30),` + - ` partition pMax values less than (MAXVALUE))`) - tk.MustExec(`insert into t values (7, "07"), (8, "08"),(23,"23"),(34,"34πŸ’₯"),(46,"46"),(57,"57")`) - tk.MustQuery(`select * from t`).Sort().Check(testkit.Rows(""+ - "23 23", - "34 34πŸ’₯", - "46 46", - "57 57", - "7 07", - "8 08")) - tk.MustQuery(`select * from t order by b`).Check(testkit.Rows(""+ - "7 07", - "8 08", - "23 23", - "34 34πŸ’₯", - "46 46", - "57 57")) - 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" + - " PRIMARY KEY (`a`) /*T![clustered_index] CLUSTERED */,\n" + - " KEY `b` (`b`)\n" + - ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin\n" + - "PARTITION BY RANGE (`a`)\n" + - "(PARTITION `p0` VALUES LESS THAN (10),\n" + - " PARTITION `p1` VALUES LESS THAN (20),\n" + - " PARTITION `p2` VALUES LESS THAN (30),\n" + - " PARTITION `pMax` VALUES LESS THAN (MAXVALUE))")) - tk.MustExec(`alter table t modify b varchar(200) charset latin1`) - tk.MustQuery(`show create table t`).Check(testkit.Rows( - "t CREATE TABLE `t` (\n" + - " `a` int(10) unsigned NOT NULL,\n" + - " `b` varchar(200) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,\n" + - " PRIMARY KEY (`a`) /*T![clustered_index] CLUSTERED */,\n" + - " KEY `b` (`b`)\n" + - ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin\n" + - "PARTITION BY RANGE (`a`)\n" + - "(PARTITION `p0` VALUES LESS THAN (10),\n" + - " PARTITION `p1` VALUES LESS THAN (20),\n" + - " PARTITION `p2` VALUES LESS THAN (30),\n" + - " PARTITION `pMax` VALUES LESS THAN (MAXVALUE))")) - tk.MustQuery(`select * from t`).Sort().Check(testkit.Rows(""+ - "23 23", - "34 34πŸ’₯", - "46 46", - "57 57", - "7 07", - "8 08")) - tk.MustQuery(`select * from t order by b`).Check(testkit.Rows(""+ - "7 07", - "8 08", - "23 23", - "34 34πŸ’₯", - "46 46", - "57 57")) - tk.MustExec(`alter table t change b c varchar(150) charset utf8mb4`) - tk.MustQuery(`show create table t`).Check(testkit.Rows( - "t CREATE TABLE `t` (\n" + - " `a` int(10) unsigned NOT NULL,\n" + - " `c` varchar(150) DEFAULT NULL,\n" + - " PRIMARY KEY (`a`) /*T![clustered_index] CLUSTERED */,\n" + - " KEY `b` (`c`)\n" + - ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin\n" + - "PARTITION BY RANGE (`a`)\n" + - "(PARTITION `p0` VALUES LESS THAN (10),\n" + - " PARTITION `p1` VALUES LESS THAN (20),\n" + - " PARTITION `p2` VALUES LESS THAN (30),\n" + - " PARTITION `pMax` VALUES LESS THAN (MAXVALUE))")) - tk.MustQuery(`select * from t`).Sort().Check(testkit.Rows(""+ - "23 23", - "34 34πŸ’₯", - "46 46", - "57 57", - "7 07", - "8 08")) - tk.MustQuery(`select * from t order by c`).Check(testkit.Rows(""+ - "7 07", - "8 08", - "23 23", - "34 34πŸ’₯", - "46 46", - "57 57")) - tk.MustGetErrCode(`alter table t modify a varchar(20)`, errno.ErrUnsupportedDDLOperation) -} - func TestAlterModifyColumnOnPartitionedTableRename(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) @@ -4687,80 +4557,3 @@ func TestAlterModifyColumnOnPartitionedTableRename(t *testing.T) { tk.MustExec(`create table t (a int, b char) partition by hash (a) partitions 3`) tk.MustContainErrMsg(`alter table t change a c int`, "[planner:1054]Unknown column 'a' in 'expression'") } - -func TestAlterModifyColumnOnPartitionedTableFail(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - schemaName := "modColPartFail" - tk.MustExec("create database " + schemaName) - tk.MustExec("use " + schemaName) - tk.MustExec(`create table t (a int unsigned, b varchar(255), key (b)) partition by range (a) (partition p0 values less than (10), partition p1 values less than (20), partition pMax values less than (MAXVALUE))`) - tk.MustExec(`insert into t values (7, "07"), (8, "08"),(23,"23"),(34,"34πŸ’₯"),(46,"46"),(57,"57")`) - tk.MustGetErrCode(`alter table t modify a varchar(255)`, errno.ErrUnsupportedDDLOperation) - tk.MustGetErrCode(`alter table t modify a float`, mysql.ErrFieldTypeNotAllowedAsPartitionField) - tk.MustExec(`drop table t`) - tk.MustExec(`create table t (b int unsigned, a varchar(255), key (b)) partition by range columns (a) (partition p0 values less than (""), partition p1 values less than ("11111"), partition pMax values less than (MAXVALUE))`) - tk.MustExec(`insert into t values (7, "07"), (8, "08"),(23,"23"),(34,"34 πŸ’₯πŸ’₯Longer than 11111"),(46,"46"),(57,"57")`) - tk.MustExec(`alter table t modify a varchar(50)`) - tk.MustGetErrCode(`alter table t modify a float`, mysql.ErrFieldTypeNotAllowedAsPartitionField) - tk.MustGetErrCode(`alter table t modify a int`, errno.ErrUnsupportedDDLOperation) - tk.MustContainErrMsg(`alter table t modify a varchar(4)`, "[ddl:8200]New column does not match partition definitions: [ddl:1654]Partition column values of incorrect type") - tk.MustGetErrCode(`alter table t modify a varchar(5)`, errno.WarnDataTruncated) - tk.MustExec(`SET SQL_MODE = ''`) - tk.MustExec(`alter table t modify a varchar(5)`) - // fix https://github.com/pingcap/tidb/issues/38669 and update this - //tk.MustQuery(`show warnings`).Check(testkit.Rows("Warning 1265 Data truncated for column 'a', value is '34 πŸ’₯πŸ’₯Longer than 11111'")) - tk.MustExec(`SET SQL_MODE = DEFAULT`) - tk.MustQuery(`select * from t`).Sort().Check(testkit.Rows(""+ - "23 23", - "34 34 πŸ’₯πŸ’₯", - "46 46", - "57 57", - "7 07", - "8 08")) - tStr := "" + - "CREATE TABLE `t` (\n" + - " `b` int(10) unsigned DEFAULT NULL,\n" + - " `a` varchar(5) DEFAULT NULL,\n" + - " KEY `b` (`b`)\n" + - ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin\n" + - "PARTITION BY RANGE COLUMNS(`a`)\n" + - "(PARTITION `p0` VALUES LESS THAN (''),\n" + - " PARTITION `p1` VALUES LESS THAN ('11111'),\n" + - " PARTITION `pMax` VALUES LESS THAN (MAXVALUE))" - tk.MustQuery(`show create table t`).Check(testkit.Rows("t " + tStr)) - tk.MustExec(`drop table t`) - tk.MustExec(tStr) - tk.MustExec(`drop table t`) - tk.MustExec("create table t (a int, b varchar(255), key (b)) partition by range (a) (partition `p-300` values less than (-300), partition p0 values less than (0), partition p300 values less than (300))") - tk.MustExec(`insert into t values (-400, "-400"), (-100, "-100"), (0, "0"), (100, "100"), (290, "290")`) - tk.MustContainErrMsg(`alter table t modify a int unsigned`, "[ddl:8200]Unsupported modify column, decreasing length of int may result in truncation and change of partition") - tk.MustContainErrMsg(`alter table t modify a tinyint`, "[ddl:8200]Unsupported modify column, decreasing length of int may result in truncation and change of partition") - tk.MustExec(`set sql_mode = ''`) - tk.MustContainErrMsg(`alter table t modify a tinyint`, "[ddl:8200]Unsupported modify column, decreasing length of int may result in truncation and change of partition") - tk.MustQuery("select * from t partition (`p-300`)").Sort().Check(testkit.Rows("-400 -400")) - tk.MustExec(`set sql_mode = default`) - tk.MustContainErrMsg(`alter table t modify a smallint`, "[ddl:8200]Unsupported modify column, decreasing length of int may result in truncation and change of partition") - tk.MustExec(`alter table t modify a bigint`) - tk.MustExec(`drop table t`) - tk.MustExec("create table t (a int, b varchar(255), key (b)) partition by range columns (a) (partition `p-300` values less than (-300), partition p0 values less than (0), partition p300 values less than (300))") - tk.MustExec(`insert into t values (-400, "-400"), (-100, "-100"), (0, "0"), (100, "100"), (290, "290")`) - tk.MustContainErrMsg(`alter table t modify a int unsigned`, "[ddl:8200]Unsupported modify column: can't change the partitioning column, since it would require reorganize all partitions") - tk.MustContainErrMsg(`alter table t modify a tinyint`, "[ddl:8200]New column does not match partition definitions: [ddl:1654]Partition column values of incorrect type") - tk.MustExec(`set sql_mode = ''`) - tk.MustContainErrMsg(`alter table t modify a tinyint`, "[ddl:8200]New column does not match partition definitions: [ddl:1654]Partition column values of incorrect type") - tk.MustQuery("select * from t partition (`p-300`)").Sort().Check(testkit.Rows("-400 -400")) - tk.MustExec(`set sql_mode = default`) - // OK to decrease, since with RANGE COLUMNS, it will check the partition definition values against the new type - tk.MustExec(`alter table t modify a smallint`) - tk.MustExec(`alter table t modify a bigint`) - - tk.MustExec(`drop table t`) - - tk.MustExec(`create table t (a int, b varchar(255), key (b)) partition by list columns (b) (partition p1 values in ("1", "ab", "12345"), partition p2 values in ("2", "abc", "999999"))`) - tk.MustExec(`insert into t values (1, "1"), (2, "2"), (999999, "999999")`) - tk.MustContainErrMsg(`alter table t modify column b varchar(5)`, "[ddl:8200]New column does not match partition definitions: [ddl:1654]Partition column values of incorrect type") - tk.MustExec(`set sql_mode = ''`) - tk.MustContainErrMsg(`alter table t modify column b varchar(5)`, "[ddl:8200]New column does not match partition definitions: [ddl:1654]Partition column values of incorrect type") - tk.MustExec(`set sql_mode = default`) -} diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 84e64dcb61d6b..2850a3aa968a5 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -4681,6 +4681,9 @@ func GetModifiableColumnJob( if err = isGeneratedRelatedColumn(t.Meta(), newCol.ColumnInfo, col.ColumnInfo); err != nil { return nil, errors.Trace(err) } + if t.Meta().Partition != nil { + return nil, dbterror.ErrUnsupportedModifyColumn.GenWithStackByArgs("table is partition table") + } } // Check that the column change does not affect the partitioning column diff --git a/ddl/failtest/fail_db_test.go b/ddl/failtest/fail_db_test.go index d12c2182f9730..4a938e5fd2ad4 100644 --- a/ddl/failtest/fail_db_test.go +++ b/ddl/failtest/fail_db_test.go @@ -494,6 +494,8 @@ func TestModifyColumn(t *testing.T) { tk.MustExec("admin check table t") // Test unsupported statements. + tk.MustExec("create table t1(a int) partition by hash (a) partitions 2") + tk.MustGetErrMsg("alter table t1 modify column a mediumint", "[ddl:8200]Unsupported modify column: table is partition table") tk.MustExec("create table t2(id int, a int, b int generated always as (abs(a)) virtual, c int generated always as (a+1) stored)") tk.MustGetErrMsg("alter table t2 modify column b mediumint", "[ddl:8200]Unsupported modify column: newCol IsGenerated false, oldCol IsGenerated true") tk.MustGetErrMsg("alter table t2 modify column c mediumint", "[ddl:8200]Unsupported modify column: newCol IsGenerated false, oldCol IsGenerated true") @@ -530,7 +532,7 @@ func TestModifyColumn(t *testing.T) { tk.MustExec("insert into t5 values (1,1),(2,2),(3,3),(4,4),(5,5);") tk.MustExec("alter table t5 modify a int not null;") - tk.MustExec("drop table t, t2, t3, t4, t5") + tk.MustExec("drop table t, t1, t2, t3, t4, t5") } func TestIssue38699(t *testing.T) {