Skip to content

Commit

Permalink
ddl: fix issue of alter last partition failed when partition column i…
Browse files Browse the repository at this point in the history
…s datetime (pingcap#48815)

Signed-off-by: crazycs520 <crazycs520@gmail.com>
  • Loading branch information
crazycs520 committed Nov 24, 2023
1 parent 7d3792d commit db5e907
Show file tree
Hide file tree
Showing 2 changed files with 179 additions and 8 deletions.
148 changes: 148 additions & 0 deletions ddl/db_partition_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -5365,4 +5365,152 @@ func TestRangeExchangeValidate(t *testing.T) {
// TODO: add test not in first partition (both last without maxvalue and also not last with/without maxvalue)
}

func TestAlterLastIntervalPartition(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec(`use test`)
tk.MustExec(`create table t (id int, create_time datetime)
partition by range columns (create_time)
interval (1 day)
first partition less than ('2023-01-01')
last partition less than ('2023-01-03');`)
ctx := tk.Session()
tbl, err := domain.GetDomain(ctx).InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t"))
require.NoError(t, err)
pd := tbl.Meta().Partition.Definitions
require.Equal(t, 3, len(pd))
require.Equal(t, "'2023-01-01 00:00:00'", pd[0].LessThan[0])
require.Equal(t, "'2023-01-02 00:00:00'", pd[1].LessThan[0])
require.Equal(t, "'2023-01-03 00:00:00'", pd[2].LessThan[0])
tk.MustExec("alter table t last partition less than ('2024-01-04')")
tk.MustExec("alter table t last partition less than ('2025-01-01 00:00:00')")
tbl, err = domain.GetDomain(ctx).InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t"))
require.NoError(t, err)
pd = tbl.Meta().Partition.Definitions
require.Equal(t, 732, len(pd))
require.Equal(t, "'2023-01-01 00:00:00'", pd[0].LessThan[0])
require.Equal(t, "'2023-01-02 00:00:00'", pd[1].LessThan[0])
require.Equal(t, "'2023-01-03 00:00:00'", pd[2].LessThan[0])
require.Equal(t, "'2024-12-31 00:00:00'", pd[730].LessThan[0])
require.Equal(t, "'2025-01-01 00:00:00'", pd[731].LessThan[0])

// Test for interval 2 days.
tk.MustExec(`create table t2 (id int, create_time datetime)
partition by range columns (create_time)
interval (2 day)
first partition less than ('2023-01-01')
last partition less than ('2023-01-05');`)
tbl, err = domain.GetDomain(ctx).InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t2"))
require.NoError(t, err)
pd = tbl.Meta().Partition.Definitions
require.Equal(t, 3, len(pd))
require.Equal(t, "'2023-01-01 00:00:00'", pd[0].LessThan[0])
require.Equal(t, "'2023-01-03 00:00:00'", pd[1].LessThan[0])
require.Equal(t, "'2023-01-05 00:00:00'", pd[2].LessThan[0])
tk.MustExec("alter table t2 last partition less than ('2023-01-09')")
tk.MustExec("alter table t2 last partition less than ('2023-01-11 00:00:00')")
tbl, err = domain.GetDomain(ctx).InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t2"))
require.NoError(t, err)
pd = tbl.Meta().Partition.Definitions
require.Equal(t, 6, len(pd))
require.Equal(t, "'2023-01-01 00:00:00'", pd[0].LessThan[0])
require.Equal(t, "'2023-01-03 00:00:00'", pd[1].LessThan[0])
require.Equal(t, "'2023-01-05 00:00:00'", pd[2].LessThan[0])
require.Equal(t, "'2023-01-07 00:00:00'", pd[3].LessThan[0])
require.Equal(t, "'2023-01-09 00:00:00'", pd[4].LessThan[0])
require.Equal(t, "'2023-01-11 00:00:00'", pd[5].LessThan[0])

// Test for day with time.
tk.MustExec(`create table t3 (id int, create_time datetime)
partition by range columns (create_time)
interval (2 day)
first partition less than ('2023-01-01 12:01:02')
last partition less than ('2023-01-05 12:01:02');`)
tbl, err = domain.GetDomain(ctx).InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t3"))
require.NoError(t, err)
pd = tbl.Meta().Partition.Definitions
require.Equal(t, 3, len(pd))
require.Equal(t, "'2023-01-01 12:01:02'", pd[0].LessThan[0])
require.Equal(t, "'2023-01-03 12:01:02'", pd[1].LessThan[0])
require.Equal(t, "'2023-01-05 12:01:02'", pd[2].LessThan[0])
tk.MustExec("alter table t3 last partition less than ('2023-01-09 12:01:02')")
tbl, err = domain.GetDomain(ctx).InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t3"))
require.NoError(t, err)
pd = tbl.Meta().Partition.Definitions
require.Equal(t, 5, len(pd))
require.Equal(t, "'2023-01-01 12:01:02'", pd[0].LessThan[0])
require.Equal(t, "'2023-01-03 12:01:02'", pd[1].LessThan[0])
require.Equal(t, "'2023-01-05 12:01:02'", pd[2].LessThan[0])
require.Equal(t, "'2023-01-07 12:01:02'", pd[3].LessThan[0])
require.Equal(t, "'2023-01-09 12:01:02'", pd[4].LessThan[0])

// Some other test.
tk.MustExec(`create table t4 (id int, create_time datetime)
partition by range columns (create_time)
interval (48 hour)
first partition less than ('2023-01-01')
last partition less than ('2023-01-05');`)
tbl, err = domain.GetDomain(ctx).InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t4"))
require.NoError(t, err)
pd = tbl.Meta().Partition.Definitions
require.Equal(t, 3, len(pd))
require.Equal(t, "'2023-01-01 00:00:00'", pd[0].LessThan[0])
require.Equal(t, "'2023-01-03 00:00:00'", pd[1].LessThan[0])
require.Equal(t, "'2023-01-05 00:00:00'", pd[2].LessThan[0])
tk.MustExec("alter table t4 last partition less than ('2023-01-09 00:00:00')")
tbl, err = domain.GetDomain(ctx).InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t4"))
require.NoError(t, err)
pd = tbl.Meta().Partition.Definitions
require.Equal(t, 5, len(pd))
require.Equal(t, "'2023-01-01 00:00:00'", pd[0].LessThan[0])
require.Equal(t, "'2023-01-03 00:00:00'", pd[1].LessThan[0])
require.Equal(t, "'2023-01-05 00:00:00'", pd[2].LessThan[0])
require.Equal(t, "'2023-01-07 00:00:00'", pd[3].LessThan[0])
require.Equal(t, "'2023-01-09 00:00:00'", pd[4].LessThan[0])
tk.MustQuery("show create table t4").Check(testkit.Rows("t4 CREATE TABLE `t4` (\n" +
" `id` int(11) DEFAULT NULL,\n" +
" `create_time` datetime DEFAULT NULL\n" +
") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin\n" +
"PARTITION BY RANGE COLUMNS(`create_time`)\n" +
"(PARTITION `P_LT_2023-01-01 00:00:00` VALUES LESS THAN ('2023-01-01 00:00:00'),\n" +
" PARTITION `P_LT_2023-01-03 00:00:00` VALUES LESS THAN ('2023-01-03 00:00:00'),\n" +
" PARTITION `P_LT_2023-01-05 00:00:00` VALUES LESS THAN ('2023-01-05 00:00:00'),\n" +
" PARTITION `P_LT_2023-01-07 00:00:00` VALUES LESS THAN ('2023-01-07 00:00:00'),\n" +
" PARTITION `P_LT_2023-01-09 00:00:00` VALUES LESS THAN ('2023-01-09 00:00:00'))"))

tk.MustExec(`create table t5 (id int, create_time datetime)
partition by range columns (create_time)
interval (1 month)
first partition less than ('2023-01-01')
last partition less than ('2023-05-01');`)
tk.MustQuery("show create table t5").Check(testkit.Rows("t5 CREATE TABLE `t5` (\n" +
" `id` int(11) DEFAULT NULL,\n" +
" `create_time` datetime DEFAULT NULL\n" +
") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin\n" +
"PARTITION BY RANGE COLUMNS(`create_time`)\n" +
"(PARTITION `P_LT_2023-01-01 00:00:00` VALUES LESS THAN ('2023-01-01 00:00:00'),\n" +
" PARTITION `P_LT_2023-02-01 00:00:00` VALUES LESS THAN ('2023-02-01 00:00:00'),\n" +
" PARTITION `P_LT_2023-03-01 00:00:00` VALUES LESS THAN ('2023-03-01 00:00:00'),\n" +
" PARTITION `P_LT_2023-04-01 00:00:00` VALUES LESS THAN ('2023-04-01 00:00:00'),\n" +
" PARTITION `P_LT_2023-05-01 00:00:00` VALUES LESS THAN ('2023-05-01 00:00:00'))"))

tk.MustExec("CREATE TABLE `t6` (\n" +
" `id` int(11) DEFAULT NULL,\n" +
" `create_time` datetime DEFAULT NULL\n" +
") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin\n" +
"PARTITION BY RANGE COLUMNS(`create_time`)\n" +
"(PARTITION `P_LT_2023-01-01` VALUES LESS THAN ('2023-01-01'),\n" +
" PARTITION `P_LT_2023-01-02` VALUES LESS THAN ('2023-01-02'))")
tk.MustExec("alter table t6 last partition less than ('2023-01-04')")
tk.MustQuery("show create table t6").Check(testkit.Rows("t6 CREATE TABLE `t6` (\n" +
" `id` int(11) DEFAULT NULL,\n" +
" `create_time` datetime DEFAULT NULL\n" +
") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin\n" +
"PARTITION BY RANGE COLUMNS(`create_time`)\n" +
"(PARTITION `P_LT_2023-01-01` VALUES LESS THAN ('2023-01-01'),\n" +
" PARTITION `P_LT_2023-01-02` VALUES LESS THAN ('2023-01-02'),\n" +
" PARTITION `P_LT_2023-01-03 00:00:00` VALUES LESS THAN ('2023-01-03 00:00:00'),\n" +
" PARTITION `P_LT_2023-01-04 00:00:00` VALUES LESS THAN ('2023-01-04 00:00:00'))"))
}

// TODO: check EXCHANGE how it handles null (for all types of partitioning!!!)
39 changes: 31 additions & 8 deletions ddl/partition.go
Original file line number Diff line number Diff line change
Expand Up @@ -711,12 +711,20 @@ func getPartitionIntervalFromTable(ctx sessionctx.Context, tbInfo *model.TableIn
}

// comparePartitionAstAndModel compares a generated *ast.PartitionOptions and a *model.PartitionInfo
func comparePartitionAstAndModel(ctx sessionctx.Context, pAst *ast.PartitionOptions, pModel *model.PartitionInfo) error {
func comparePartitionAstAndModel(ctx sessionctx.Context, pAst *ast.PartitionOptions, pModel *model.PartitionInfo, partCol *model.ColumnInfo) error {
a := pAst.Definitions
m := pModel.Definitions
if len(pAst.Definitions) != len(pModel.Definitions) {
return dbterror.ErrGeneralUnsupportedDDL.GenWithStackByArgs("INTERVAL partitioning: number of partitions generated != partition defined (%d != %d)", len(a), len(m))
}

evalFn := func(expr ast.ExprNode) (types.Datum, error) {
val, err := expression.EvalAstExpr(ctx, ast.NewValueExpr(expr, "", ""))
if err != nil || partCol == nil {
return val, err
}
return val.ConvertTo(ctx.GetSessionVars().StmtCtx, &partCol.FieldType)
}
for i := range pAst.Definitions {
// Allow options to differ! (like Placement Rules)
// Allow names to differ!
Expand All @@ -739,16 +747,19 @@ func comparePartitionAstAndModel(ctx sessionctx.Context, pAst *ast.PartitionOpti
if len(lessThan) > 1 && lessThan[:1] == "'" && lessThan[len(lessThan)-1:] == "'" {
lessThan = driver.UnwrapFromSingleQuotes(lessThan)
}
cmpExpr := &ast.BinaryOperationExpr{
Op: opcode.EQ,
L: ast.NewValueExpr(lessThan, "", ""),
R: generatedExpr,
lessThanVal, err := evalFn(ast.NewValueExpr(lessThan, "", ""))
if err != nil {
return err
}
cmp, err := expression.EvalAstExpr(ctx, cmpExpr)
generatedExprVal, err := evalFn(generatedExpr)
if err != nil {
return err
}
if cmp.GetInt64() != 1 {
cmp, err := lessThanVal.Compare(ctx.GetSessionVars().StmtCtx, &generatedExprVal, collate.GetBinaryCollator())
if err != nil {
return err
}
if cmp != 0 {
return dbterror.ErrGeneralUnsupportedDDL.GenWithStackByArgs(fmt.Sprintf("INTERVAL partitioning: LESS THAN for partition %s differs between generated and defined", m[i].Name.O))
}
}
Expand Down Expand Up @@ -923,7 +934,7 @@ func generatePartitionDefinitionsFromInterval(ctx sessionctx.Context, partOption
// Seems valid, so keep the defined so that the user defined names are kept etc.
partOptions.Definitions = definedPartDefs
} else if len(tbInfo.Partition.Definitions) > 0 {
err := comparePartitionAstAndModel(ctx, partOptions, tbInfo.Partition)
err := comparePartitionAstAndModel(ctx, partOptions, tbInfo.Partition, partCol)
if err != nil {
return err
}
Expand Down Expand Up @@ -997,6 +1008,12 @@ func GeneratePartDefsFromInterval(ctx sessionctx.Context, tp ast.AlterTableType,
if err != nil {
return err
}
if partCol != nil {
lastVal, err = lastVal.ConvertTo(ctx.GetSessionVars().StmtCtx, &partCol.FieldType)
if err != nil {
return err
}
}
var partDefs []*ast.PartitionDefinition
if len(partitionOptions.Definitions) != 0 {
partDefs = partitionOptions.Definitions
Expand Down Expand Up @@ -1040,6 +1057,12 @@ func GeneratePartDefsFromInterval(ctx sessionctx.Context, tp ast.AlterTableType,
if err != nil {
return err
}
if partCol != nil {
currVal, err = currVal.ConvertTo(ctx.GetSessionVars().StmtCtx, &partCol.FieldType)
if err != nil {
return err
}
}
cmp, err := currVal.Compare(ctx.GetSessionVars().StmtCtx, &lastVal, collate.GetBinaryCollator())
if err != nil {
return err
Expand Down

0 comments on commit db5e907

Please sign in to comment.