Skip to content

Commit 3c5ea36

Browse files
YangKeaoti-chi-bot
authored andcommitted
This is an automated cherry-pick of pingcap#40033
Signed-off-by: ti-chi-bot <[email protected]>
1 parent f700b56 commit 3c5ea36

32 files changed

+18854
-4903
lines changed

ddl/BUILD.bazel

+2
Original file line numberDiff line numberDiff line change
@@ -71,6 +71,7 @@ go_library(
7171
"//parser",
7272
"//parser/ast",
7373
"//parser/charset",
74+
"//parser/duration",
7475
"//parser/format",
7576
"//parser/model",
7677
"//parser/mysql",
@@ -220,6 +221,7 @@ go_test(
220221
"//parser/ast",
221222
"//parser/auth",
222223
"//parser/charset",
224+
"//parser/duration",
223225
"//parser/model",
224226
"//parser/mysql",
225227
"//parser/terror",

ddl/ddl_api.go

+39-34
Original file line numberDiff line numberDiff line change
@@ -41,6 +41,7 @@ import (
4141
"github.com/pingcap/tidb/parser"
4242
"github.com/pingcap/tidb/parser/ast"
4343
"github.com/pingcap/tidb/parser/charset"
44+
"github.com/pingcap/tidb/parser/duration"
4445
"github.com/pingcap/tidb/parser/format"
4546
"github.com/pingcap/tidb/parser/model"
4647
"github.com/pingcap/tidb/parser/mysql"
@@ -3024,7 +3025,7 @@ func SetDirectPlacementOpt(placementSettings *model.PlacementSettings, placement
30243025

30253026
// handleTableOptions updates tableInfo according to table options.
30263027
func handleTableOptions(options []*ast.TableOption, tbInfo *model.TableInfo) error {
3027-
var handledTTLOrTTLEnable bool
3028+
var ttlOptionsHandled bool
30283029

30293030
for _, op := range options {
30303031
switch op.Tp {
@@ -3062,23 +3063,28 @@ func handleTableOptions(options []*ast.TableOption, tbInfo *model.TableInfo) err
30623063
tbInfo.PlacementPolicyRef = &model.PolicyRefInfo{
30633064
Name: model.NewCIStr(op.StrValue),
30643065
}
3065-
case ast.TableOptionTTL, ast.TableOptionTTLEnable:
3066-
if handledTTLOrTTLEnable {
3066+
case ast.TableOptionTTL, ast.TableOptionTTLEnable, ast.TableOptionTTLJobInterval:
3067+
if ttlOptionsHandled {
30673068
continue
30683069
}
30693070

3070-
ttlInfo, ttlEnable, err := getTTLInfoInOptions(options)
3071+
ttlInfo, ttlEnable, ttlJobInterval, err := getTTLInfoInOptions(options)
30713072
if err != nil {
30723073
return err
30733074
}
30743075
// It's impossible that `ttlInfo` and `ttlEnable` are all nil, because we have met this option.
30753076
// After exclude the situation `ttlInfo == nil && ttlEnable != nil`, we could say `ttlInfo != nil`
3076-
if ttlInfo == nil && ttlEnable != nil {
3077-
return errors.Trace(dbterror.ErrSetTTLEnableForNonTTLTable)
3077+
if ttlInfo == nil {
3078+
if ttlEnable != nil {
3079+
return errors.Trace(dbterror.ErrSetTTLOptionForNonTTLTable.FastGenByArgs("TTL_ENABLE"))
3080+
}
3081+
if ttlJobInterval != nil {
3082+
return errors.Trace(dbterror.ErrSetTTLOptionForNonTTLTable.FastGenByArgs("TTL_JOB_INTERVAL"))
3083+
}
30783084
}
30793085

30803086
tbInfo.TTLInfo = ttlInfo
3081-
handledTTLOrTTLEnable = true
3087+
ttlOptionsHandled = true
30823088
}
30833089
}
30843090
shardingBits := shardingBits(tbInfo)
@@ -3270,7 +3276,7 @@ func (d *ddl) AlterTable(ctx context.Context, sctx sessionctx.Context, stmt *ast
32703276
}
32713277
for _, spec := range validSpecs {
32723278
var handledCharsetOrCollate bool
3273-
var handledTTLOrTTLEnable bool
3279+
var ttlOptionsHandled bool
32743280
switch spec.Tp {
32753281
case ast.AlterTableAddColumns:
32763282
err = d.AddColumn(sctx, ident, spec)
@@ -3407,20 +3413,21 @@ func (d *ddl) AlterTable(ctx context.Context, sctx sessionctx.Context, stmt *ast
34073413
Name: model.NewCIStr(opt.StrValue),
34083414
}
34093415
case ast.TableOptionEngine:
3410-
case ast.TableOptionTTL, ast.TableOptionTTLEnable:
3416+
case ast.TableOptionTTL, ast.TableOptionTTLEnable, ast.TableOptionTTLJobInterval:
34113417
var ttlInfo *model.TTLInfo
34123418
var ttlEnable *bool
3419+
var ttlJobInterval *duration.Duration
34133420

3414-
if handledTTLOrTTLEnable {
3421+
if ttlOptionsHandled {
34153422
continue
34163423
}
3417-
ttlInfo, ttlEnable, err = getTTLInfoInOptions(spec.Options)
3424+
ttlInfo, ttlEnable, ttlJobInterval, err = getTTLInfoInOptions(spec.Options)
34183425
if err != nil {
34193426
return err
34203427
}
3421-
err = d.AlterTableTTLInfoOrEnable(sctx, ident, ttlInfo, ttlEnable)
3428+
err = d.AlterTableTTLInfoOrEnable(sctx, ident, ttlInfo, ttlEnable, ttlJobInterval)
34223429

3423-
handledTTLOrTTLEnable = true
3430+
ttlOptionsHandled = true
34243431
default:
34253432
err = dbterror.ErrUnsupportedAlterTableOption
34263433
}
@@ -5348,11 +5355,13 @@ func (d *ddl) AlterTableSetTiFlashReplica(ctx sessionctx.Context, ident ast.Iden
53485355
}
53495356

53505357
// AlterTableTTLInfoOrEnable submit ddl job to change table info according to the ttlInfo, or ttlEnable
5351-
// at least one of the `ttlInfo` or `ttlEnable` should be not nil.
5358+
// at least one of the `ttlInfo`, `ttlEnable` or `ttlCronJobSchedule` should be not nil.
53525359
// When `ttlInfo` is nil, and `ttlEnable` is not, it will use the original `.TTLInfo` in the table info and modify the
53535360
// `.Enable`. If the `.TTLInfo` in the table info is empty, this function will return an error.
5361+
// When `ttlInfo` is nil, and `ttlCronJobSchedule` is not, it will use the original `.TTLInfo` in the table info and modify the
5362+
// `.JobInterval`. If the `.TTLInfo` in the table info is empty, this function will return an error.
53545363
// When `ttlInfo` is not nil, it simply submits the job with the `ttlInfo` and ignore the `ttlEnable`.
5355-
func (d *ddl) AlterTableTTLInfoOrEnable(ctx sessionctx.Context, ident ast.Ident, ttlInfo *model.TTLInfo, ttlEnable *bool) error {
5364+
func (d *ddl) AlterTableTTLInfoOrEnable(ctx sessionctx.Context, ident ast.Ident, ttlInfo *model.TTLInfo, ttlEnable *bool, ttlCronJobSchedule *duration.Duration) error {
53565365
is := d.infoCache.GetLatest()
53575366
schema, ok := is.SchemaByName(ident.Schema)
53585367
if !ok {
@@ -5375,29 +5384,25 @@ func (d *ddl) AlterTableTTLInfoOrEnable(ctx sessionctx.Context, ident ast.Ident,
53755384
if err != nil {
53765385
return err
53775386
}
5378-
job = &model.Job{
5379-
SchemaID: schema.ID,
5380-
TableID: tableID,
5381-
SchemaName: schema.Name.L,
5382-
TableName: tableName,
5383-
Type: model.ActionAlterTTLInfo,
5384-
BinlogInfo: &model.HistoryInfo{},
5385-
Args: []interface{}{ttlInfo, ttlEnable},
5386-
}
53875387
} else {
53885388
if tblInfo.TTLInfo == nil {
5389-
return errors.Trace(dbterror.ErrSetTTLEnableForNonTTLTable)
5389+
if ttlEnable != nil {
5390+
return errors.Trace(dbterror.ErrSetTTLOptionForNonTTLTable.FastGenByArgs("TTL_ENABLE"))
5391+
}
5392+
if ttlCronJobSchedule != nil {
5393+
return errors.Trace(dbterror.ErrSetTTLOptionForNonTTLTable.FastGenByArgs("TTL_JOB_INTERVAL"))
5394+
}
53905395
}
5396+
}
53915397

5392-
job = &model.Job{
5393-
SchemaID: schema.ID,
5394-
TableID: tableID,
5395-
SchemaName: schema.Name.L,
5396-
TableName: tableName,
5397-
Type: model.ActionAlterTTLInfo,
5398-
BinlogInfo: &model.HistoryInfo{},
5399-
Args: []interface{}{ttlInfo, ttlEnable},
5400-
}
5398+
job = &model.Job{
5399+
SchemaID: schema.ID,
5400+
TableID: tableID,
5401+
SchemaName: schema.Name.L,
5402+
TableName: tableName,
5403+
Type: model.ActionAlterTTLInfo,
5404+
BinlogInfo: &model.HistoryInfo{},
5405+
Args: []interface{}{ttlInfo, ttlEnable, ttlCronJobSchedule},
54015406
}
54025407

54035408
err = d.DoDDLJob(ctx, job)

ddl/serial_test.go

+67
Original file line numberDiff line numberDiff line change
@@ -461,6 +461,73 @@ func TestCancelAddIndexPanic(t *testing.T) {
461461
require.Truef(t, strings.HasPrefix(errMsg, "[ddl:8214]Cancelled DDL job"), "%v", errMsg)
462462
}
463463

464+
<<<<<<< HEAD
465+
=======
466+
func TestRecoverTableWithTTL(t *testing.T) {
467+
store, _ := createMockStoreAndDomain(t)
468+
tk := testkit.NewTestKit(t, store)
469+
tk.MustExec("create database if not exists test_recover")
470+
tk.MustExec("use test_recover")
471+
defer func(originGC bool) {
472+
if originGC {
473+
util.EmulatorGCEnable()
474+
} else {
475+
util.EmulatorGCDisable()
476+
}
477+
}(util.IsEmulatorGCEnable())
478+
479+
// disable emulator GC.
480+
// Otherwise emulator GC will delete table record as soon as possible after execute drop table ddl.
481+
util.EmulatorGCDisable()
482+
gcTimeFormat := "20060102-15:04:05 -0700 MST"
483+
safePointSQL := `INSERT HIGH_PRIORITY INTO mysql.tidb VALUES ('tikv_gc_safe_point', '%[1]s', '')
484+
ON DUPLICATE KEY
485+
UPDATE variable_value = '%[1]s'`
486+
tk.MustExec(fmt.Sprintf(safePointSQL, time.Now().Add(-time.Hour).Format(gcTimeFormat)))
487+
getDDLJobID := func(table, tp string) int64 {
488+
rs, err := tk.Exec("admin show ddl jobs")
489+
require.NoError(t, err)
490+
rows, err := session.GetRows4Test(context.Background(), tk.Session(), rs)
491+
require.NoError(t, err)
492+
for _, row := range rows {
493+
if row.GetString(2) == table && row.GetString(3) == tp {
494+
return row.GetInt64(0)
495+
}
496+
}
497+
require.FailNowf(t, "can't find %s table of %s", tp, table)
498+
return -1
499+
}
500+
501+
// recover table
502+
tk.MustExec("create table t_recover1 (t timestamp) TTL=`t`+INTERVAL 1 DAY")
503+
tk.MustExec("drop table t_recover1")
504+
tk.MustExec("recover table t_recover1")
505+
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' */"))
506+
507+
// recover table with job id
508+
tk.MustExec("create table t_recover2 (t timestamp) TTL=`t`+INTERVAL 1 DAY")
509+
tk.MustExec("drop table t_recover2")
510+
jobID := getDDLJobID("t_recover2", "drop table")
511+
tk.MustExec(fmt.Sprintf("recover table BY JOB %d", jobID))
512+
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' */"))
513+
514+
// flashback table
515+
tk.MustExec("create table t_recover3 (t timestamp) TTL=`t`+INTERVAL 1 DAY")
516+
tk.MustExec("drop table t_recover3")
517+
tk.MustExec("flashback table t_recover3")
518+
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' */"))
519+
520+
// flashback database
521+
tk.MustExec("create database if not exists test_recover2")
522+
tk.MustExec("create table test_recover2.t1 (t timestamp) TTL=`t`+INTERVAL 1 DAY")
523+
tk.MustExec("create table test_recover2.t2 (t timestamp) TTL=`t`+INTERVAL 1 DAY")
524+
tk.MustExec("drop database test_recover2")
525+
tk.MustExec("flashback database test_recover2")
526+
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' */"))
527+
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' */"))
528+
}
529+
530+
>>>>>>> b226dfcb05 (ttl: add `ttl_job_interval` attribute and remove `tidb_ttl_job_interval` variable (#40033))
464531
func TestRecoverTableByJobID(t *testing.T) {
465532
store, _ := createMockStoreAndDomain(t)
466533
tk := testkit.NewTestKit(t, store)

ddl/ttl.go

+35-9
Original file line numberDiff line numberDiff line change
@@ -23,6 +23,7 @@ import (
2323
"github.com/pingcap/tidb/meta"
2424
"github.com/pingcap/tidb/parser"
2525
"github.com/pingcap/tidb/parser/ast"
26+
"github.com/pingcap/tidb/parser/duration"
2627
"github.com/pingcap/tidb/parser/format"
2728
"github.com/pingcap/tidb/parser/model"
2829
"github.com/pingcap/tidb/sessionctx"
@@ -50,8 +51,9 @@ func onTTLInfoChange(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, err er
5051
// at least one for them is not nil
5152
var ttlInfo *model.TTLInfo
5253
var ttlInfoEnable *bool
54+
var ttlInfoJobInterval *duration.Duration
5355

54-
if err := job.DecodeArgs(&ttlInfo, &ttlInfoEnable); err != nil {
56+
if err := job.DecodeArgs(&ttlInfo, &ttlInfoEnable, &ttlInfoJobInterval); err != nil {
5557
job.State = model.JobStateCancelled
5658
return ver, errors.Trace(err)
5759
}
@@ -66,15 +68,25 @@ func onTTLInfoChange(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, err er
6668
if ttlInfoEnable == nil && tblInfo.TTLInfo != nil {
6769
ttlInfo.Enable = tblInfo.TTLInfo.Enable
6870
}
71+
if ttlInfoJobInterval == nil && tblInfo.TTLInfo != nil {
72+
ttlInfo.JobInterval = tblInfo.TTLInfo.JobInterval
73+
}
6974
tblInfo.TTLInfo = ttlInfo
7075
}
7176
if ttlInfoEnable != nil {
7277
if tblInfo.TTLInfo == nil {
73-
return ver, errors.Trace(dbterror.ErrSetTTLEnableForNonTTLTable)
78+
return ver, errors.Trace(dbterror.ErrSetTTLOptionForNonTTLTable.FastGenByArgs("TTL_ENABLE"))
7479
}
7580

7681
tblInfo.TTLInfo.Enable = *ttlInfoEnable
7782
}
83+
if ttlInfoJobInterval != nil {
84+
if tblInfo.TTLInfo == nil {
85+
return ver, errors.Trace(dbterror.ErrSetTTLOptionForNonTTLTable.FastGenByArgs("TTL_JOB_INTERVAL"))
86+
}
87+
88+
tblInfo.TTLInfo.JobInterval = *ttlInfoJobInterval
89+
}
7890

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

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

168181
intervalExpr := sb.String()
@@ -171,14 +184,27 @@ func getTTLInfoInOptions(options []*ast.TableOption) (ttlInfo *model.TTLInfo, tt
171184
IntervalExprStr: intervalExpr,
172185
IntervalTimeUnit: int(op.TimeUnitValue.Unit),
173186
Enable: true,
187+
JobInterval: duration.Duration{Hour: 1},
174188
}
175189
case ast.TableOptionTTLEnable:
176190
ttlEnable = &op.BoolValue
191+
case ast.TableOptionTTLJobInterval:
192+
schedule, err := duration.ParseDuration(op.StrValue)
193+
if err != nil {
194+
// this branch is actually unreachable, as the value has been validated in parser
195+
return nil, nil, nil, err
196+
}
197+
ttlCronJobSchedule = &schedule
177198
}
178199
}
179200

180-
if ttlInfo != nil && ttlEnable != nil {
181-
ttlInfo.Enable = *ttlEnable
201+
if ttlInfo != nil {
202+
if ttlEnable != nil {
203+
ttlInfo.Enable = *ttlEnable
204+
}
205+
if ttlCronJobSchedule != nil {
206+
ttlInfo.JobInterval = *ttlCronJobSchedule
207+
}
182208
}
183-
return ttlInfo, ttlEnable, nil
209+
return ttlInfo, ttlEnable, ttlCronJobSchedule, nil
184210
}

0 commit comments

Comments
 (0)