diff --git a/pkg/ddl/create_table.go b/pkg/ddl/create_table.go index 043f051251f8e..42a01fb41e121 100644 --- a/pkg/ddl/create_table.go +++ b/pkg/ddl/create_table.go @@ -434,7 +434,7 @@ func checkTableInfoValidWithStmt(ctx *metabuild.Context, tbInfo *model.TableInfo return errors.Trace(err) } if s.Partition != nil { - if err := checkPartitionFuncType(ctx, s.Partition.Expr, s.Table.Schema.O, tbInfo); err != nil { + if err := checkPartitionFuncType(ctx.GetExprCtx(), s.Partition.Expr, s.Table.Schema.O, tbInfo); err != nil { return errors.Trace(err) } if err := checkPartitioningKeysConstraints(ctx, s, tbInfo); err != nil { diff --git a/pkg/ddl/partition.go b/pkg/ddl/partition.go index 238f6bd890991..9ea3dfadcad9f 100644 --- a/pkg/ddl/partition.go +++ b/pkg/ddl/partition.go @@ -1788,29 +1788,35 @@ func checkResultOK(ok bool) error { } // checkPartitionFuncType checks partition function return type. -func checkPartitionFuncType(ctx *metabuild.Context, expr ast.ExprNode, schema string, tblInfo *model.TableInfo) error { - if expr == nil { +func checkPartitionFuncType(ctx expression.BuildContext, anyExpr any, schema string, tblInfo *model.TableInfo) error { + if anyExpr == nil { return nil } - - exprCtx := ctx.GetExprCtx() - evalCtx := exprCtx.GetEvalCtx() - if schema == "" { - schema = evalCtx.CurrentDB() + var e expression.Expression + var err error + switch expr := anyExpr.(type) { + case string: + if expr == "" { + return nil + } + e, err = expression.ParseSimpleExpr(ctx, expr, expression.WithTableInfo(schema, tblInfo)) + case ast.ExprNode: + e, err = expression.BuildSimpleExpr(ctx, expr, expression.WithTableInfo(schema, tblInfo)) + default: + return errors.Trace(dbterror.ErrPartitionFuncNotAllowed.GenWithStackByArgs("PARTITION")) } - - e, err := expression.BuildSimpleExpr(exprCtx, expr, expression.WithTableInfo(schema, tblInfo)) if err != nil { return errors.Trace(err) } - if e.GetType(evalCtx).EvalType() == types.ETInt { + if e.GetType(ctx.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")) } @@ -3095,6 +3101,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.GetExprCtx(), 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 diff --git a/pkg/ddl/reorg_partition_test.go b/pkg/ddl/reorg_partition_test.go index 36c7cd16012b4..d386528c08c78 100644 --- a/pkg/ddl/reorg_partition_test.go +++ b/pkg/ddl/reorg_partition_test.go @@ -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" @@ -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) +} diff --git a/tests/integrationtest/r/ddl/reorg_partition.result b/tests/integrationtest/r/ddl/reorg_partition.result index f2cff44392a07..2f3bbad1059ad 100644 --- a/tests/integrationtest/r/ddl/reorg_partition.result +++ b/tests/integrationtest/r/ddl/reorg_partition.result @@ -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; diff --git a/tests/integrationtest/t/ddl/reorg_partition.test b/tests/integrationtest/t/ddl/reorg_partition.test index 8a546f0a787d7..e041209e6622b 100644 --- a/tests/integrationtest/t/ddl/reorg_partition.test +++ b/tests/integrationtest/t/ddl/reorg_partition.test @@ -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;