Skip to content

Commit

Permalink
MODIFY COLUMN on columns in the partitioning expression (pingcap#38530)
Browse files Browse the repository at this point in the history
  • Loading branch information
mjonss committed Oct 26, 2022
1 parent 64051f9 commit 88b122b
Show file tree
Hide file tree
Showing 4 changed files with 222 additions and 58 deletions.
69 changes: 69 additions & 0 deletions ddl/db_partition_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -4662,4 +4662,73 @@ 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)`)
// TODO: Investigate why there are no warnings here?!?
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`)
}
115 changes: 104 additions & 11 deletions ddl/ddl_api.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -2789,23 +2790,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)
}
}
Expand Down Expand Up @@ -4562,6 +4570,91 @@ 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")
}
}
// TODO: Test with LIST partitioning and decreasing the length so it is shorter
// than existing partition definitions. (RANGE is OK, since LESS THAN)

// 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.
// Try 1: Test generating the string, then parse it with the new column?
//func buildPartitionDefinitionsInfo(ctx sessionctx.Context, defs []*ast.PartitionDefinition, tbInfo *model.TableInfo) (partitions []model.PartitionDefinition, err error) {

// TODO: do not allow truncations, not even if not Strict SQLMode!!!

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
// TODO: Test and handle SQLMode?

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
if _, err = buildPartitionDefinitionsInfo(sctx, pAst.Definitions, &newTblInfo); 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")
Expand Down
48 changes: 48 additions & 0 deletions ddl/partition.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
48 changes: 1 addition & 47 deletions executor/show.go
Original file line number Diff line number Diff line change
Expand Up @@ -1216,7 +1216,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
}

Expand Down Expand Up @@ -1349,52 +1349,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
Expand Down

0 comments on commit 88b122b

Please sign in to comment.