Skip to content

Commit

Permalink
ttl: add ttl_job_interval attribute and remove `tidb_ttl_job_interv…
Browse files Browse the repository at this point in the history
…al` variable (#40033)

close #40028
  • Loading branch information
YangKeao authored Jan 6, 2023
1 parent 43ebc64 commit b226dfc
Show file tree
Hide file tree
Showing 32 changed files with 9,956 additions and 9,583 deletions.
2 changes: 2 additions & 0 deletions ddl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -71,6 +71,7 @@ go_library(
"//parser",
"//parser/ast",
"//parser/charset",
"//parser/duration",
"//parser/format",
"//parser/model",
"//parser/mysql",
Expand Down Expand Up @@ -222,6 +223,7 @@ go_test(
"//parser/ast",
"//parser/auth",
"//parser/charset",
"//parser/duration",
"//parser/model",
"//parser/mysql",
"//parser/terror",
Expand Down
73 changes: 39 additions & 34 deletions ddl/ddl_api.go
Original file line number Diff line number Diff line change
Expand Up @@ -41,6 +41,7 @@ import (
"github.com/pingcap/tidb/parser"
"github.com/pingcap/tidb/parser/ast"
"github.com/pingcap/tidb/parser/charset"
"github.com/pingcap/tidb/parser/duration"
"github.com/pingcap/tidb/parser/format"
"github.com/pingcap/tidb/parser/model"
"github.com/pingcap/tidb/parser/mysql"
Expand Down Expand Up @@ -3031,7 +3032,7 @@ func SetDirectPlacementOpt(placementSettings *model.PlacementSettings, placement

// handleTableOptions updates tableInfo according to table options.
func handleTableOptions(options []*ast.TableOption, tbInfo *model.TableInfo) error {
var handledTTLOrTTLEnable bool
var ttlOptionsHandled bool

for _, op := range options {
switch op.Tp {
Expand Down Expand Up @@ -3069,23 +3070,28 @@ func handleTableOptions(options []*ast.TableOption, tbInfo *model.TableInfo) err
tbInfo.PlacementPolicyRef = &model.PolicyRefInfo{
Name: model.NewCIStr(op.StrValue),
}
case ast.TableOptionTTL, ast.TableOptionTTLEnable:
if handledTTLOrTTLEnable {
case ast.TableOptionTTL, ast.TableOptionTTLEnable, ast.TableOptionTTLJobInterval:
if ttlOptionsHandled {
continue
}

ttlInfo, ttlEnable, err := getTTLInfoInOptions(options)
ttlInfo, ttlEnable, ttlJobInterval, err := getTTLInfoInOptions(options)
if err != nil {
return err
}
// It's impossible that `ttlInfo` and `ttlEnable` are all nil, because we have met this option.
// After exclude the situation `ttlInfo == nil && ttlEnable != nil`, we could say `ttlInfo != nil`
if ttlInfo == nil && ttlEnable != nil {
return errors.Trace(dbterror.ErrSetTTLEnableForNonTTLTable)
if ttlInfo == nil {
if ttlEnable != nil {
return errors.Trace(dbterror.ErrSetTTLOptionForNonTTLTable.FastGenByArgs("TTL_ENABLE"))
}
if ttlJobInterval != nil {
return errors.Trace(dbterror.ErrSetTTLOptionForNonTTLTable.FastGenByArgs("TTL_JOB_INTERVAL"))
}
}

tbInfo.TTLInfo = ttlInfo
handledTTLOrTTLEnable = true
ttlOptionsHandled = true
}
}
shardingBits := shardingBits(tbInfo)
Expand Down Expand Up @@ -3285,7 +3291,7 @@ func (d *ddl) AlterTable(ctx context.Context, sctx sessionctx.Context, stmt *ast
}
for _, spec := range validSpecs {
var handledCharsetOrCollate bool
var handledTTLOrTTLEnable bool
var ttlOptionsHandled bool
switch spec.Tp {
case ast.AlterTableAddColumns:
err = d.AddColumn(sctx, ident, spec)
Expand Down Expand Up @@ -3422,20 +3428,21 @@ func (d *ddl) AlterTable(ctx context.Context, sctx sessionctx.Context, stmt *ast
Name: model.NewCIStr(opt.StrValue),
}
case ast.TableOptionEngine:
case ast.TableOptionTTL, ast.TableOptionTTLEnable:
case ast.TableOptionTTL, ast.TableOptionTTLEnable, ast.TableOptionTTLJobInterval:
var ttlInfo *model.TTLInfo
var ttlEnable *bool
var ttlJobInterval *duration.Duration

if handledTTLOrTTLEnable {
if ttlOptionsHandled {
continue
}
ttlInfo, ttlEnable, err = getTTLInfoInOptions(spec.Options)
ttlInfo, ttlEnable, ttlJobInterval, err = getTTLInfoInOptions(spec.Options)
if err != nil {
return err
}
err = d.AlterTableTTLInfoOrEnable(sctx, ident, ttlInfo, ttlEnable)
err = d.AlterTableTTLInfoOrEnable(sctx, ident, ttlInfo, ttlEnable, ttlJobInterval)

handledTTLOrTTLEnable = true
ttlOptionsHandled = true
default:
err = dbterror.ErrUnsupportedAlterTableOption
}
Expand Down Expand Up @@ -5397,11 +5404,13 @@ func (d *ddl) AlterTableSetTiFlashReplica(ctx sessionctx.Context, ident ast.Iden
}

// AlterTableTTLInfoOrEnable submit ddl job to change table info according to the ttlInfo, or ttlEnable
// at least one of the `ttlInfo` or `ttlEnable` should be not nil.
// at least one of the `ttlInfo`, `ttlEnable` or `ttlCronJobSchedule` should be not nil.
// When `ttlInfo` is nil, and `ttlEnable` is not, it will use the original `.TTLInfo` in the table info and modify the
// `.Enable`. If the `.TTLInfo` in the table info is empty, this function will return an error.
// When `ttlInfo` is nil, and `ttlCronJobSchedule` is not, it will use the original `.TTLInfo` in the table info and modify the
// `.JobInterval`. If the `.TTLInfo` in the table info is empty, this function will return an error.
// When `ttlInfo` is not nil, it simply submits the job with the `ttlInfo` and ignore the `ttlEnable`.
func (d *ddl) AlterTableTTLInfoOrEnable(ctx sessionctx.Context, ident ast.Ident, ttlInfo *model.TTLInfo, ttlEnable *bool) error {
func (d *ddl) AlterTableTTLInfoOrEnable(ctx sessionctx.Context, ident ast.Ident, ttlInfo *model.TTLInfo, ttlEnable *bool, ttlCronJobSchedule *duration.Duration) error {
is := d.infoCache.GetLatest()
schema, ok := is.SchemaByName(ident.Schema)
if !ok {
Expand All @@ -5424,29 +5433,25 @@ func (d *ddl) AlterTableTTLInfoOrEnable(ctx sessionctx.Context, ident ast.Ident,
if err != nil {
return err
}
job = &model.Job{
SchemaID: schema.ID,
TableID: tableID,
SchemaName: schema.Name.L,
TableName: tableName,
Type: model.ActionAlterTTLInfo,
BinlogInfo: &model.HistoryInfo{},
Args: []interface{}{ttlInfo, ttlEnable},
}
} else {
if tblInfo.TTLInfo == nil {
return errors.Trace(dbterror.ErrSetTTLEnableForNonTTLTable)
if ttlEnable != nil {
return errors.Trace(dbterror.ErrSetTTLOptionForNonTTLTable.FastGenByArgs("TTL_ENABLE"))
}
if ttlCronJobSchedule != nil {
return errors.Trace(dbterror.ErrSetTTLOptionForNonTTLTable.FastGenByArgs("TTL_JOB_INTERVAL"))
}
}
}

job = &model.Job{
SchemaID: schema.ID,
TableID: tableID,
SchemaName: schema.Name.L,
TableName: tableName,
Type: model.ActionAlterTTLInfo,
BinlogInfo: &model.HistoryInfo{},
Args: []interface{}{ttlInfo, ttlEnable},
}
job = &model.Job{
SchemaID: schema.ID,
TableID: tableID,
SchemaName: schema.Name.L,
TableName: tableName,
Type: model.ActionAlterTTLInfo,
BinlogInfo: &model.HistoryInfo{},
Args: []interface{}{ttlInfo, ttlEnable, ttlCronJobSchedule},
}

err = d.DoDDLJob(ctx, job)
Expand Down
10 changes: 5 additions & 5 deletions ddl/serial_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -501,29 +501,29 @@ func TestRecoverTableWithTTL(t *testing.T) {
tk.MustExec("create table t_recover1 (t timestamp) TTL=`t`+INTERVAL 1 DAY")
tk.MustExec("drop table t_recover1")
tk.MustExec("recover table t_recover1")
tk.MustQuery("show create table t_recover1").Check(testkit.Rows("t_recover1 CREATE TABLE `t_recover1` (\n `t` timestamp NULL DEFAULT NULL\n) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin /*T![ttl] TTL=`t` + INTERVAL 1 DAY */ /*T![ttl] TTL_ENABLE='OFF' */"))
tk.MustQuery("show create table t_recover1").Check(testkit.Rows("t_recover1 CREATE TABLE `t_recover1` (\n `t` timestamp NULL DEFAULT NULL\n) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin /*T![ttl] TTL=`t` + INTERVAL 1 DAY */ /*T![ttl] TTL_ENABLE='OFF' */ /*T![ttl] TTL_JOB_INTERVAL='1h' */"))

// recover table with job id
tk.MustExec("create table t_recover2 (t timestamp) TTL=`t`+INTERVAL 1 DAY")
tk.MustExec("drop table t_recover2")
jobID := getDDLJobID("t_recover2", "drop table")
tk.MustExec(fmt.Sprintf("recover table BY JOB %d", jobID))
tk.MustQuery("show create table t_recover2").Check(testkit.Rows("t_recover2 CREATE TABLE `t_recover2` (\n `t` timestamp NULL DEFAULT NULL\n) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin /*T![ttl] TTL=`t` + INTERVAL 1 DAY */ /*T![ttl] TTL_ENABLE='OFF' */"))
tk.MustQuery("show create table t_recover2").Check(testkit.Rows("t_recover2 CREATE TABLE `t_recover2` (\n `t` timestamp NULL DEFAULT NULL\n) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin /*T![ttl] TTL=`t` + INTERVAL 1 DAY */ /*T![ttl] TTL_ENABLE='OFF' */ /*T![ttl] TTL_JOB_INTERVAL='1h' */"))

// flashback table
tk.MustExec("create table t_recover3 (t timestamp) TTL=`t`+INTERVAL 1 DAY")
tk.MustExec("drop table t_recover3")
tk.MustExec("flashback table t_recover3")
tk.MustQuery("show create table t_recover3").Check(testkit.Rows("t_recover3 CREATE TABLE `t_recover3` (\n `t` timestamp NULL DEFAULT NULL\n) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin /*T![ttl] TTL=`t` + INTERVAL 1 DAY */ /*T![ttl] TTL_ENABLE='OFF' */"))
tk.MustQuery("show create table t_recover3").Check(testkit.Rows("t_recover3 CREATE TABLE `t_recover3` (\n `t` timestamp NULL DEFAULT NULL\n) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin /*T![ttl] TTL=`t` + INTERVAL 1 DAY */ /*T![ttl] TTL_ENABLE='OFF' */ /*T![ttl] TTL_JOB_INTERVAL='1h' */"))

// flashback database
tk.MustExec("create database if not exists test_recover2")
tk.MustExec("create table test_recover2.t1 (t timestamp) TTL=`t`+INTERVAL 1 DAY")
tk.MustExec("create table test_recover2.t2 (t timestamp) TTL=`t`+INTERVAL 1 DAY")
tk.MustExec("drop database test_recover2")
tk.MustExec("flashback database test_recover2")
tk.MustQuery("show create table test_recover2.t1").Check(testkit.Rows("t1 CREATE TABLE `t1` (\n `t` timestamp NULL DEFAULT NULL\n) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin /*T![ttl] TTL=`t` + INTERVAL 1 DAY */ /*T![ttl] TTL_ENABLE='OFF' */"))
tk.MustQuery("show create table test_recover2.t2").Check(testkit.Rows("t2 CREATE TABLE `t2` (\n `t` timestamp NULL DEFAULT NULL\n) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin /*T![ttl] TTL=`t` + INTERVAL 1 DAY */ /*T![ttl] TTL_ENABLE='OFF' */"))
tk.MustQuery("show create table test_recover2.t1").Check(testkit.Rows("t1 CREATE TABLE `t1` (\n `t` timestamp NULL DEFAULT NULL\n) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin /*T![ttl] TTL=`t` + INTERVAL 1 DAY */ /*T![ttl] TTL_ENABLE='OFF' */ /*T![ttl] TTL_JOB_INTERVAL='1h' */"))
tk.MustQuery("show create table test_recover2.t2").Check(testkit.Rows("t2 CREATE TABLE `t2` (\n `t` timestamp NULL DEFAULT NULL\n) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin /*T![ttl] TTL=`t` + INTERVAL 1 DAY */ /*T![ttl] TTL_ENABLE='OFF' */ /*T![ttl] TTL_JOB_INTERVAL='1h' */"))
}

func TestRecoverTableByJobID(t *testing.T) {
Expand Down
44 changes: 35 additions & 9 deletions ddl/ttl.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ import (
"github.com/pingcap/tidb/meta"
"github.com/pingcap/tidb/parser"
"github.com/pingcap/tidb/parser/ast"
"github.com/pingcap/tidb/parser/duration"
"github.com/pingcap/tidb/parser/format"
"github.com/pingcap/tidb/parser/model"
"github.com/pingcap/tidb/sessionctx"
Expand Down Expand Up @@ -50,8 +51,9 @@ func onTTLInfoChange(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, err er
// at least one for them is not nil
var ttlInfo *model.TTLInfo
var ttlInfoEnable *bool
var ttlInfoJobInterval *duration.Duration

if err := job.DecodeArgs(&ttlInfo, &ttlInfoEnable); err != nil {
if err := job.DecodeArgs(&ttlInfo, &ttlInfoEnable, &ttlInfoJobInterval); err != nil {
job.State = model.JobStateCancelled
return ver, errors.Trace(err)
}
Expand All @@ -66,15 +68,25 @@ func onTTLInfoChange(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, err er
if ttlInfoEnable == nil && tblInfo.TTLInfo != nil {
ttlInfo.Enable = tblInfo.TTLInfo.Enable
}
if ttlInfoJobInterval == nil && tblInfo.TTLInfo != nil {
ttlInfo.JobInterval = tblInfo.TTLInfo.JobInterval
}
tblInfo.TTLInfo = ttlInfo
}
if ttlInfoEnable != nil {
if tblInfo.TTLInfo == nil {
return ver, errors.Trace(dbterror.ErrSetTTLEnableForNonTTLTable)
return ver, errors.Trace(dbterror.ErrSetTTLOptionForNonTTLTable.FastGenByArgs("TTL_ENABLE"))
}

tblInfo.TTLInfo.Enable = *ttlInfoEnable
}
if ttlInfoJobInterval != nil {
if tblInfo.TTLInfo == nil {
return ver, errors.Trace(dbterror.ErrSetTTLOptionForNonTTLTable.FastGenByArgs("TTL_JOB_INTERVAL"))
}

tblInfo.TTLInfo.JobInterval = *ttlInfoJobInterval
}

ver, err = updateVersionAndTableInfo(d, t, job, tblInfo, true)
if err != nil {
Expand Down Expand Up @@ -151,9 +163,10 @@ func checkDropColumnWithTTLConfig(tblInfo *model.TableInfo, colName string) erro
}

// getTTLInfoInOptions returns the aggregated ttlInfo, the ttlEnable, or an error.
// if TTL or TTL_ENABLE is not set in the config, the corresponding return value will be nil.
// if both of them are set, the `ttlInfo.Enable` will be equal with `ttlEnable`.
func getTTLInfoInOptions(options []*ast.TableOption) (ttlInfo *model.TTLInfo, ttlEnable *bool, err error) {
// if TTL, TTL_ENABLE or TTL_JOB_INTERVAL is not set in the config, the corresponding return value will be nil.
// if both of TTL and TTL_ENABLE are set, the `ttlInfo.Enable` will be equal with `ttlEnable`.
// if both of TTL and TTL_JOB_INTERVAL are set, the `ttlInfo.JobInterval` will be equal with `ttlCronJobSchedule`.
func getTTLInfoInOptions(options []*ast.TableOption) (ttlInfo *model.TTLInfo, ttlEnable *bool, ttlCronJobSchedule *duration.Duration, err error) {
for _, op := range options {
switch op.Tp {
case ast.TableOptionTTL:
Expand All @@ -162,7 +175,7 @@ func getTTLInfoInOptions(options []*ast.TableOption) (ttlInfo *model.TTLInfo, tt
restoreCtx := format.NewRestoreCtx(restoreFlags, &sb)
err := op.Value.Restore(restoreCtx)
if err != nil {
return nil, nil, err
return nil, nil, nil, err
}

intervalExpr := sb.String()
Expand All @@ -171,14 +184,27 @@ func getTTLInfoInOptions(options []*ast.TableOption) (ttlInfo *model.TTLInfo, tt
IntervalExprStr: intervalExpr,
IntervalTimeUnit: int(op.TimeUnitValue.Unit),
Enable: true,
JobInterval: duration.Duration{Hour: 1},
}
case ast.TableOptionTTLEnable:
ttlEnable = &op.BoolValue
case ast.TableOptionTTLJobInterval:
schedule, err := duration.ParseDuration(op.StrValue)
if err != nil {
// this branch is actually unreachable, as the value has been validated in parser
return nil, nil, nil, err
}
ttlCronJobSchedule = &schedule
}
}

if ttlInfo != nil && ttlEnable != nil {
ttlInfo.Enable = *ttlEnable
if ttlInfo != nil {
if ttlEnable != nil {
ttlInfo.Enable = *ttlEnable
}
if ttlCronJobSchedule != nil {
ttlInfo.JobInterval = *ttlCronJobSchedule
}
}
return ttlInfo, ttlEnable, nil
return ttlInfo, ttlEnable, ttlCronJobSchedule, nil
}
Loading

0 comments on commit b226dfc

Please sign in to comment.