From 506f044ec96bd9a6db7bc8e96f75730363bcf525 Mon Sep 17 00:00:00 2001 From: Mattias Jonsson Date: Fri, 20 Sep 2024 12:43:15 +0200 Subject: [PATCH 1/7] Also check column types in ALTER TABLE t PARTITION BY ... --- pkg/ddl/partition.go | 27 +++++++++++++++++++ .../r/ddl/reorg_partition.result | 11 ++++++++ .../t/ddl/reorg_partition.test | 12 +++++++++ 3 files changed, 50 insertions(+) diff --git a/pkg/ddl/partition.go b/pkg/ddl/partition.go index 0d908e2796717..8c8d35f8c924b 100644 --- a/pkg/ddl/partition.go +++ b/pkg/ddl/partition.go @@ -1833,6 +1833,26 @@ func checkPartitionFuncType(ctx sessionctx.Context, expr ast.ExprNode, schema st return errors.Trace(dbterror.ErrPartitionFuncNotAllowed.GenWithStackByArgs("PARTITION")) } +func checkPartitionFuncTypeExprString(ctx sessionctx.Context, expr, schema string, tblInfo *model.TableInfo) error { + if len(expr) == 0 { + return nil + } + if schema == "" { + schema = ctx.GetSessionVars().CurrentDB + } + e, err := expression.ParseSimpleExpr(ctx.GetExprCtx(), expr, expression.WithTableInfo(schema, tblInfo)) + if err != nil { + return errors.Trace(err) + } + if e.GetType(ctx.GetExprCtx().GetEvalCtx()).EvalType() == types.ETInt { + return nil + } + if col, ok := e.(*expression.Column); ok { + return errors.Trace(dbterror.ErrNotAllowedTypeInPartition.GenWithStackByArgs(col.OrigName)) + } + return errors.Trace(dbterror.ErrPartitionFuncNotAllowed.GenWithStackByArgs("PARTITION")) +} + // checkRangePartitionValue checks whether `less than value` is strictly increasing for each partition. // Side effect: it may simplify the partition range definition from a constant expression to an integer. func checkRangePartitionValue(ctx sessionctx.Context, tblInfo *model.TableInfo) error { @@ -3109,6 +3129,13 @@ 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 err = checkPartitionFuncTypeExprString(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 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; From e41112d4d5d46b7ded7524bdd8201f286fb688ec Mon Sep 17 00:00:00 2001 From: Mattias Jonsson Date: Tue, 24 Sep 2024 10:37:57 +0200 Subject: [PATCH 2/7] minor refactor/combining two functions to one --- pkg/ddl/partition.go | 39 ++++++++++++--------------------------- 1 file changed, 12 insertions(+), 27 deletions(-) diff --git a/pkg/ddl/partition.go b/pkg/ddl/partition.go index 8c8d35f8c924b..d4c70cdac74c0 100644 --- a/pkg/ddl/partition.go +++ b/pkg/ddl/partition.go @@ -1809,38 +1809,23 @@ 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)) - 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)) - } - - return errors.Trace(dbterror.ErrPartitionFuncNotAllowed.GenWithStackByArgs("PARTITION")) -} - -func checkPartitionFuncTypeExprString(ctx sessionctx.Context, expr, schema string, tblInfo *model.TableInfo) error { - if len(expr) == 0 { - return nil - } - if schema == "" { - schema = ctx.GetSessionVars().CurrentDB + var e expression.Expression + var err error + switch expr := anyExpr.(type) { + case *ast.ColumnNameExpr: + e, err = expression.BuildSimpleExpr(ctx.GetExprCtx(), expr, expression.WithTableInfo(schema, tblInfo)) + case string: + e, err = expression.ParseSimpleExpr(ctx.GetExprCtx(), expr, expression.WithTableInfo(schema, tblInfo)) + default: + return errors.Trace(dbterror.ErrPartitionFuncNotAllowed.GenWithStackByArgs("PARTITION")) } - e, err := expression.ParseSimpleExpr(ctx.GetExprCtx(), expr, expression.WithTableInfo(schema, tblInfo)) if err != nil { return errors.Trace(err) } @@ -3131,7 +3116,7 @@ func (w *worker) onReorganizePartition(jobCtx *jobContext, t *meta.Meta, job *mo if job.Type == model.ActionAlterTablePartitioning { // Also verify same things as in CREATE TABLE ... PARTITION BY - if err = checkPartitionFuncTypeExprString(sctx, partInfo.Expr, job.SchemaName, tblInfo); err != nil { + if err = checkPartitionFuncType(sctx, partInfo.Expr, job.SchemaName, tblInfo); err != nil { job.State = model.JobStateCancelled return ver, err } From 8b3876caaa0e722d801fa2b0828c732595c7a98e Mon Sep 17 00:00:00 2001 From: Mattias Jonsson Date: Tue, 24 Sep 2024 14:13:37 +0200 Subject: [PATCH 3/7] Fixed code, added more tests --- pkg/ddl/partition.go | 25 ++++++++--- pkg/ddl/reorg_partition_test.go | 74 +++++++++++++++++++++++++++++++++ 2 files changed, 94 insertions(+), 5 deletions(-) diff --git a/pkg/ddl/partition.go b/pkg/ddl/partition.go index d4c70cdac74c0..ab5e71c61bf69 100644 --- a/pkg/ddl/partition.go +++ b/pkg/ddl/partition.go @@ -1819,10 +1819,13 @@ func checkPartitionFuncType(ctx sessionctx.Context, anyExpr any, schema string, var e expression.Expression var err error switch expr := anyExpr.(type) { - case *ast.ColumnNameExpr: - e, err = expression.BuildSimpleExpr(ctx.GetExprCtx(), expr, expression.WithTableInfo(schema, tblInfo)) case string: + if expr == "" { + return errors.Trace(dbterror.ErrPartitionFuncNotAllowed.GenWithStackByArgs("PARTITION")) + } 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")) } @@ -1833,6 +1836,9 @@ func checkPartitionFuncType(ctx sessionctx.Context, anyExpr any, schema string, return nil } 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")) @@ -3116,9 +3122,18 @@ func (w *worker) onReorganizePartition(jobCtx *jobContext, t *meta.Meta, job *mo if job.Type == model.ActionAlterTablePartitioning { // Also verify same things as in CREATE TABLE ... PARTITION BY - if err = checkPartitionFuncType(sctx, partInfo.Expr, job.SchemaName, tblInfo); err != nil { - job.State = model.JobStateCancelled - return ver, err + if len(partInfo.Columns) > 0 { + for _, col := range partInfo.Columns { + if err = checkPartitionFuncType(sctx, col.O, job.SchemaName, tblInfo); 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. diff --git a/pkg/ddl/reorg_partition_test.go b/pkg/ddl/reorg_partition_test.go index 36c7cd16012b4..9b9f9747e1950 100644 --- a/pkg/ddl/reorg_partition_test.go +++ b/pkg/ddl/reorg_partition_test.go @@ -558,3 +558,77 @@ 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) + // KEY + tk.MustContainErrMsg(`create table kj `+cols+` partition by key(j) partitions 2`, "[ddl:1659]Field 'j' is of a not allowed type for this type of partitioning") + tk.MustContainErrMsg(`alter table t partition by key(j) partitions 2`, "[ddl:1659]Field 'j' is of a not allowed type for this type of partitioning") + // LIST + // LIST COLUMNS + tk.MustContainErrMsg(`create table cb `+cols+` partition by list columns (b) (partition pDef default)`, "[ddl:1659]Field 'b' is of a not allowed type for this type of partitioning") + tk.MustContainErrMsg(`create table cf `+cols+` partition by list columns (f) (partition pDef default)`, "[ddl:1659]Field 'f' is of a not allowed type for this type of partitioning") + tk.MustContainErrMsg(`create table cts `+cols+` partition by list columns (ts) (partition pDef default)`, "[ddl:1659]Field 'ts' is of a not allowed type for this type of partitioning") + tk.MustContainErrMsg(`create table cj `+cols+` partition by list columns (j) (partition pDef default)`, "[ddl:1659]Field 'j' is of a not allowed type for this type of partitioning") + tk.MustContainErrMsg(`alter table t partition by list columns (b) (partition pDef default)`, "[ddl:1659]Field 'b' is of a not allowed type for this type of partitioning") + tk.MustContainErrMsg(`alter table t partition by list columns (f) (partition pDef default)`, "[ddl:1659]Field 'f' is of a not allowed type for this type of partitioning") + tk.MustContainErrMsg(`alter table t partition by list columns (ts) (partition pDef default)`, "[ddl:1659]Field 'ts' is of a not allowed type for this type of partitioning") + tk.MustContainErrMsg(`alter table t partition by list columns (j) (partition pDef default)`, "[ddl:1659]Field 'j' is of a not allowed type for this type of partitioning") + // HASH + tk.MustContainErrMsg(`create table hts `+cols+` partition by hash(year(ts)) partitions 2`, "[ddl:1486]Constant, random or timezone-dependent expressions in (sub)partitioning function are not allowed") + tk.MustContainErrMsg(`alter table t partition by hash(year(ts)) partitions 2`, "[ddl:1486]Constant, random or timezone-dependent expressions in (sub)partitioning function are not allowed") + tk.MustContainErrMsg(`create table hd `+cols+` partition by hash(d) partitions 2`, "[ddl:1659]Field 'd' is of a not allowed type for this type of partitioning") + tk.MustContainErrMsg(`alter table t partition by hash(d) partitions 3`, "[ddl:1659]Field 'test.t.d' is of a not allowed type for this type of partitioning") + tk.MustContainErrMsg(`create table hts `+cols+` partition by hash(ts) partitions 2`, "[ddl:1659]Field 'ts' is of a not allowed type for this type of partitioning") + tk.MustContainErrMsg(`alter table t partition by hash(ts) partitions 3`, "[ddl:1659]Field 'test.t.ts' is of a not allowed type for this type of partitioning") + tk.MustContainErrMsg(`create table hts6 `+cols+` partition by hash(ts6) partitions 2`, "[ddl:1659]Field 'ts6' is of a not allowed type for this type of partitioning") + tk.MustContainErrMsg(`alter table t partition by hash(ts6) partitions 3`, "[ddl:1659]Field 'test.t.ts6' is of a not allowed type for this type of partitioning") + + // RANGE + tk.MustContainErrMsg(`create table tt `+cols+` partition by range (f) (partition pMax values less than (MAXVALUE))`, "[ddl:1659]Field 'f' is of a not allowed type for this type of partitioning") + tk.MustContainErrMsg(`alter table t partition by range(f) (partition pMax values less than (MAXVALUE))`, "[ddl:1659]Field 'test.t.f' is of a not allowed type for this type of partitioning") + tk.MustContainErrMsg(`create table tt `+cols+` partition by range (d) (partition pMax values less than (MAXVALUE))`, "[ddl:1659]Field 'd' is of a not allowed type for this type of partitioning") + tk.MustContainErrMsg(`alter table t partition by range(d) (partition pMax values less than (MAXVALUE))`, "[ddl:1659]Field 'test.t.d' is of a not allowed type for this type of partitioning") + tk.MustContainErrMsg(`create table tt `+cols+` partition by range (dt) (partition pMax values less than (MAXVALUE))`, "[ddl:1659]Field 'dt' is of a not allowed type for this type of partitioning") + tk.MustContainErrMsg(`alter table t partition by range(dt) (partition pMax values less than (MAXVALUE))`, "[ddl:1659]Field 'test.t.dt' is of a not allowed type for this type of partitioning") + tk.MustContainErrMsg(`create table tt `+cols+` partition by range (dt6) (partition pMax values less than (MAXVALUE))`, "[ddl:1659]Field 'dt6' is of a not allowed type for this type of partitioning") + tk.MustContainErrMsg(`alter table t partition by range(dt6) (partition pMax values less than (MAXVALUE))`, "[ddl:1659]Field 'test.t.dt6' is of a not allowed type for this type of partitioning") + tk.MustContainErrMsg(`create table tt `+cols+` partition by range (ts) (partition pMax values less than (MAXVALUE))`, "[ddl:1659]Field 'ts' is of a not allowed type for this type of partitioning") + tk.MustContainErrMsg(`alter table t partition by range(ts) (partition pMax values less than (MAXVALUE))`, "[ddl:1659]Field 'test.t.ts' is of a not allowed type for this type of partitioning") + tk.MustContainErrMsg(`create table tt `+cols+` partition by range (ts6) (partition pMax values less than (MAXVALUE))`, "[ddl:1659]Field 'ts6' is of a not allowed type for this type of partitioning") + tk.MustContainErrMsg(`alter table t partition by range(ts6) (partition pMax values less than (MAXVALUE))`, "[ddl:1659]Field 'test.t.ts6' is of a not allowed type for this type of partitioning") + + // 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) +} From 73a475600525f03e8bccf7675a02cd03a48f82cb Mon Sep 17 00:00:00 2001 From: Mattias Jonsson Date: Tue, 24 Sep 2024 16:04:11 +0200 Subject: [PATCH 4/7] Fixed code, updated tests --- pkg/ddl/partition.go | 10 ++-- pkg/ddl/reorg_partition_test.go | 84 +++++++++++++++++++-------------- 2 files changed, 53 insertions(+), 41 deletions(-) diff --git a/pkg/ddl/partition.go b/pkg/ddl/partition.go index ab5e71c61bf69..73209b609efa3 100644 --- a/pkg/ddl/partition.go +++ b/pkg/ddl/partition.go @@ -3123,11 +3123,11 @@ func (w *worker) onReorganizePartition(jobCtx *jobContext, t *meta.Meta, job *mo if job.Type == model.ActionAlterTablePartitioning { // Also verify same things as in CREATE TABLE ... PARTITION BY if len(partInfo.Columns) > 0 { - for _, col := range partInfo.Columns { - if err = checkPartitionFuncType(sctx, col.O, job.SchemaName, tblInfo); err != nil { - job.State = model.JobStateCancelled - return ver, err - } + // shallow copy, only for reading/checking + tmpTblInfo := *tblInfo + tmpTblInfo.Partition = partInfo + if err = checkColumnsPartitionType(&tmpTblInfo); err != nil { + return ver, err } } else { if err = checkPartitionFuncType(sctx, partInfo.Expr, job.SchemaName, tblInfo); err != nil { diff --git a/pkg/ddl/reorg_partition_test.go b/pkg/ddl/reorg_partition_test.go index 9b9f9747e1950..ea8656748a076 100644 --- a/pkg/ddl/reorg_partition_test.go +++ b/pkg/ddl/reorg_partition_test.go @@ -19,6 +19,7 @@ import ( "context" "encoding/hex" "fmt" + "github.com/pingcap/tidb/pkg/util/dbterror" "testing" "github.com/pingcap/failpoint" @@ -566,42 +567,53 @@ func TestPartitionByColumnChecks(t *testing.T) { 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) - // KEY - tk.MustContainErrMsg(`create table kj `+cols+` partition by key(j) partitions 2`, "[ddl:1659]Field 'j' is of a not allowed type for this type of partitioning") - tk.MustContainErrMsg(`alter table t partition by key(j) partitions 2`, "[ddl:1659]Field 'j' is of a not allowed type for this type of partitioning") - // LIST - // LIST COLUMNS - tk.MustContainErrMsg(`create table cb `+cols+` partition by list columns (b) (partition pDef default)`, "[ddl:1659]Field 'b' is of a not allowed type for this type of partitioning") - tk.MustContainErrMsg(`create table cf `+cols+` partition by list columns (f) (partition pDef default)`, "[ddl:1659]Field 'f' is of a not allowed type for this type of partitioning") - tk.MustContainErrMsg(`create table cts `+cols+` partition by list columns (ts) (partition pDef default)`, "[ddl:1659]Field 'ts' is of a not allowed type for this type of partitioning") - tk.MustContainErrMsg(`create table cj `+cols+` partition by list columns (j) (partition pDef default)`, "[ddl:1659]Field 'j' is of a not allowed type for this type of partitioning") - tk.MustContainErrMsg(`alter table t partition by list columns (b) (partition pDef default)`, "[ddl:1659]Field 'b' is of a not allowed type for this type of partitioning") - tk.MustContainErrMsg(`alter table t partition by list columns (f) (partition pDef default)`, "[ddl:1659]Field 'f' is of a not allowed type for this type of partitioning") - tk.MustContainErrMsg(`alter table t partition by list columns (ts) (partition pDef default)`, "[ddl:1659]Field 'ts' is of a not allowed type for this type of partitioning") - tk.MustContainErrMsg(`alter table t partition by list columns (j) (partition pDef default)`, "[ddl:1659]Field 'j' is of a not allowed type for this type of partitioning") - // HASH - tk.MustContainErrMsg(`create table hts `+cols+` partition by hash(year(ts)) partitions 2`, "[ddl:1486]Constant, random or timezone-dependent expressions in (sub)partitioning function are not allowed") - tk.MustContainErrMsg(`alter table t partition by hash(year(ts)) partitions 2`, "[ddl:1486]Constant, random or timezone-dependent expressions in (sub)partitioning function are not allowed") - tk.MustContainErrMsg(`create table hd `+cols+` partition by hash(d) partitions 2`, "[ddl:1659]Field 'd' is of a not allowed type for this type of partitioning") - tk.MustContainErrMsg(`alter table t partition by hash(d) partitions 3`, "[ddl:1659]Field 'test.t.d' is of a not allowed type for this type of partitioning") - tk.MustContainErrMsg(`create table hts `+cols+` partition by hash(ts) partitions 2`, "[ddl:1659]Field 'ts' is of a not allowed type for this type of partitioning") - tk.MustContainErrMsg(`alter table t partition by hash(ts) partitions 3`, "[ddl:1659]Field 'test.t.ts' is of a not allowed type for this type of partitioning") - tk.MustContainErrMsg(`create table hts6 `+cols+` partition by hash(ts6) partitions 2`, "[ddl:1659]Field 'ts6' is of a not allowed type for this type of partitioning") - tk.MustContainErrMsg(`alter table t partition by hash(ts6) partitions 3`, "[ddl:1659]Field 'test.t.ts6' is of a not allowed type for this type of partitioning") - - // RANGE - tk.MustContainErrMsg(`create table tt `+cols+` partition by range (f) (partition pMax values less than (MAXVALUE))`, "[ddl:1659]Field 'f' is of a not allowed type for this type of partitioning") - tk.MustContainErrMsg(`alter table t partition by range(f) (partition pMax values less than (MAXVALUE))`, "[ddl:1659]Field 'test.t.f' is of a not allowed type for this type of partitioning") - tk.MustContainErrMsg(`create table tt `+cols+` partition by range (d) (partition pMax values less than (MAXVALUE))`, "[ddl:1659]Field 'd' is of a not allowed type for this type of partitioning") - tk.MustContainErrMsg(`alter table t partition by range(d) (partition pMax values less than (MAXVALUE))`, "[ddl:1659]Field 'test.t.d' is of a not allowed type for this type of partitioning") - tk.MustContainErrMsg(`create table tt `+cols+` partition by range (dt) (partition pMax values less than (MAXVALUE))`, "[ddl:1659]Field 'dt' is of a not allowed type for this type of partitioning") - tk.MustContainErrMsg(`alter table t partition by range(dt) (partition pMax values less than (MAXVALUE))`, "[ddl:1659]Field 'test.t.dt' is of a not allowed type for this type of partitioning") - tk.MustContainErrMsg(`create table tt `+cols+` partition by range (dt6) (partition pMax values less than (MAXVALUE))`, "[ddl:1659]Field 'dt6' is of a not allowed type for this type of partitioning") - tk.MustContainErrMsg(`alter table t partition by range(dt6) (partition pMax values less than (MAXVALUE))`, "[ddl:1659]Field 'test.t.dt6' is of a not allowed type for this type of partitioning") - tk.MustContainErrMsg(`create table tt `+cols+` partition by range (ts) (partition pMax values less than (MAXVALUE))`, "[ddl:1659]Field 'ts' is of a not allowed type for this type of partitioning") - tk.MustContainErrMsg(`alter table t partition by range(ts) (partition pMax values less than (MAXVALUE))`, "[ddl:1659]Field 'test.t.ts' is of a not allowed type for this type of partitioning") - tk.MustContainErrMsg(`create table tt `+cols+` partition by range (ts6) (partition pMax values less than (MAXVALUE))`, "[ddl:1659]Field 'ts6' is of a not allowed type for this type of partitioning") - tk.MustContainErrMsg(`alter table t partition by range(ts6) (partition pMax values less than (MAXVALUE))`, "[ddl:1659]Field 'test.t.ts6' is of a not allowed type for this type of partitioning") + 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! From 876f2b23b554b5f882909c1407dcb4063d2e0d91 Mon Sep 17 00:00:00 2001 From: Mattias Jonsson Date: Tue, 24 Sep 2024 16:06:25 +0200 Subject: [PATCH 5/7] linting --- pkg/ddl/reorg_partition_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/ddl/reorg_partition_test.go b/pkg/ddl/reorg_partition_test.go index ea8656748a076..d386528c08c78 100644 --- a/pkg/ddl/reorg_partition_test.go +++ b/pkg/ddl/reorg_partition_test.go @@ -19,7 +19,6 @@ import ( "context" "encoding/hex" "fmt" - "github.com/pingcap/tidb/pkg/util/dbterror" "testing" "github.com/pingcap/failpoint" @@ -34,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" From cb634b03e9f8a8e0408bd0af37e256dc29668bcf Mon Sep 17 00:00:00 2001 From: Mattias Jonsson Date: Tue, 24 Sep 2024 16:22:18 +0200 Subject: [PATCH 6/7] Updated according to review comments --- pkg/ddl/partition.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/pkg/ddl/partition.go b/pkg/ddl/partition.go index 73209b609efa3..415f3a7568326 100644 --- a/pkg/ddl/partition.go +++ b/pkg/ddl/partition.go @@ -1821,7 +1821,7 @@ func checkPartitionFuncType(ctx sessionctx.Context, anyExpr any, schema string, switch expr := anyExpr.(type) { case string: if expr == "" { - return errors.Trace(dbterror.ErrPartitionFuncNotAllowed.GenWithStackByArgs("PARTITION")) + return nil } e, err = expression.ParseSimpleExpr(ctx.GetExprCtx(), expr, expression.WithTableInfo(schema, tblInfo)) case ast.ExprNode: @@ -3127,6 +3127,7 @@ func (w *worker) onReorganizePartition(jobCtx *jobContext, t *meta.Meta, job *mo tmpTblInfo := *tblInfo tmpTblInfo.Partition = partInfo if err = checkColumnsPartitionType(&tmpTblInfo); err != nil { + job.State = model.JobStateCancelled return ver, err } } else { From da68d2d08a1f59ae2329a1b6c47b62b8e1424ddb Mon Sep 17 00:00:00 2001 From: Mattias Jonsson Date: Wed, 25 Sep 2024 11:09:24 +0200 Subject: [PATCH 7/7] Stabilize TestShowPlacement, so it passes unit tests --- pkg/executor/show_placement_test.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pkg/executor/show_placement_test.go b/pkg/executor/show_placement_test.go index d6c7f4c10bcb7..bb0ffd4e75c09 100644 --- a/pkg/executor/show_placement_test.go +++ b/pkg/executor/show_placement_test.go @@ -87,11 +87,11 @@ func TestShowPlacement(t *testing.T) { tk.MustExec("create table db2.t2 (id int) PLACEMENT POLICY pa2") defer tk.MustExec("drop table if exists db2.t2") - tk.MustQuery("show placement").Check(testkit.Rows( + tk.MustQuery("show placement").Sort().Check(testkit.Rows( + "DATABASE db2 LEADER_CONSTRAINTS=\"[+region=us-east-1]\" FOLLOWERS=3 FOLLOWER_CONSTRAINTS=\"[+region=us-east-2]\" PENDING", "POLICY pa1 PRIMARY_REGION=\"cn-east-1\" REGIONS=\"cn-east-1,cn-east-2\" SCHEDULE=\"EVEN\" SURVIVAL_PREFERENCES=\"[zone, dc, host]\" NULL", "POLICY pa2 LEADER_CONSTRAINTS=\"[+region=us-east-1]\" FOLLOWERS=3 FOLLOWER_CONSTRAINTS=\"[+region=us-east-2]\" NULL", "POLICY pb1 CONSTRAINTS=\"[+disk=ssd]\" VOTERS=5 VOTER_CONSTRAINTS=\"[+region=bj]\" LEARNERS=3 LEARNER_CONSTRAINTS=\"[+region=sh]\" NULL", - "DATABASE db2 LEADER_CONSTRAINTS=\"[+region=us-east-1]\" FOLLOWERS=3 FOLLOWER_CONSTRAINTS=\"[+region=us-east-2]\" PENDING", "TABLE db2.t2 LEADER_CONSTRAINTS=\"[+region=us-east-1]\" FOLLOWERS=3 FOLLOWER_CONSTRAINTS=\"[+region=us-east-2]\" PENDING", "TABLE test.t1 PRIMARY_REGION=\"cn-east-1\" REGIONS=\"cn-east-1,cn-east-2\" SCHEDULE=\"EVEN\" SURVIVAL_PREFERENCES=\"[zone, dc, host]\" PENDING", "TABLE test.t3 PRIMARY_REGION=\"cn-east-1\" REGIONS=\"cn-east-1,cn-east-2\" SCHEDULE=\"EVEN\" SURVIVAL_PREFERENCES=\"[zone, dc, host]\" PENDING",