Skip to content

Commit

Permalink
ddl: Disable RENAME COLUMN for partitioning columns | tidb-test=pr/20…
Browse files Browse the repository at this point in the history
…54 (#40151)

close #40150
  • Loading branch information
mjonss authored Dec 29, 2022
1 parent 667bc4c commit c8124a0
Show file tree
Hide file tree
Showing 9 changed files with 29 additions and 17 deletions.
11 changes: 10 additions & 1 deletion ddl/column_change_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -438,6 +438,15 @@ func testNewContext(store kv.Storage) sessionctx.Context {
return ctx
}

func TestIssue40150(t *testing.T) {
store, _ := testkit.CreateMockStoreAndDomain(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")

tk.MustExec("CREATE TABLE t40150 (a int) PARTITION BY HASH (a) PARTITIONS 2")
tk.MustContainErrMsg(`alter table t40150 rename column a to c`, "[ddl:3855]Column 'a' has a partitioning function dependency and cannot be dropped or renamed")
}

func TestIssue40135(t *testing.T) {
store, dom := testkit.CreateMockStoreAndDomain(t)
tk := testkit.NewTestKit(t, store)
Expand All @@ -459,5 +468,5 @@ func TestIssue40135(t *testing.T) {
dom.DDL().SetHook(hook)
tk.MustExec("alter table t40135 modify column a MEDIUMINT NULL DEFAULT '6243108' FIRST")

require.ErrorContains(t, checkErr, "[ddl:8200]Unsupported modify column: Column 'a' has a partitioning function dependency and cannot be renamed")
require.ErrorContains(t, checkErr, "[ddl:3855]Column 'a' has a partitioning function dependency and cannot be dropped or renamed")
}
2 changes: 1 addition & 1 deletion ddl/column_modify_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -289,7 +289,7 @@ func TestDropColumn(t *testing.T) {
tk.MustExec("drop table if exists t1")
tk.MustExec("create table t1 (a int,b int) partition by hash(a) partitions 4;")
err := tk.ExecToErr("alter table t1 drop column a")
require.EqualError(t, err, "[ddl:3885]Column 'a' has a partitioning function dependency and cannot be dropped or renamed")
require.EqualError(t, err, "[ddl:3855]Column 'a' has a partitioning function dependency and cannot be dropped or renamed")
}

func TestChangeColumn(t *testing.T) {
Expand Down
18 changes: 9 additions & 9 deletions ddl/db_partition_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -4535,19 +4535,19 @@ func TestAlterModifyColumnOnPartitionedTableRename(t *testing.T) {
tk.MustExec("create database " + schemaName)
tk.MustExec("use " + schemaName)
tk.MustExec(`create table t (a int, b char) partition by range (a) (partition p0 values less than (10))`)
tk.MustContainErrMsg(`alter table t change a c int`, "[ddl:8200]Unsupported modify column: Column 'a' has a partitioning function dependency and cannot be renamed")
tk.MustContainErrMsg(`alter table t change a c int`, "[ddl:3855]Column 'a' has a partitioning function dependency and cannot be dropped or renamed")
tk.MustExec(`drop table t`)
tk.MustExec(`create table t (a char, b char) partition by range columns (a) (partition p0 values less than ('z'))`)
tk.MustContainErrMsg(`alter table t change a c char`, "[ddl:8200]Unsupported modify column: Column 'a' has a partitioning function dependency and cannot be renamed")
tk.MustContainErrMsg(`alter table t change a c char`, "[ddl:3855]Column 'a' has a partitioning function dependency and cannot be dropped or renamed")
tk.MustExec(`drop table t`)
tk.MustExec(`create table t (a int, b char) partition by list (a) (partition p0 values in (10))`)
tk.MustContainErrMsg(`alter table t change a c int`, "[ddl:8200]Unsupported modify column: Column 'a' has a partitioning function dependency and cannot be renamed")
tk.MustContainErrMsg(`alter table t change a c int`, "[ddl:3855]Column 'a' has a partitioning function dependency and cannot be dropped or renamed")
tk.MustExec(`drop table t`)
tk.MustExec(`create table t (a char, b char) partition by list columns (a) (partition p0 values in ('z'))`)
tk.MustContainErrMsg(`alter table t change a c char`, "[ddl:8200]Unsupported modify column: Column 'a' has a partitioning function dependency and cannot be renamed")
tk.MustContainErrMsg(`alter table t change a c char`, "[ddl:3855]Column 'a' has a partitioning function dependency and cannot be dropped or renamed")
tk.MustExec(`drop table 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`, "[ddl:8200]Unsupported modify column: Column 'a' has a partitioning function dependency and cannot be renamed")
tk.MustContainErrMsg(`alter table t change a c int`, "[ddl:3855]Column 'a' has a partitioning function dependency and cannot be dropped or renamed")
}

func TestDropPartitionKeyColumn(t *testing.T) {
Expand All @@ -4560,24 +4560,24 @@ func TestDropPartitionKeyColumn(t *testing.T) {
tk.MustExec("create table t1 (a tinyint, b char) partition by range (a) ( partition p0 values less than (10) )")
err := tk.ExecToErr("alter table t1 drop column a")
require.Error(t, err)
require.Equal(t, "[ddl:3885]Column 'a' has a partitioning function dependency and cannot be dropped or renamed", err.Error())
require.Equal(t, "[ddl:3855]Column 'a' has a partitioning function dependency and cannot be dropped or renamed", err.Error())
tk.MustExec("alter table t1 drop column b")

tk.MustExec("create table t2 (a tinyint, b char) partition by range (a-1) ( partition p0 values less than (10) )")
err = tk.ExecToErr("alter table t2 drop column a")
require.Error(t, err)
require.Equal(t, "[ddl:3885]Column 'a' has a partitioning function dependency and cannot be dropped or renamed", err.Error())
require.Equal(t, "[ddl:3855]Column 'a' has a partitioning function dependency and cannot be dropped or renamed", err.Error())
tk.MustExec("alter table t2 drop column b")

tk.MustExec("create table t3 (a tinyint, b char) partition by hash(a) partitions 4;")
err = tk.ExecToErr("alter table t3 drop column a")
require.Error(t, err)
require.Equal(t, "[ddl:3885]Column 'a' has a partitioning function dependency and cannot be dropped or renamed", err.Error())
require.Equal(t, "[ddl:3855]Column 'a' has a partitioning function dependency and cannot be dropped or renamed", err.Error())
tk.MustExec("alter table t3 drop column b")

tk.MustExec("create table t4 (a char, b char) partition by list columns (a) ( partition p0 values in ('0'), partition p1 values in ('a'), partition p2 values in ('b'));")
err = tk.ExecToErr("alter table t4 drop column a")
require.Error(t, err)
require.Equal(t, "[ddl:3885]Column 'a' has a partitioning function dependency and cannot be dropped or renamed", err.Error())
require.Equal(t, "[ddl:3855]Column 'a' has a partitioning function dependency and cannot be dropped or renamed", err.Error())
tk.MustExec("alter table t4 drop column b")
}
2 changes: 1 addition & 1 deletion ddl/db_table_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -871,7 +871,7 @@ func TestDDLWithInvalidTableInfo(t *testing.T) {

tk.MustExec("create table t (a bigint, b int, c int generated always as (b+1)) partition by hash(a) partitions 4;")
// Test drop partition column.
tk.MustGetErrMsg("alter table t drop column a;", "[ddl:3885]Column 'a' has a partitioning function dependency and cannot be dropped or renamed")
tk.MustGetErrMsg("alter table t drop column a;", "[ddl:3855]Column 'a' has a partitioning function dependency and cannot be dropped or renamed")
// Test modify column with invalid expression.
tk.MustGetErrMsg("alter table t modify column c int GENERATED ALWAYS AS ((case when (a = 0) then 0when (a > 0) then (b / a) end));", "[parser:1064]You have an error in your SQL syntax; check the manual that corresponds to your TiDB version for the right syntax to use line 1 column 97 near \"then (b / a) end));\" ")
// Test add column with invalid expression.
Expand Down
7 changes: 6 additions & 1 deletion ddl/ddl_api.go
Original file line number Diff line number Diff line change
Expand Up @@ -4734,7 +4734,7 @@ func GetModifiableColumnJob(
// TODO: update the partitioning columns with new names if column is renamed
// Would be an extension from MySQL which does not support it.
if col.Name.L != newCol.Name.L {
return nil, dbterror.ErrUnsupportedModifyColumn.GenWithStackByArgs(fmt.Sprintf("Column '%s' has a partitioning function dependency and cannot be renamed", col.Name.O))
return nil, dbterror.ErrDependentByPartitionFunctional.GenWithStackByArgs(col.Name.L)
}
if !isColTypeAllowedAsPartitioningCol(newCol.FieldType) {
return nil, dbterror.ErrNotAllowedTypeInPartition.GenWithStackByArgs(newCol.Name.O)
Expand Down Expand Up @@ -5110,6 +5110,11 @@ func (d *ddl) RenameColumn(ctx sessionctx.Context, ident ast.Ident, spec *ast.Al
}
}

err = checkDropColumnWithPartitionConstraint(tbl, oldColName)
if err != nil {
return errors.Trace(err)
}

tzName, tzOffset := ddlutil.GetTimeZone(ctx)

newCol := oldCol.Clone()
Expand Down
2 changes: 1 addition & 1 deletion errno/errcode.go
Original file line number Diff line number Diff line change
Expand Up @@ -915,7 +915,7 @@ const (
ErrFunctionalIndexRowValueIsNotAllowed = 3800
ErrDependentByFunctionalIndex = 3837
ErrCannotConvertString = 3854
ErrDependentByPartitionFunctional = 3885
ErrDependentByPartitionFunctional = 3855
ErrInvalidJSONValueForFuncIndex = 3903
ErrJSONValueOutOfRangeForFuncIndex = 3904
ErrFunctionalIndexDataIsTooLong = 3907
Expand Down
2 changes: 1 addition & 1 deletion errors.toml
Original file line number Diff line number Diff line change
Expand Up @@ -1176,7 +1176,7 @@ error = '''
Column '%s' has an expression index dependency and cannot be dropped or renamed
'''

["ddl:3885"]
["ddl:3855"]
error = '''
Column '%s' has a partitioning function dependency and cannot be dropped or renamed
'''
Expand Down
1 change: 0 additions & 1 deletion parser/mysql/errcode.go
Original file line number Diff line number Diff line change
Expand Up @@ -944,7 +944,6 @@ const (
ErrFKIncompatibleColumns = 3780
ErrFunctionalIndexRowValueIsNotAllowed = 3800
ErrDependentByFunctionalIndex = 3837
ErrDependentByPartitionFunctional = 3885
ErrInvalidJsonValueForFuncIndex = 3903 //nolint: revive
ErrJsonValueOutOfRangeForFuncIndex = 3904 //nolint: revive
ErrFunctionalIndexDataIsTooLong = 3907
Expand Down
1 change: 0 additions & 1 deletion parser/mysql/errname.go
Original file line number Diff line number Diff line change
Expand Up @@ -956,7 +956,6 @@ var MySQLErrName = map[uint16]*ErrMessage{
ErrJsonValueOutOfRangeForFuncIndex: Message("Out of range JSON value for CAST for functional index '%s'", nil),
ErrFunctionalIndexDataIsTooLong: Message("Data too long for functional index '%s'", nil),
ErrFunctionalIndexNotApplicable: Message("Cannot use functional index '%s' due to type or collation conversion", nil),
ErrDependentByPartitionFunctional: Message("Column '%s' has a partitioning function dependency and cannot be dropped or renamed", nil),

// MariaDB errors.
ErrOnlyOneDefaultPartionAllowed: Message("Only one DEFAULT partition allowed", nil),
Expand Down

0 comments on commit c8124a0

Please sign in to comment.