From 00b99ec17ec4345db07849327cc16295e75235e4 Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Sat, 5 Nov 2022 03:34:00 +0800 Subject: [PATCH] ddl: restricting MODIFY COLUMN on partitioning columns. (#38670) (#38904) close pingcap/tidb#38530 --- ddl/db_partition_test.go | 78 ++++++++++++ ddl/ddl_api.go | 112 ++++++++++++++++-- ddl/partition.go | 48 ++++++++ executor/show.go | 48 +------- .../pessimistictest/pessimistic_test.go | 6 + 5 files changed, 234 insertions(+), 58 deletions(-) diff --git a/ddl/db_partition_test.go b/ddl/db_partition_test.go index fb92c3087b78b..b10f01c272ef9 100644 --- a/ddl/db_partition_test.go +++ b/ddl/db_partition_test.go @@ -4662,4 +4662,82 @@ func TestAlterModifyColumnOnPartitionedTable(t *testing.T) { "34 34💥", "46 46", "57 57")) + tk.MustGetErrCode(`alter table t modify a varchar(20)`, errno.ErrUnsupportedDDLOperation) +} + +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()) + 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 3ce114b2e78a2..e67c0eb001128 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -38,6 +38,7 @@ import ( "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta/autoid" + "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/charset" "github.com/pingcap/tidb/parser/format" @@ -2831,23 +2832,30 @@ func checkPartitionByList(ctx sessionctx.Context, tbInfo *model.TableInfo) error return checkListPartitionValue(ctx, tbInfo) } +func isColTypeAllowedAsPartitioningCol(fieldType types.FieldType) bool { + // The permitted data types are shown in the following list: + // All integer types + // DATE and DATETIME + // CHAR, VARCHAR, BINARY, and VARBINARY + // See https://dev.mysql.com/doc/mysql-partitioning-excerpt/5.7/en/partitioning-columns.html + // Note that also TIME is allowed in MySQL. Also see https://bugs.mysql.com/bug.php?id=84362 + switch fieldType.GetType() { + case mysql.TypeTiny, mysql.TypeShort, mysql.TypeInt24, mysql.TypeLong, mysql.TypeLonglong: + case mysql.TypeDate, mysql.TypeDatetime, mysql.TypeDuration: + case mysql.TypeVarchar, mysql.TypeString: + default: + return false + } + return true +} + func checkColumnsPartitionType(tbInfo *model.TableInfo) error { for _, col := range tbInfo.Partition.Columns { colInfo := tbInfo.FindPublicColumnByName(col.L) if colInfo == nil { return errors.Trace(dbterror.ErrFieldNotFoundPart) } - // The permitted data types are shown in the following list: - // All integer types - // DATE and DATETIME - // CHAR, VARCHAR, BINARY, and VARBINARY - // See https://dev.mysql.com/doc/mysql-partitioning-excerpt/5.7/en/partitioning-columns.html - // Note that also TIME is allowed in MySQL. Also see https://bugs.mysql.com/bug.php?id=84362 - switch colInfo.FieldType.GetType() { - case mysql.TypeTiny, mysql.TypeShort, mysql.TypeInt24, mysql.TypeLong, mysql.TypeLonglong: - case mysql.TypeDate, mysql.TypeDatetime, mysql.TypeDuration: - case mysql.TypeVarchar, mysql.TypeString: - default: + if !isColTypeAllowedAsPartitioningCol(colInfo.FieldType) { return dbterror.ErrNotAllowedTypeInPartition.GenWithStackByArgs(col.O) } } @@ -4604,6 +4612,88 @@ func GetModifiableColumnJob( } } + // Check that the column change does not affect the partitioning column + // It must keep the same type, int [unsigned], [var]char, date[time] + if t.Meta().Partition != nil { + pt, ok := t.(table.PartitionedTable) + if !ok { + // Should never happen! + return nil, dbterror.ErrNotAllowedTypeInPartition.GenWithStackByArgs(newCol.Name.O) + } + isPartitioningColumn := false + for _, name := range pt.GetPartitionColumnNames() { + if strings.EqualFold(name.L, col.Name.L) { + isPartitioningColumn = true + } + } + if isPartitioningColumn { + if !isColTypeAllowedAsPartitioningCol(newCol.FieldType) { + return nil, dbterror.ErrNotAllowedTypeInPartition.GenWithStackByArgs(newCol.Name.O) + } + pi := pt.Meta().GetPartitionInfo() + if len(pi.Columns) == 0 { + // non COLUMNS partitioning, only checks INTs, not their actual range + // There are many edge cases, like when truncating SQL Mode is allowed + // which will change the partitioning expression value resulting in a + // different partition. Better be safe and not allow decreasing of length. + // TODO: Should we allow it in strict mode? Wait for a use case / request. + if newCol.FieldType.GetFlen() < col.FieldType.GetFlen() { + return nil, dbterror.ErrUnsupportedModifyCollation.GenWithStack("Unsupported modify column, decreasing length of int may result in truncation and change of partition") + } + } + // Basically only allow changes of the length/decimals for the column + // Note that enum is not allowed, so elems are not checked + // TODO: support partition by ENUM + if newCol.FieldType.EvalType() != col.FieldType.EvalType() || + newCol.FieldType.GetFlag() != col.FieldType.GetFlag() || + newCol.FieldType.GetCollate() != col.FieldType.GetCollate() || + newCol.FieldType.GetCharset() != col.FieldType.GetCharset() { + return nil, dbterror.ErrUnsupportedModifyColumn.GenWithStackByArgs("can't change the partitioning column, since it would require reorganize all partitions") + } + // Generate a new PartitionInfo and validate it together with the new column definition + // Checks if all partition definition values are compatible. + // Similar to what buildRangePartitionDefinitions would do in terms of checks. + + tblInfo := pt.Meta() + newTblInfo := *tblInfo + // Replace col with newCol and see if we can generate a new SHOW CREATE TABLE + // and reparse it and build new partition definitions (which will do additional + // checks columns vs partition definition values + newCols := make([]*model.ColumnInfo, 0, len(newTblInfo.Columns)) + for _, c := range newTblInfo.Columns { + if c.ID == col.ID { + newCols = append(newCols, newCol.ColumnInfo) + continue + } + newCols = append(newCols, c) + } + newTblInfo.Columns = newCols + + var buf bytes.Buffer + AppendPartitionInfo(tblInfo.GetPartitionInfo(), &buf, mysql.ModeNone) + // The parser supports ALTER TABLE ... PARTITION BY ... even if the ddl code does not yet :) + // Ignoring warnings + stmt, _, err := parser.New().ParseSQL("ALTER TABLE t " + buf.String()) + if err != nil { + // Should never happen! + return nil, dbterror.ErrUnsupportedModifyColumn.GenWithStack("cannot parse generated PartitionInfo") + } + at, ok := stmt[0].(*ast.AlterTableStmt) + if !ok || len(at.Specs) != 1 || at.Specs[0].Partition == nil { + return nil, dbterror.ErrUnsupportedModifyColumn.GenWithStack("cannot parse generated PartitionInfo") + } + pAst := at.Specs[0].Partition + sv := sctx.GetSessionVars().StmtCtx + oldTruncAsWarn, oldIgnoreTrunc := sv.TruncateAsWarning, sv.IgnoreTruncate + sv.TruncateAsWarning, sv.IgnoreTruncate = false, false + _, err = buildPartitionDefinitionsInfo(sctx, pAst.Definitions, &newTblInfo) + sv.TruncateAsWarning, sv.IgnoreTruncate = oldTruncAsWarn, oldIgnoreTrunc + if err != nil { + return nil, dbterror.ErrUnsupportedModifyColumn.GenWithStack("New column does not match partition definitions: %s", err.Error()) + } + } + } + // We don't support modifying column from not_auto_increment to auto_increment. if !mysql.HasAutoIncrementFlag(col.GetFlag()) && mysql.HasAutoIncrementFlag(newCol.GetFlag()) { return nil, dbterror.ErrUnsupportedModifyColumn.GenWithStackByArgs("can't set auto_increment") diff --git a/ddl/partition.go b/ddl/partition.go index c9a00a52fe5ec..5a6a5b561b146 100644 --- a/ddl/partition.go +++ b/ddl/partition.go @@ -2807,6 +2807,54 @@ func hexIfNonPrint(s string) string { return "0x" + hex.EncodeToString([]byte(driver.UnwrapFromSingleQuotes(s))) } +// AppendPartitionInfo is used in SHOW CREATE TABLE as well as generation the SQL syntax +// for the PartitionInfo during validation of various DDL commands +func AppendPartitionInfo(partitionInfo *model.PartitionInfo, buf *bytes.Buffer, sqlMode mysql.SQLMode) { + if partitionInfo == nil { + return + } + // Since MySQL 5.1/5.5 is very old and TiDB aims for 5.7/8.0 compatibility, we will not + // include the /*!50100 or /*!50500 comments for TiDB. + // This also solves the issue with comments within comments that would happen for + // PLACEMENT POLICY options. + if partitionInfo.Type == model.PartitionTypeHash { + defaultPartitionDefinitions := true + for i, def := range partitionInfo.Definitions { + if def.Name.O != fmt.Sprintf("p%d", i) { + defaultPartitionDefinitions = false + break + } + if len(def.Comment) > 0 || def.PlacementPolicyRef != nil { + defaultPartitionDefinitions = false + break + } + } + + if defaultPartitionDefinitions { + fmt.Fprintf(buf, "\nPARTITION BY HASH (%s) PARTITIONS %d", partitionInfo.Expr, partitionInfo.Num) + return + } + } + // this if statement takes care of lists/range columns case + if len(partitionInfo.Columns) > 0 { + // partitionInfo.Type == model.PartitionTypeRange || partitionInfo.Type == model.PartitionTypeList + // Notice that MySQL uses two spaces between LIST and COLUMNS... + fmt.Fprintf(buf, "\nPARTITION BY %s COLUMNS(", partitionInfo.Type.String()) + for i, col := range partitionInfo.Columns { + buf.WriteString(stringutil.Escape(col.O, sqlMode)) + if i < len(partitionInfo.Columns)-1 { + buf.WriteString(",") + } + } + buf.WriteString(")\n(") + } else { + fmt.Fprintf(buf, "\nPARTITION BY %s (%s)\n(", partitionInfo.Type.String(), partitionInfo.Expr) + } + + AppendPartitionDefs(partitionInfo, buf, sqlMode) + buf.WriteString(")") +} + // AppendPartitionDefs generates a list of partition definitions needed for SHOW CREATE TABLE (in executor/show.go) // as well as needed for generating the ADD PARTITION query for INTERVAL partitioning of ALTER TABLE t LAST PARTITION // and generating the CREATE TABLE query from CREATE TABLE ... INTERVAL diff --git a/executor/show.go b/executor/show.go index 9aa6b56f353b5..e96e95b21e699 100644 --- a/executor/show.go +++ b/executor/show.go @@ -1219,7 +1219,7 @@ func ConstructResultOfShowCreateTable(ctx sessionctx.Context, tableInfo *model.T } // add partition info here. - appendPartitionInfo(tableInfo.Partition, buf, sqlMode) + ddl.AppendPartitionInfo(tableInfo.Partition, buf, sqlMode) return nil } @@ -1352,52 +1352,6 @@ func fetchShowCreateTable4View(ctx sessionctx.Context, tb *model.TableInfo, buf fmt.Fprintf(buf, ") AS %s", tb.View.SelectStmt) } -func appendPartitionInfo(partitionInfo *model.PartitionInfo, buf *bytes.Buffer, sqlMode mysql.SQLMode) { - if partitionInfo == nil { - return - } - // Since MySQL 5.1/5.5 is very old and TiDB aims for 5.7/8.0 compatibility, we will not - // include the /*!50100 or /*!50500 comments for TiDB. - // This also solves the issue with comments within comments that would happen for - // PLACEMENT POLICY options. - if partitionInfo.Type == model.PartitionTypeHash { - defaultPartitionDefinitions := true - for i, def := range partitionInfo.Definitions { - if def.Name.O != fmt.Sprintf("p%d", i) { - defaultPartitionDefinitions = false - break - } - if len(def.Comment) > 0 || def.PlacementPolicyRef != nil { - defaultPartitionDefinitions = false - break - } - } - - if defaultPartitionDefinitions { - fmt.Fprintf(buf, "\nPARTITION BY HASH (%s) PARTITIONS %d", partitionInfo.Expr, partitionInfo.Num) - return - } - } - // this if statement takes care of lists/range columns case - if len(partitionInfo.Columns) > 0 { - // partitionInfo.Type == model.PartitionTypeRange || partitionInfo.Type == model.PartitionTypeList - // Notice that MySQL uses two spaces between LIST and COLUMNS... - fmt.Fprintf(buf, "\nPARTITION BY %s COLUMNS(", partitionInfo.Type.String()) - for i, col := range partitionInfo.Columns { - buf.WriteString(stringutil.Escape(col.O, sqlMode)) - if i < len(partitionInfo.Columns)-1 { - buf.WriteString(",") - } - } - buf.WriteString(")\n(") - } else { - fmt.Fprintf(buf, "\nPARTITION BY %s (%s)\n(", partitionInfo.Type.String(), partitionInfo.Expr) - } - - ddl.AppendPartitionDefs(partitionInfo, buf, sqlMode) - buf.WriteString(")") -} - // ConstructResultOfShowCreateDatabase constructs the result for show create database. func ConstructResultOfShowCreateDatabase(ctx sessionctx.Context, dbInfo *model.DBInfo, ifNotExists bool, buf *bytes.Buffer) (err error) { sqlMode := ctx.GetSessionVars().SQLMode diff --git a/tests/realtikvtest/pessimistictest/pessimistic_test.go b/tests/realtikvtest/pessimistictest/pessimistic_test.go index 60dead95abf1a..5e2da1d200651 100644 --- a/tests/realtikvtest/pessimistictest/pessimistic_test.go +++ b/tests/realtikvtest/pessimistictest/pessimistic_test.go @@ -3039,6 +3039,12 @@ func TestAmendForColumnChange(t *testing.T) { } // Start a pessimistic transaction for partition table, the amend should fail. + if i == 5 { + // alter table t_part modify column c_int bigint(20) default 100 + // Unsupported modify column: can't change the partitioning column, since it would require reorganize all partitions + // Skip this case + continue + } tk.MustExec("begin pessimistic") tk.MustExec(`insert into t_part values(5, "555", "2000-01-05", "2020-01-05", "5.5", "555.555", 5.5)`) tk2.MustExec(colChangeFunc(true, i))