Skip to content

Commit 8c35443

Browse files
authored
ddl: fix issue of alter last partition failed when partition column is datetime (#48815) (#48848)
close #48814
1 parent 24476a1 commit 8c35443

File tree

3 files changed

+180
-9
lines changed

3 files changed

+180
-9
lines changed

pkg/ddl/partition.go

Lines changed: 31 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -768,12 +768,20 @@ func getPartitionIntervalFromTable(ctx sessionctx.Context, tbInfo *model.TableIn
768768
}
769769

770770
// comparePartitionAstAndModel compares a generated *ast.PartitionOptions and a *model.PartitionInfo
771-
func comparePartitionAstAndModel(ctx sessionctx.Context, pAst *ast.PartitionOptions, pModel *model.PartitionInfo) error {
771+
func comparePartitionAstAndModel(ctx sessionctx.Context, pAst *ast.PartitionOptions, pModel *model.PartitionInfo, partCol *model.ColumnInfo) error {
772772
a := pAst.Definitions
773773
m := pModel.Definitions
774774
if len(pAst.Definitions) != len(pModel.Definitions) {
775775
return dbterror.ErrGeneralUnsupportedDDL.GenWithStackByArgs("INTERVAL partitioning: number of partitions generated != partition defined (%d != %d)", len(a), len(m))
776776
}
777+
778+
evalFn := func(expr ast.ExprNode) (types.Datum, error) {
779+
val, err := expression.EvalAstExpr(ctx, ast.NewValueExpr(expr, "", ""))
780+
if err != nil || partCol == nil {
781+
return val, err
782+
}
783+
return val.ConvertTo(ctx.GetSessionVars().StmtCtx, &partCol.FieldType)
784+
}
777785
for i := range pAst.Definitions {
778786
// Allow options to differ! (like Placement Rules)
779787
// Allow names to differ!
@@ -796,16 +804,19 @@ func comparePartitionAstAndModel(ctx sessionctx.Context, pAst *ast.PartitionOpti
796804
if len(lessThan) > 1 && lessThan[:1] == "'" && lessThan[len(lessThan)-1:] == "'" {
797805
lessThan = driver.UnwrapFromSingleQuotes(lessThan)
798806
}
799-
cmpExpr := &ast.BinaryOperationExpr{
800-
Op: opcode.EQ,
801-
L: ast.NewValueExpr(lessThan, "", ""),
802-
R: generatedExpr,
807+
lessThanVal, err := evalFn(ast.NewValueExpr(lessThan, "", ""))
808+
if err != nil {
809+
return err
803810
}
804-
cmp, err := expression.EvalAstExpr(ctx, cmpExpr)
811+
generatedExprVal, err := evalFn(generatedExpr)
805812
if err != nil {
806813
return err
807814
}
808-
if cmp.GetInt64() != 1 {
815+
cmp, err := lessThanVal.Compare(ctx.GetSessionVars().StmtCtx, &generatedExprVal, collate.GetBinaryCollator())
816+
if err != nil {
817+
return err
818+
}
819+
if cmp != 0 {
809820
return dbterror.ErrGeneralUnsupportedDDL.GenWithStackByArgs(fmt.Sprintf("INTERVAL partitioning: LESS THAN for partition %s differs between generated and defined", m[i].Name.O))
810821
}
811822
}
@@ -980,7 +991,7 @@ func generatePartitionDefinitionsFromInterval(ctx sessionctx.Context, partOption
980991
// Seems valid, so keep the defined so that the user defined names are kept etc.
981992
partOptions.Definitions = definedPartDefs
982993
} else if len(tbInfo.Partition.Definitions) > 0 {
983-
err := comparePartitionAstAndModel(ctx, partOptions, tbInfo.Partition)
994+
err := comparePartitionAstAndModel(ctx, partOptions, tbInfo.Partition, partCol)
984995
if err != nil {
985996
return err
986997
}
@@ -1054,6 +1065,12 @@ func GeneratePartDefsFromInterval(ctx sessionctx.Context, tp ast.AlterTableType,
10541065
if err != nil {
10551066
return err
10561067
}
1068+
if partCol != nil {
1069+
lastVal, err = lastVal.ConvertTo(ctx.GetSessionVars().StmtCtx, &partCol.FieldType)
1070+
if err != nil {
1071+
return err
1072+
}
1073+
}
10571074
var partDefs []*ast.PartitionDefinition
10581075
if len(partitionOptions.Definitions) != 0 {
10591076
partDefs = partitionOptions.Definitions
@@ -1097,6 +1114,12 @@ func GeneratePartDefsFromInterval(ctx sessionctx.Context, tp ast.AlterTableType,
10971114
if err != nil {
10981115
return err
10991116
}
1117+
if partCol != nil {
1118+
currVal, err = currVal.ConvertTo(ctx.GetSessionVars().StmtCtx, &partCol.FieldType)
1119+
if err != nil {
1120+
return err
1121+
}
1122+
}
11001123
cmp, err := currVal.Compare(ctx.GetSessionVars().StmtCtx, &lastVal, collate.GetBinaryCollator())
11011124
if err != nil {
11021125
return err

pkg/ddl/tests/partition/BUILD.bazel

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -8,7 +8,7 @@ go_test(
88
"main_test.go",
99
],
1010
flaky = True,
11-
shard_count = 47,
11+
shard_count = 48,
1212
deps = [
1313
"//pkg/config",
1414
"//pkg/ddl",

pkg/ddl/tests/partition/db_partition_test.go

Lines changed: 148 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -3645,4 +3645,152 @@ func TestRemovePartitioningAutoIDs(t *testing.T) {
36453645
"32 31 10", "35 34 21", "38 37 22", "41 40 23"))
36463646
}
36473647

3648+
func TestAlterLastIntervalPartition(t *testing.T) {
3649+
store := testkit.CreateMockStore(t)
3650+
tk := testkit.NewTestKit(t, store)
3651+
tk.MustExec(`use test`)
3652+
tk.MustExec(`create table t (id int, create_time datetime)
3653+
partition by range columns (create_time)
3654+
interval (1 day)
3655+
first partition less than ('2023-01-01')
3656+
last partition less than ('2023-01-03');`)
3657+
ctx := tk.Session()
3658+
tbl, err := domain.GetDomain(ctx).InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t"))
3659+
require.NoError(t, err)
3660+
pd := tbl.Meta().Partition.Definitions
3661+
require.Equal(t, 3, len(pd))
3662+
require.Equal(t, "'2023-01-01 00:00:00'", pd[0].LessThan[0])
3663+
require.Equal(t, "'2023-01-02 00:00:00'", pd[1].LessThan[0])
3664+
require.Equal(t, "'2023-01-03 00:00:00'", pd[2].LessThan[0])
3665+
tk.MustExec("alter table t last partition less than ('2024-01-04')")
3666+
tk.MustExec("alter table t last partition less than ('2025-01-01 00:00:00')")
3667+
tbl, err = domain.GetDomain(ctx).InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t"))
3668+
require.NoError(t, err)
3669+
pd = tbl.Meta().Partition.Definitions
3670+
require.Equal(t, 732, len(pd))
3671+
require.Equal(t, "'2023-01-01 00:00:00'", pd[0].LessThan[0])
3672+
require.Equal(t, "'2023-01-02 00:00:00'", pd[1].LessThan[0])
3673+
require.Equal(t, "'2023-01-03 00:00:00'", pd[2].LessThan[0])
3674+
require.Equal(t, "'2024-12-31 00:00:00'", pd[730].LessThan[0])
3675+
require.Equal(t, "'2025-01-01 00:00:00'", pd[731].LessThan[0])
3676+
3677+
// Test for interval 2 days.
3678+
tk.MustExec(`create table t2 (id int, create_time datetime)
3679+
partition by range columns (create_time)
3680+
interval (2 day)
3681+
first partition less than ('2023-01-01')
3682+
last partition less than ('2023-01-05');`)
3683+
tbl, err = domain.GetDomain(ctx).InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t2"))
3684+
require.NoError(t, err)
3685+
pd = tbl.Meta().Partition.Definitions
3686+
require.Equal(t, 3, len(pd))
3687+
require.Equal(t, "'2023-01-01 00:00:00'", pd[0].LessThan[0])
3688+
require.Equal(t, "'2023-01-03 00:00:00'", pd[1].LessThan[0])
3689+
require.Equal(t, "'2023-01-05 00:00:00'", pd[2].LessThan[0])
3690+
tk.MustExec("alter table t2 last partition less than ('2023-01-09')")
3691+
tk.MustExec("alter table t2 last partition less than ('2023-01-11 00:00:00')")
3692+
tbl, err = domain.GetDomain(ctx).InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t2"))
3693+
require.NoError(t, err)
3694+
pd = tbl.Meta().Partition.Definitions
3695+
require.Equal(t, 6, len(pd))
3696+
require.Equal(t, "'2023-01-01 00:00:00'", pd[0].LessThan[0])
3697+
require.Equal(t, "'2023-01-03 00:00:00'", pd[1].LessThan[0])
3698+
require.Equal(t, "'2023-01-05 00:00:00'", pd[2].LessThan[0])
3699+
require.Equal(t, "'2023-01-07 00:00:00'", pd[3].LessThan[0])
3700+
require.Equal(t, "'2023-01-09 00:00:00'", pd[4].LessThan[0])
3701+
require.Equal(t, "'2023-01-11 00:00:00'", pd[5].LessThan[0])
3702+
3703+
// Test for day with time.
3704+
tk.MustExec(`create table t3 (id int, create_time datetime)
3705+
partition by range columns (create_time)
3706+
interval (2 day)
3707+
first partition less than ('2023-01-01 12:01:02')
3708+
last partition less than ('2023-01-05 12:01:02');`)
3709+
tbl, err = domain.GetDomain(ctx).InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t3"))
3710+
require.NoError(t, err)
3711+
pd = tbl.Meta().Partition.Definitions
3712+
require.Equal(t, 3, len(pd))
3713+
require.Equal(t, "'2023-01-01 12:01:02'", pd[0].LessThan[0])
3714+
require.Equal(t, "'2023-01-03 12:01:02'", pd[1].LessThan[0])
3715+
require.Equal(t, "'2023-01-05 12:01:02'", pd[2].LessThan[0])
3716+
tk.MustExec("alter table t3 last partition less than ('2023-01-09 12:01:02')")
3717+
tbl, err = domain.GetDomain(ctx).InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t3"))
3718+
require.NoError(t, err)
3719+
pd = tbl.Meta().Partition.Definitions
3720+
require.Equal(t, 5, len(pd))
3721+
require.Equal(t, "'2023-01-01 12:01:02'", pd[0].LessThan[0])
3722+
require.Equal(t, "'2023-01-03 12:01:02'", pd[1].LessThan[0])
3723+
require.Equal(t, "'2023-01-05 12:01:02'", pd[2].LessThan[0])
3724+
require.Equal(t, "'2023-01-07 12:01:02'", pd[3].LessThan[0])
3725+
require.Equal(t, "'2023-01-09 12:01:02'", pd[4].LessThan[0])
3726+
3727+
// Some other test.
3728+
tk.MustExec(`create table t4 (id int, create_time datetime)
3729+
partition by range columns (create_time)
3730+
interval (48 hour)
3731+
first partition less than ('2023-01-01')
3732+
last partition less than ('2023-01-05');`)
3733+
tbl, err = domain.GetDomain(ctx).InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t4"))
3734+
require.NoError(t, err)
3735+
pd = tbl.Meta().Partition.Definitions
3736+
require.Equal(t, 3, len(pd))
3737+
require.Equal(t, "'2023-01-01 00:00:00'", pd[0].LessThan[0])
3738+
require.Equal(t, "'2023-01-03 00:00:00'", pd[1].LessThan[0])
3739+
require.Equal(t, "'2023-01-05 00:00:00'", pd[2].LessThan[0])
3740+
tk.MustExec("alter table t4 last partition less than ('2023-01-09 00:00:00')")
3741+
tbl, err = domain.GetDomain(ctx).InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t4"))
3742+
require.NoError(t, err)
3743+
pd = tbl.Meta().Partition.Definitions
3744+
require.Equal(t, 5, len(pd))
3745+
require.Equal(t, "'2023-01-01 00:00:00'", pd[0].LessThan[0])
3746+
require.Equal(t, "'2023-01-03 00:00:00'", pd[1].LessThan[0])
3747+
require.Equal(t, "'2023-01-05 00:00:00'", pd[2].LessThan[0])
3748+
require.Equal(t, "'2023-01-07 00:00:00'", pd[3].LessThan[0])
3749+
require.Equal(t, "'2023-01-09 00:00:00'", pd[4].LessThan[0])
3750+
tk.MustQuery("show create table t4").Check(testkit.Rows("t4 CREATE TABLE `t4` (\n" +
3751+
" `id` int(11) DEFAULT NULL,\n" +
3752+
" `create_time` datetime DEFAULT NULL\n" +
3753+
") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin\n" +
3754+
"PARTITION BY RANGE COLUMNS(`create_time`)\n" +
3755+
"(PARTITION `P_LT_2023-01-01 00:00:00` VALUES LESS THAN ('2023-01-01 00:00:00'),\n" +
3756+
" PARTITION `P_LT_2023-01-03 00:00:00` VALUES LESS THAN ('2023-01-03 00:00:00'),\n" +
3757+
" PARTITION `P_LT_2023-01-05 00:00:00` VALUES LESS THAN ('2023-01-05 00:00:00'),\n" +
3758+
" PARTITION `P_LT_2023-01-07 00:00:00` VALUES LESS THAN ('2023-01-07 00:00:00'),\n" +
3759+
" PARTITION `P_LT_2023-01-09 00:00:00` VALUES LESS THAN ('2023-01-09 00:00:00'))"))
3760+
3761+
tk.MustExec(`create table t5 (id int, create_time datetime)
3762+
partition by range columns (create_time)
3763+
interval (1 month)
3764+
first partition less than ('2023-01-01')
3765+
last partition less than ('2023-05-01');`)
3766+
tk.MustQuery("show create table t5").Check(testkit.Rows("t5 CREATE TABLE `t5` (\n" +
3767+
" `id` int(11) DEFAULT NULL,\n" +
3768+
" `create_time` datetime DEFAULT NULL\n" +
3769+
") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin\n" +
3770+
"PARTITION BY RANGE COLUMNS(`create_time`)\n" +
3771+
"(PARTITION `P_LT_2023-01-01 00:00:00` VALUES LESS THAN ('2023-01-01 00:00:00'),\n" +
3772+
" PARTITION `P_LT_2023-02-01 00:00:00` VALUES LESS THAN ('2023-02-01 00:00:00'),\n" +
3773+
" PARTITION `P_LT_2023-03-01 00:00:00` VALUES LESS THAN ('2023-03-01 00:00:00'),\n" +
3774+
" PARTITION `P_LT_2023-04-01 00:00:00` VALUES LESS THAN ('2023-04-01 00:00:00'),\n" +
3775+
" PARTITION `P_LT_2023-05-01 00:00:00` VALUES LESS THAN ('2023-05-01 00:00:00'))"))
3776+
3777+
tk.MustExec("CREATE TABLE `t6` (\n" +
3778+
" `id` int(11) DEFAULT NULL,\n" +
3779+
" `create_time` datetime DEFAULT NULL\n" +
3780+
") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin\n" +
3781+
"PARTITION BY RANGE COLUMNS(`create_time`)\n" +
3782+
"(PARTITION `P_LT_2023-01-01` VALUES LESS THAN ('2023-01-01'),\n" +
3783+
" PARTITION `P_LT_2023-01-02` VALUES LESS THAN ('2023-01-02'))")
3784+
tk.MustExec("alter table t6 last partition less than ('2023-01-04')")
3785+
tk.MustQuery("show create table t6").Check(testkit.Rows("t6 CREATE TABLE `t6` (\n" +
3786+
" `id` int(11) DEFAULT NULL,\n" +
3787+
" `create_time` datetime DEFAULT NULL\n" +
3788+
") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin\n" +
3789+
"PARTITION BY RANGE COLUMNS(`create_time`)\n" +
3790+
"(PARTITION `P_LT_2023-01-01` VALUES LESS THAN ('2023-01-01'),\n" +
3791+
" PARTITION `P_LT_2023-01-02` VALUES LESS THAN ('2023-01-02'),\n" +
3792+
" PARTITION `P_LT_2023-01-03 00:00:00` VALUES LESS THAN ('2023-01-03 00:00:00'),\n" +
3793+
" PARTITION `P_LT_2023-01-04 00:00:00` VALUES LESS THAN ('2023-01-04 00:00:00'))"))
3794+
}
3795+
36483796
// TODO: check EXCHANGE how it handles null (for all types of partitioning!!!)

0 commit comments

Comments
 (0)