Skip to content

Commit

Permalink
ddl: Also check column types in ALTER TABLE t PARTITION BY ... (#56197)
Browse files Browse the repository at this point in the history
close #56094
  • Loading branch information
mjonss authored Sep 25, 2024
1 parent a06f6aa commit cabedbc
Show file tree
Hide file tree
Showing 4 changed files with 146 additions and 9 deletions.
46 changes: 37 additions & 9 deletions pkg/ddl/partition.go
Original file line number Diff line number Diff line change
Expand Up @@ -1788,27 +1788,38 @@ func checkResultOK(ok bool) error {
}

// checkPartitionFuncType checks partition function return type.
func checkPartitionFuncType(ctx sessionctx.Context, expr ast.ExprNode, schema string, tblInfo *model.TableInfo) error {
if expr == nil {
func checkPartitionFuncType(ctx sessionctx.Context, anyExpr any, schema string, tblInfo *model.TableInfo) error {
if anyExpr == nil {
return nil
}

if schema == "" {
schema = ctx.GetSessionVars().CurrentDB
}

e, err := expression.BuildSimpleExpr(ctx.GetExprCtx(), expr, expression.WithTableInfo(schema, tblInfo))
var e expression.Expression
var err error
switch expr := anyExpr.(type) {
case string:
if expr == "" {
return nil
}
e, err = expression.ParseSimpleExpr(ctx.GetExprCtx(), expr, expression.WithTableInfo(schema, tblInfo))
case ast.ExprNode:
e, err = expression.BuildSimpleExpr(ctx.GetExprCtx(), expr, expression.WithTableInfo(schema, tblInfo))
default:
return errors.Trace(dbterror.ErrPartitionFuncNotAllowed.GenWithStackByArgs("PARTITION"))
}
if err != nil {
return errors.Trace(err)
}
if e.GetType(ctx.GetExprCtx().GetEvalCtx()).EvalType() == types.ETInt {
return nil
}

if col, ok := expr.(*ast.ColumnNameExpr); ok {
return errors.Trace(dbterror.ErrNotAllowedTypeInPartition.GenWithStackByArgs(col.Name.Name.L))
if col, ok := e.(*expression.Column); ok {
if col2, ok2 := anyExpr.(*ast.ColumnNameExpr); ok2 {
return errors.Trace(dbterror.ErrNotAllowedTypeInPartition.GenWithStackByArgs(col2.Name.Name.L))
}
return errors.Trace(dbterror.ErrNotAllowedTypeInPartition.GenWithStackByArgs(col.OrigName))
}

return errors.Trace(dbterror.ErrPartitionFuncNotAllowed.GenWithStackByArgs("PARTITION"))
}

Expand Down Expand Up @@ -3093,6 +3104,23 @@ func (w *worker) onReorganizePartition(jobCtx *jobContext, t *meta.Meta, job *mo
return ver, err
}

if job.Type == model.ActionAlterTablePartitioning {
// Also verify same things as in CREATE TABLE ... PARTITION BY
if len(partInfo.Columns) > 0 {
// shallow copy, only for reading/checking
tmpTblInfo := *tblInfo
tmpTblInfo.Partition = partInfo
if err = checkColumnsPartitionType(&tmpTblInfo); err != nil {
job.State = model.JobStateCancelled
return ver, err
}
} else {
if err = checkPartitionFuncType(sctx, partInfo.Expr, job.SchemaName, tblInfo); err != nil {
job.State = model.JobStateCancelled
return ver, err
}
}
}
// move the adding definition into tableInfo.
updateAddingPartitionInfo(partInfo, tblInfo)
orgDefs := tblInfo.Partition.Definitions
Expand Down
86 changes: 86 additions & 0 deletions pkg/ddl/reorg_partition_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,7 @@ import (
"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/dbterror"
"github.com/pingcap/tidb/pkg/util/mathutil"
"github.com/stretchr/testify/require"
"go.uber.org/zap"
Expand Down Expand Up @@ -558,3 +559,88 @@ func TestReorgPartitionRollback(t *testing.T) {
require.NoError(t, err)
noNewTablesAfter(t, tk, ctx, tbl)
}

func TestPartitionByColumnChecks(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
cols := "(i int, f float, c char(20), b bit(2), b32 bit(32), b64 bit(64), d date, dt datetime, dt6 datetime(6), ts timestamp, ts6 timestamp(6), j json)"
vals := `(1, 2.2, "A and c", b'10', b'10001000100010001000100010001000', b'1000100010001000100010001000100010001000100010001000100010001000', '2024-09-24', '2024-09-24 13:01:02', '2024-09-24 13:01:02.123456', '2024-09-24 13:01:02', '2024-09-24 13:01:02.123456', '{"key1": "value1", "key2": "value2"}')`
tk.MustExec(`create table t ` + cols)
testCases := []struct {
partClause string
err error
}{
{"key (c) partitions 2", nil},
{"key (j) partitions 2", dbterror.ErrNotAllowedTypeInPartition},
{"list (c) (partition pDef default)", dbterror.ErrNotAllowedTypeInPartition},
{"list (b) (partition pDef default)", nil},
{"list (f) (partition pDef default)", dbterror.ErrNotAllowedTypeInPartition},
{"list (j) (partition pDef default)", dbterror.ErrNotAllowedTypeInPartition},
{"list columns (b) (partition pDef default)", dbterror.ErrNotAllowedTypeInPartition},
{"list columns (f) (partition pDef default)", dbterror.ErrNotAllowedTypeInPartition},
{"list columns (ts) (partition pDef default)", dbterror.ErrNotAllowedTypeInPartition},
{"list columns (j) (partition pDef default)", dbterror.ErrNotAllowedTypeInPartition},
{"hash (year(ts)) partitions 2", dbterror.ErrWrongExprInPartitionFunc},
{"hash (ts) partitions 2", dbterror.ErrNotAllowedTypeInPartition},
{"hash (ts6) partitions 2", dbterror.ErrNotAllowedTypeInPartition},
{"hash (d) partitions 2", dbterror.ErrNotAllowedTypeInPartition},
{"hash (f) partitions 2", dbterror.ErrNotAllowedTypeInPartition},
{"range (c) (partition pMax values less than (maxvalue))", dbterror.ErrNotAllowedTypeInPartition},
{"range (f) (partition pMax values less than (maxvalue))", dbterror.ErrNotAllowedTypeInPartition},
{"range (d) (partition pMax values less than (maxvalue))", dbterror.ErrNotAllowedTypeInPartition},
{"range (dt) (partition pMax values less than (maxvalue))", dbterror.ErrNotAllowedTypeInPartition},
{"range (dt6) (partition pMax values less than (maxvalue))", dbterror.ErrNotAllowedTypeInPartition},
{"range (ts) (partition pMax values less than (maxvalue))", dbterror.ErrNotAllowedTypeInPartition},
{"range (ts6) (partition pMax values less than (maxvalue))", dbterror.ErrNotAllowedTypeInPartition},
{"range (j) (partition pMax values less than (maxvalue))", dbterror.ErrNotAllowedTypeInPartition},
{"range columns (b) (partition pMax values less than (maxvalue))", dbterror.ErrNotAllowedTypeInPartition},
{"range columns (b64) (partition pMax values less than (maxvalue))", dbterror.ErrNotAllowedTypeInPartition},
{"range columns (c) (partition pMax values less than (maxvalue))", nil},
{"range columns (f) (partition pMax values less than (maxvalue))", dbterror.ErrNotAllowedTypeInPartition},
{"range columns (d) (partition pMax values less than (maxvalue))", nil},
{"range columns (dt) (partition pMax values less than (maxvalue))", nil},
{"range columns (dt6) (partition pMax values less than (maxvalue))", nil},
{"range columns (ts) (partition pMax values less than (maxvalue))", dbterror.ErrNotAllowedTypeInPartition},
{"range columns (ts6) (partition pMax values less than (maxvalue))", dbterror.ErrNotAllowedTypeInPartition},
{"range columns (j) (partition pMax values less than (maxvalue))", dbterror.ErrNotAllowedTypeInPartition},
}
for _, testCase := range testCases {
err := tk.ExecToErr(`create table tt ` + cols + ` partition by ` + testCase.partClause)
require.ErrorIs(t, err, testCase.err, testCase.partClause)
if testCase.err == nil {
tk.MustExec(`drop table tt`)
}
err = tk.ExecToErr(`alter table t partition by ` + testCase.partClause)
require.ErrorIs(t, err, testCase.err)
}

// Not documented or tested!!
// KEY - Allows more types than documented, should be OK!
tk.MustExec(`create table kb ` + cols + ` partition by key(b) partitions 2`)
tk.MustExec(`create table kf ` + cols + ` partition by key(f) partitions 2`)
tk.MustExec(`create table kts ` + cols + ` partition by key(ts) partitions 2`)
// HASH/LIST/RANGE - Treats bit values as int, BIT(>=32) for HASH fails due to overflow...
tk.MustExec(`create table hb ` + cols + ` partition by hash(b) partitions 2`)
tk.MustExec(`insert into hb values ` + vals)
tk.MustQuery(`select count(*) from hb where b = b'10'`).Check(testkit.Rows("1"))
tk.MustExec(`alter table hb partition by hash(b) partitions 3`)
tk.MustExec(`insert into hb values ` + vals)
tk.MustQuery(`select count(*) from hb where b = b'10'`).Check(testkit.Rows("2"))
tk.MustExec(`create table hb32 ` + cols + ` partition by hash(b32) partitions 2`)
tk.MustContainErrMsg(`insert into hb32 values `+vals, "[types:1690]constant 2290649224 overflows int")
tk.MustExec(`alter table hb32 partition by hash(b32) partitions 3`)
tk.MustContainErrMsg(`insert into hb32 values `+vals, "[types:1690]constant 2290649224 overflows int")
tk.MustExec(`create table rb ` + cols + ` partition by range (b) (partition pMax values less than (MAXVALUE))`)
tk.MustExec(`insert into rb values ` + vals)
tk.MustExec(`alter table rb partition by range(b) (partition pMax values less than (MAXVALUE))`)
tk.MustExec(`insert into rb values ` + vals)
tk.MustExec(`create table rb32 ` + cols + ` partition by range (b32) (partition pMax values less than (MAXVALUE))`)
tk.MustExec(`insert into rb32 values ` + vals)
tk.MustExec(`alter table rb32 partition by range(b32) (partition pMax values less than (MAXVALUE))`)
tk.MustExec(`insert into rb32 values ` + vals)
tk.MustExec(`create table rb64 ` + cols + ` partition by range (b64) (partition pMax values less than (MAXVALUE))`)
tk.MustExec(`insert into rb64 values ` + vals)
tk.MustExec(`alter table rb64 partition by range(b64) (partition pMax values less than (MAXVALUE))`)
tk.MustExec(`insert into rb64 values ` + vals)
}
11 changes: 11 additions & 0 deletions tests/integrationtest/r/ddl/reorg_partition.result
Original file line number Diff line number Diff line change
Expand Up @@ -830,3 +830,14 @@ a b c
2021-05-04 2021-05-04 10:10:10 4
2022-05-04 2022-05-04 10:10:10 5
2022-05-05 2022-05-06 11:11:11 6
drop table t;
create table t(a varchar(20), b int) partition by hash(a) partitions 2;
Error 1659 (HY000): Field 'a' is of a not allowed type for this type of partitioning
create table t(a varchar(20), b int);
alter table t partition by hash(a) partitions 2;
Error 1659 (HY000): Field 'ddl__reorg_partition.t.a' is of a not allowed type for this type of partitioning
alter table t partition by key() partitions 2;
Error 1105 (HY000): expression should not be an empty string
alter table t partition by key(c) partitions 2;
Error 1488 (HY000): Field in list of fields for partition function not found in table
drop table t;
12 changes: 12 additions & 0 deletions tests/integrationtest/t/ddl/reorg_partition.test
Original file line number Diff line number Diff line change
Expand Up @@ -390,4 +390,16 @@ ADMIN CHECK TABLE t;
SELECT * FROM t PARTITION(pAll);
--sorted_result
SELECT * FROM t;
drop table t;

# Issue 56094
-- error 1659
create table t(a varchar(20), b int) partition by hash(a) partitions 2;
create table t(a varchar(20), b int);
-- error 1659
alter table t partition by hash(a) partitions 2;
-- error 1105
alter table t partition by key() partitions 2;
-- error 1488
alter table t partition by key(c) partitions 2;
drop table t;

0 comments on commit cabedbc

Please sign in to comment.