From 7f632be1d5c8d09fd07863ab968355b26a1c0b15 Mon Sep 17 00:00:00 2001 From: YangKeao Date: Sun, 27 Nov 2022 21:42:00 -0500 Subject: [PATCH] ddl: add ttl related jobs / execution (#39298) close pingcap/tidb#39269, close pingcap/tidb#39270, close pingcap/tidb#39271 --- build/nogo_config.json | 2 + ddl/BUILD.bazel | 2 + ddl/column.go | 18 ++++ ddl/ddl_api.go | 150 ++++++++++++++++++++++++++++++ ddl/ddl_worker.go | 4 + ddl/table_test.go | 81 ++++++++++++++++ ddl/ttl.go | 163 +++++++++++++++++++++++++++++++++ ddl/ttl_test.go | 116 +++++++++++++++++++++++ errno/errcode.go | 3 + errno/errname.go | 3 + errors.toml | 15 +++ executor/BUILD.bazel | 1 + executor/ddl_test.go | 55 +++++++++++ executor/show.go | 23 +++++ executor/showtest/show_test.go | 21 +++++ parser/model/ddl.go | 6 +- parser/model/model.go | 21 +++++ parser/model/model_test.go | 20 ++++ util/dbterror/ddl_terror.go | 7 ++ 19 files changed, 710 insertions(+), 1 deletion(-) create mode 100644 ddl/ttl.go create mode 100644 ddl/ttl_test.go diff --git a/build/nogo_config.json b/build/nogo_config.json index 8704ede461138..ce3715232b472 100644 --- a/build/nogo_config.json +++ b/build/nogo_config.json @@ -365,6 +365,8 @@ "ddl/backfilling.go": "ddl/backfilling.go", "ddl/column.go": "ddl/column.go", "ddl/index.go": "ddl/index.go", + "ddl/ttl.go": "ddl/ttl.go", + "ddl/ttl_test.go": "ddl/ttl_test.go", "ddl/ingest/": "ddl/ingest/", "expression/builtin_cast.go": "expression/builtin_cast code", "server/conn.go": "server/conn.go", diff --git a/ddl/BUILD.bazel b/ddl/BUILD.bazel index 388bf91f2e7f7..e5ef726cfc749 100644 --- a/ddl/BUILD.bazel +++ b/ddl/BUILD.bazel @@ -45,6 +45,7 @@ go_library( "stat.go", "table.go", "table_lock.go", + "ttl.go", ], importpath = "github.com/pingcap/tidb/ddl", visibility = [ @@ -195,6 +196,7 @@ go_test( "table_split_test.go", "table_test.go", "tiflash_replica_test.go", + "ttl_test.go", ], embed = [":ddl"], flaky = True, diff --git a/ddl/column.go b/ddl/column.go index cd68c8e3eda27..96ab745377a92 100644 --- a/ddl/column.go +++ b/ddl/column.go @@ -342,6 +342,9 @@ func checkDropColumn(d *ddlCtx, t *meta.Meta, job *model.Job) (*model.TableInfo, if err = checkDropColumnWithForeignKeyConstraintInOwner(d, t, job, tblInfo, colName.L); err != nil { return nil, nil, nil, false, errors.Trace(err) } + if err = checkDropColumnWithTTLConfig(tblInfo, colName.L); err != nil { + return nil, nil, nil, false, errors.Trace(err) + } idxInfos := listIndicesWithColumn(colName.L, tblInfo.Indices) return tblInfo, colInfo, idxInfos, false, nil } @@ -858,6 +861,9 @@ func adjustTableInfoAfterModifyColumnWithData(tblInfo *model.TableInfo, pos *ast indexesToRemove := filterIndexesToRemove(changingIdxs, newName, tblInfo) replaceOldIndexes(tblInfo, indexesToRemove) } + if tblInfo.TTLInfo != nil { + updateTTLInfoWhenModifyColumn(tblInfo, oldCol.Name, changingCol.Name) + } // Move the new column to a correct offset. destOffset, err := LocateOffsetToMove(changingCol.Offset, pos, tblInfo) if err != nil { @@ -932,6 +938,17 @@ func updateFKInfoWhenModifyColumn(tblInfo *model.TableInfo, oldCol, newCol model } } +func updateTTLInfoWhenModifyColumn(tblInfo *model.TableInfo, oldCol, newCol model.CIStr) { + if oldCol.L == newCol.L { + return + } + if tblInfo.TTLInfo != nil { + if tblInfo.TTLInfo.ColumnName.L == oldCol.L { + tblInfo.TTLInfo.ColumnName = newCol + } + } +} + // filterIndexesToRemove filters out the indexes that can be removed. func filterIndexesToRemove(changingIdxs []*model.IndexInfo, colName model.CIStr, tblInfo *model.TableInfo) []*model.IndexInfo { indexesToRemove := make([]*model.IndexInfo, 0, len(changingIdxs)) @@ -1474,6 +1491,7 @@ func adjustTableInfoAfterModifyColumn( tblInfo.MoveColumnInfo(oldCol.Offset, destOffset) updateNewIdxColsNameOffset(tblInfo.Indices, oldCol.Name, newCol) updateFKInfoWhenModifyColumn(tblInfo, oldCol.Name, newCol.Name) + updateTTLInfoWhenModifyColumn(tblInfo, oldCol.Name, newCol.Name) return nil } diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 6111dd3fb4fdb..505d413fff8c2 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -2115,6 +2115,11 @@ func checkTableInfoValidWithStmt(ctx sessionctx.Context, tbInfo *model.TableInfo } } } + if tbInfo.TTLInfo != nil { + if err := checkTTLInfoValid(ctx, tbInfo); err != nil { + return errors.Trace(err) + } + } return nil } @@ -2193,6 +2198,10 @@ func BuildTableInfoWithLike(ctx sessionctx.Context, ident ast.Ident, referTblInf copy(pi.Definitions, referTblInfo.Partition.Definitions) tblInfo.Partition = &pi } + + if referTblInfo.TTLInfo != nil { + tblInfo.TTLInfo = referTblInfo.TTLInfo.Clone() + } return &tblInfo, nil } @@ -3000,6 +3009,8 @@ 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 + for _, op := range options { switch op.Tp { case ast.TableOptionAutoIncrement: @@ -3036,6 +3047,23 @@ 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 { + continue + } + + ttlInfo, ttlEnable, 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) + } + + tbInfo.TTLInfo = ttlInfo + handledTTLOrTTLEnable = true } } shardingBits := shardingBits(tbInfo) @@ -3227,6 +3255,7 @@ func (d *ddl) AlterTable(ctx context.Context, sctx sessionctx.Context, stmt *ast } for _, spec := range validSpecs { var handledCharsetOrCollate bool + var handledTTLOrTTLEnable bool switch spec.Tp { case ast.AlterTableAddColumns: err = d.AddColumn(sctx, ident, spec) @@ -3363,6 +3392,20 @@ 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: + var ttlInfo *model.TTLInfo + var ttlEnable *bool + + if handledTTLOrTTLEnable { + continue + } + ttlInfo, ttlEnable, err = getTTLInfoInOptions(spec.Options) + if err != nil { + return err + } + err = d.AlterTableTTLInfoOrEnable(sctx, ident, ttlInfo, ttlEnable) + + handledTTLOrTTLEnable = true default: err = dbterror.ErrUnsupportedAlterTableOption } @@ -3406,6 +3449,9 @@ func (d *ddl) AlterTable(ctx context.Context, sctx sessionctx.Context, stmt *ast case ast.AlterTableDisableKeys, ast.AlterTableEnableKeys: // Nothing to do now, see https://github.com/pingcap/tidb/issues/1051 // MyISAM specific + case ast.AlterTableRemoveTTL: + // the parser makes sure we have only one `ast.AlterTableRemoveTTL` in an alter statement + err = d.AlterTableRemoveTTL(sctx, ident) default: err = errors.Trace(dbterror.ErrUnsupportedAlterTableSpec) } @@ -4238,6 +4284,11 @@ func checkIsDroppableColumn(ctx sessionctx.Context, is infoschema.InfoSchema, sc if err != nil { return false, errors.Trace(err) } + // Check the column with TTL config + err = checkDropColumnWithTTLConfig(tblInfo, colName.L) + if err != nil { + return false, errors.Trace(err) + } // We don't support dropping column with PK handle covered now. if col.IsPKHandleColumn(tblInfo) { return false, dbterror.ErrUnsupportedPKHandle @@ -4724,6 +4775,13 @@ func GetModifiableColumnJob( return nil, errors.Trace(err) } + if t.Meta().TTLInfo != nil { + // the column referenced by TTL should be a time type + if t.Meta().TTLInfo.ColumnName.L == originalColName.L && !types.IsTypeTime(newCol.ColumnInfo.FieldType.GetType()) { + return nil, errors.Trace(dbterror.ErrUnsupportedColumnInTTLConfig.GenWithStackByArgs(newCol.ColumnInfo.Name.O)) + } + } + var newAutoRandBits uint64 if newAutoRandBits, err = checkAutoRandom(t.Meta(), col, specNewColumn); err != nil { return nil, errors.Trace(err) @@ -5262,6 +5320,98 @@ func (d *ddl) AlterTableSetTiFlashReplica(ctx sessionctx.Context, ident ast.Iden return errors.Trace(err) } +// 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. +// 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 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 { + is := d.infoCache.GetLatest() + schema, ok := is.SchemaByName(ident.Schema) + if !ok { + return infoschema.ErrDatabaseNotExists.GenWithStackByArgs(ident.Schema) + } + + tb, err := is.TableByName(ident.Schema, ident.Name) + if err != nil { + return errors.Trace(infoschema.ErrTableNotExists.GenWithStackByArgs(ident.Schema, ident.Name)) + } + + tblInfo := tb.Meta().Clone() + tableID := tblInfo.ID + tableName := tblInfo.Name.L + + var job *model.Job + if ttlInfo != nil { + tblInfo.TTLInfo = ttlInfo + err = checkTTLInfoValid(ctx, tblInfo) + 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) + } + + job = &model.Job{ + SchemaID: schema.ID, + TableID: tableID, + SchemaName: schema.Name.L, + TableName: tableName, + Type: model.ActionAlterTTLInfo, + BinlogInfo: &model.HistoryInfo{}, + Args: []interface{}{ttlInfo, ttlEnable}, + } + } + + err = d.DoDDLJob(ctx, job) + err = d.callHookOnChanged(job, err) + return errors.Trace(err) +} + +func (d *ddl) AlterTableRemoveTTL(ctx sessionctx.Context, ident ast.Ident) error { + is := d.infoCache.GetLatest() + + schema, ok := is.SchemaByName(ident.Schema) + if !ok { + return infoschema.ErrDatabaseNotExists.GenWithStackByArgs(ident.Schema) + } + + tb, err := is.TableByName(ident.Schema, ident.Name) + if err != nil { + return errors.Trace(infoschema.ErrTableNotExists.GenWithStackByArgs(ident.Schema, ident.Name)) + } + + tblInfo := tb.Meta().Clone() + tableID := tblInfo.ID + tableName := tblInfo.Name.L + + if tblInfo.TTLInfo != nil { + job := &model.Job{ + SchemaID: schema.ID, + TableID: tableID, + SchemaName: schema.Name.L, + TableName: tableName, + Type: model.ActionAlterTTLRemove, + BinlogInfo: &model.HistoryInfo{}, + } + err = d.DoDDLJob(ctx, job) + err = d.callHookOnChanged(job, err) + return errors.Trace(err) + } + + return nil +} + func isTableTiFlashSupported(schema *model.DBInfo, tb table.Table) error { // Memory tables and system tables are not supported by TiFlash if util.IsMemOrSysDB(schema.Name.L) { diff --git a/ddl/ddl_worker.go b/ddl/ddl_worker.go index 956f4c805347f..1a13feb81e079 100644 --- a/ddl/ddl_worker.go +++ b/ddl/ddl_worker.go @@ -1287,6 +1287,10 @@ func (w *worker) runDDLJob(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, ver, err = w.onFlashbackCluster(d, t, job) case model.ActionMultiSchemaChange: ver, err = onMultiSchemaChange(w, d, t, job) + case model.ActionAlterTTLInfo: + ver, err = onTTLInfoChange(d, t, job) + case model.ActionAlterTTLRemove: + ver, err = onTTLInfoRemove(d, t, job) default: // Invalid job, cancel it. job.State = model.JobStateCancelled diff --git a/ddl/table_test.go b/ddl/table_test.go index a9320e01cc781..9641c24eb0f19 100644 --- a/ddl/table_test.go +++ b/ddl/table_test.go @@ -24,7 +24,9 @@ import ( "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/meta/autoid" + "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessiontxn" "github.com/pingcap/tidb/table" @@ -371,3 +373,82 @@ func TestCreateTables(t *testing.T) { testGetTable(t, domain, genIDs[1]) testGetTable(t, domain, genIDs[2]) } + +func TestAlterTTL(t *testing.T) { + store, domain := testkit.CreateMockStoreAndDomainWithSchemaLease(t, testLease) + + d := domain.DDL() + + dbInfo, err := testSchemaInfo(store, "test_table") + require.NoError(t, err) + testCreateSchema(t, testkit.NewTestKit(t, store).Session(), d, dbInfo) + + ctx := testkit.NewTestKit(t, store).Session() + + // initialize a table with ttlInfo + tableName := "t" + tblInfo, err := testTableInfo(store, tableName, 2) + require.NoError(t, err) + tblInfo.Columns[0].FieldType = *types.NewFieldType(mysql.TypeDatetime) + tblInfo.Columns[1].FieldType = *types.NewFieldType(mysql.TypeDatetime) + tblInfo.TTLInfo = &model.TTLInfo{ + ColumnName: tblInfo.Columns[0].Name, + IntervalExprStr: "5", + IntervalTimeUnit: int(ast.TimeUnitDay), + } + + // create table + job := testCreateTable(t, ctx, d, dbInfo, tblInfo) + testCheckTableState(t, store, dbInfo, tblInfo, model.StatePublic) + testCheckJobDone(t, store, job.ID, true) + + // submit ddl job to modify ttlInfo + tableInfoAfterAlterTTLInfo := tblInfo.Clone() + require.NoError(t, err) + tableInfoAfterAlterTTLInfo.TTLInfo = &model.TTLInfo{ + ColumnName: tblInfo.Columns[1].Name, + IntervalExprStr: "1", + IntervalTimeUnit: int(ast.TimeUnitYear), + } + + job = &model.Job{ + SchemaID: dbInfo.ID, + TableID: tblInfo.ID, + Type: model.ActionAlterTTLInfo, + BinlogInfo: &model.HistoryInfo{}, + Args: []interface{}{&model.TTLInfo{ + ColumnName: tblInfo.Columns[1].Name, + IntervalExprStr: "1", + IntervalTimeUnit: int(ast.TimeUnitYear), + }}, + } + ctx.SetValue(sessionctx.QueryString, "skip") + require.NoError(t, d.DoDDLJob(ctx, job)) + + v := getSchemaVer(t, ctx) + checkHistoryJobArgs(t, ctx, job.ID, &historyJobArgs{ver: v, tbl: nil}) + + // assert the ddlInfo as expected + historyJob, err := ddl.GetHistoryJobByID(testkit.NewTestKit(t, store).Session(), job.ID) + require.NoError(t, err) + require.Equal(t, tableInfoAfterAlterTTLInfo.TTLInfo, historyJob.BinlogInfo.TableInfo.TTLInfo) + + // submit a ddl job to modify ttlEnabled + job = &model.Job{ + SchemaID: dbInfo.ID, + TableID: tblInfo.ID, + Type: model.ActionAlterTTLRemove, + BinlogInfo: &model.HistoryInfo{}, + Args: []interface{}{true}, + } + ctx.SetValue(sessionctx.QueryString, "skip") + require.NoError(t, d.DoDDLJob(ctx, job)) + + v = getSchemaVer(t, ctx) + checkHistoryJobArgs(t, ctx, job.ID, &historyJobArgs{ver: v, tbl: nil}) + + // assert the ddlInfo as expected + historyJob, err = ddl.GetHistoryJobByID(testkit.NewTestKit(t, store).Session(), job.ID) + require.NoError(t, err) + require.Empty(t, historyJob.BinlogInfo.TableInfo.TTLInfo) +} diff --git a/ddl/ttl.go b/ddl/ttl.go new file mode 100644 index 0000000000000..7ef35158ca3ef --- /dev/null +++ b/ddl/ttl.go @@ -0,0 +1,163 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package ddl + +import ( + "fmt" + "strings" + + "github.com/pingcap/errors" + "github.com/pingcap/tidb/expression" + "github.com/pingcap/tidb/meta" + "github.com/pingcap/tidb/parser" + "github.com/pingcap/tidb/parser/ast" + "github.com/pingcap/tidb/parser/format" + "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util/dbterror" +) + +func onTTLInfoRemove(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, err error) { + tblInfo, err := GetTableInfoAndCancelFaultJob(t, job, job.SchemaID) + if err != nil { + return ver, errors.Trace(err) + } + + tblInfo.TTLInfo = nil + ver, err = updateVersionAndTableInfo(d, t, job, tblInfo, true) + if err != nil { + return ver, errors.Trace(err) + } + job.FinishTableJob(model.JobStateDone, model.StatePublic, ver, tblInfo) + return ver, nil +} + +func onTTLInfoChange(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, err error) { + // at least one for them is not nil + var ttlInfo *model.TTLInfo + var ttlInfoEnable *bool + + if err := job.DecodeArgs(&ttlInfo, &ttlInfoEnable); err != nil { + job.State = model.JobStateCancelled + return ver, errors.Trace(err) + } + + tblInfo, err := GetTableInfoAndCancelFaultJob(t, job, job.SchemaID) + if err != nil { + return ver, errors.Trace(err) + } + + if ttlInfo != nil { + // if the TTL_ENABLE is not set explicitly, use the original value + if ttlInfoEnable == nil { + ttlInfo.Enable = tblInfo.TTLInfo.Enable + } + tblInfo.TTLInfo = ttlInfo + } + if ttlInfoEnable != nil { + if tblInfo.TTLInfo == nil { + return ver, errors.Trace(dbterror.ErrSetTTLEnableForNonTTLTable) + } + + tblInfo.TTLInfo.Enable = *ttlInfoEnable + } + + ver, err = updateVersionAndTableInfo(d, t, job, tblInfo, true) + if err != nil { + return ver, errors.Trace(err) + } + job.FinishTableJob(model.JobStateDone, model.StatePublic, ver, tblInfo) + return ver, nil +} + +func checkTTLInfoValid(ctx sessionctx.Context, tblInfo *model.TableInfo) error { + if err := checkTTLIntervalExpr(ctx, tblInfo.TTLInfo); err != nil { + return err + } + + return checkTTLInfoColumnType(tblInfo) +} + +func checkTTLIntervalExpr(ctx sessionctx.Context, ttlInfo *model.TTLInfo) error { + // FIXME: use a better way to validate the interval expression in ttl + var nowAddIntervalExpr ast.ExprNode + + unit := ast.TimeUnitType(ttlInfo.IntervalTimeUnit) + expr := fmt.Sprintf("select NOW() + INTERVAL %s %s", ttlInfo.IntervalExprStr, unit.String()) + stmts, _, err := parser.New().ParseSQL(expr) + if err != nil { + // FIXME: the error information can be wrong, as it could indicate an unknown position to user. + return errors.Trace(err) + } + nowAddIntervalExpr = stmts[0].(*ast.SelectStmt).Fields.Fields[0].Expr + _, err = expression.EvalAstExpr(ctx, nowAddIntervalExpr) + return err +} + +func checkTTLInfoColumnType(tblInfo *model.TableInfo) error { + colInfo := findColumnByName(tblInfo.TTLInfo.ColumnName.L, tblInfo) + if colInfo == nil { + return dbterror.ErrBadField.GenWithStackByArgs(tblInfo.TTLInfo.ColumnName.O, "TTL config") + } + if !types.IsTypeTime(colInfo.FieldType.GetType()) { + return dbterror.ErrUnsupportedColumnInTTLConfig.GenWithStackByArgs(tblInfo.TTLInfo.ColumnName.O) + } + + return nil +} + +func checkDropColumnWithTTLConfig(tblInfo *model.TableInfo, colName string) error { + if tblInfo.TTLInfo != nil { + if tblInfo.TTLInfo.ColumnName.L == colName { + return dbterror.ErrTTLColumnCannotDrop.GenWithStackByArgs(colName) + } + } + + return nil +} + +// 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) { + for _, op := range options { + switch op.Tp { + case ast.TableOptionTTL: + var sb strings.Builder + restoreFlags := format.RestoreStringSingleQuotes | format.RestoreNameBackQuotes + restoreCtx := format.NewRestoreCtx(restoreFlags, &sb) + err := op.Value.Restore(restoreCtx) + if err != nil { + return nil, nil, err + } + + intervalExpr := sb.String() + ttlInfo = &model.TTLInfo{ + ColumnName: op.ColumnName.Name, + IntervalExprStr: intervalExpr, + IntervalTimeUnit: int(op.TimeUnitValue.Unit), + Enable: true, + } + case ast.TableOptionTTLEnable: + ttlEnable = &op.BoolValue + } + } + + if ttlInfo != nil && ttlEnable != nil { + ttlInfo.Enable = *ttlEnable + } + return ttlInfo, ttlEnable, nil +} diff --git a/ddl/ttl_test.go b/ddl/ttl_test.go new file mode 100644 index 0000000000000..a994792a54a43 --- /dev/null +++ b/ddl/ttl_test.go @@ -0,0 +1,116 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package ddl + +import ( + "testing" + + "github.com/pingcap/tidb/parser/ast" + "github.com/pingcap/tidb/parser/model" + "github.com/stretchr/testify/assert" +) + +func Test_getTTLInfoInOptions(t *testing.T) { + falseValue := false + trueValue := true + + cases := []struct { + options []*ast.TableOption + ttlInfo *model.TTLInfo + ttlEnable *bool + err error + }{ + { + []*ast.TableOption{}, + nil, + nil, + nil, + }, + { + []*ast.TableOption{ + { + Tp: ast.TableOptionTTL, + ColumnName: &ast.ColumnName{Name: model.NewCIStr("test_column")}, + Value: ast.NewValueExpr(5, "", ""), + TimeUnitValue: &ast.TimeUnitExpr{Unit: ast.TimeUnitYear}, + }, + }, + &model.TTLInfo{ + ColumnName: model.NewCIStr("test_column"), + IntervalExprStr: "5", + IntervalTimeUnit: int(ast.TimeUnitYear), + Enable: true, + }, + nil, + nil, + }, + { + []*ast.TableOption{ + { + Tp: ast.TableOptionTTLEnable, + BoolValue: false, + }, + { + Tp: ast.TableOptionTTL, + ColumnName: &ast.ColumnName{Name: model.NewCIStr("test_column")}, + Value: ast.NewValueExpr(5, "", ""), + TimeUnitValue: &ast.TimeUnitExpr{Unit: ast.TimeUnitYear}, + }, + }, + &model.TTLInfo{ + ColumnName: model.NewCIStr("test_column"), + IntervalExprStr: "5", + IntervalTimeUnit: int(ast.TimeUnitYear), + Enable: false, + }, + &falseValue, + nil, + }, + { + []*ast.TableOption{ + { + Tp: ast.TableOptionTTLEnable, + BoolValue: false, + }, + { + Tp: ast.TableOptionTTL, + ColumnName: &ast.ColumnName{Name: model.NewCIStr("test_column")}, + Value: ast.NewValueExpr(5, "", ""), + TimeUnitValue: &ast.TimeUnitExpr{Unit: ast.TimeUnitYear}, + }, + { + Tp: ast.TableOptionTTLEnable, + BoolValue: true, + }, + }, + &model.TTLInfo{ + ColumnName: model.NewCIStr("test_column"), + IntervalExprStr: "5", + IntervalTimeUnit: int(ast.TimeUnitYear), + Enable: true, + }, + &trueValue, + nil, + }, + } + + for _, c := range cases { + ttlInfo, ttlEnable, err := getTTLInfoInOptions(c.options) + + assert.Equal(t, c.ttlInfo, ttlInfo) + assert.Equal(t, c.ttlEnable, ttlEnable) + assert.Equal(t, c.err, err) + } +} diff --git a/errno/errcode.go b/errno/errcode.go index 60257be0b76c9..cb54d3c11619a 100644 --- a/errno/errcode.go +++ b/errno/errcode.go @@ -1034,6 +1034,9 @@ const ( ErrGettingNoopVariable = 8145 ErrCannotMigrateSession = 8146 ErrLazyUniquenessCheckFailure = 8147 + ErrUnsupportedColumnInTTLConfig = 8148 + ErrTTLColumnCannotDrop = 8149 + ErrSetTTLEnableForNonTTLTable = 8150 // Error codes used by TiDB ddl package ErrUnsupportedDDLOperation = 8200 diff --git a/errno/errname.go b/errno/errname.go index 5db48f9f3ce32..bd512954c17f4 100644 --- a/errno/errname.go +++ b/errno/errname.go @@ -1029,6 +1029,9 @@ var MySQLErrName = map[uint16]*mysql.ErrMessage{ ErrGettingNoopVariable: mysql.Message("variable %s has no effect in TiDB", nil), ErrCannotMigrateSession: mysql.Message("cannot migrate the current session: %s", nil), ErrLazyUniquenessCheckFailure: mysql.Message("transaction aborted because lazy uniqueness check is enabled and an error occurred: %s", nil), + ErrUnsupportedColumnInTTLConfig: mysql.Message("Field '%-.192s' is of a not supported type for TTL config, expect DATETIME, DATE or TIMESTAMP", nil), + ErrTTLColumnCannotDrop: mysql.Message("Cannot drop column '%-.192s': needed in TTL config", nil), + ErrSetTTLEnableForNonTTLTable: mysql.Message("Cannot set TTL_ENABLE on a table without TTL config", nil), ErrWarnOptimizerHintInvalidInteger: mysql.Message("integer value is out of range in '%s'", nil), ErrWarnOptimizerHintUnsupportedHint: mysql.Message("Optimizer hint %s is not supported by TiDB and is ignored", nil), diff --git a/errors.toml b/errors.toml index 647627a3acab6..e34de79661c62 100644 --- a/errors.toml +++ b/errors.toml @@ -1191,6 +1191,21 @@ error = ''' `%s` is unsupported on temporary tables. ''' +["ddl:8148"] +error = ''' +Field '%-.192s' is of a not supported type for TTL config, expect DATETIME, DATE or TIMESTAMP +''' + +["ddl:8149"] +error = ''' +Cannot drop column '%-.192s': needed in TTL config +''' + +["ddl:8150"] +error = ''' +Cannot set TTL_ENABLE on a table without TTL config +''' + ["ddl:8200"] error = ''' Unsupported shard_row_id_bits for table with primary key as row id diff --git a/executor/BUILD.bazel b/executor/BUILD.bazel index cf91360b17a60..7473570b03661 100644 --- a/executor/BUILD.bazel +++ b/executor/BUILD.bazel @@ -121,6 +121,7 @@ go_library( "//parser/ast", "//parser/auth", "//parser/charset", + "//parser/format", "//parser/model", "//parser/mysql", "//parser/terror", diff --git a/executor/ddl_test.go b/executor/ddl_test.go index 54a67cbbf9cb7..3c97d87f1a280 100644 --- a/executor/ddl_test.go +++ b/executor/ddl_test.go @@ -1559,3 +1559,58 @@ func TestRenameMultiTables(t *testing.T) { tk.MustExec("drop database rename2") tk.MustExec("drop database rename3") } + +func TestCreateTableWithTTL(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + + tk.MustExec("CREATE TABLE t (created_at datetime) TTL = `created_at` + INTERVAL 5 DAY") + tk.MustQuery("SHOW CREATE TABLE t").Check(testkit.Rows("t CREATE TABLE `t` (\n `created_at` datetime DEFAULT NULL\n) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin TTL = `created_at` + INTERVAL 5 DAY TTL_ENABLE = 'ON'")) + tk.MustExec("DROP TABLE t") + + tk.MustGetErrMsg("CREATE TABLE t (id int) TTL = `id` + INTERVAL 5 DAY", "[ddl:8148]Field 'id' is of a not supported type for TTL config, expect DATETIME, DATE or TIMESTAMP") + + tk.MustGetErrMsg("CREATE TABLE t (id int) TTL_ENABLE = 'ON'", "[ddl:8150]Cannot set TTL_ENABLE on a table without TTL config") + + // when multiple ttl and ttl_enable configs are submitted, only the last one will be handled + tk.MustExec("CREATE TABLE t (created_at datetime) TTL_ENABLE = 'ON' TTL = `created_at` + INTERVAL 1 DAY TTL = `created_at` + INTERVAL 2 DAY TTL = `created_at` + INTERVAL 3 DAY TTL_ENABLE = 'OFF'") + tk.MustQuery("SHOW CREATE TABLE t").Check(testkit.Rows("t CREATE TABLE `t` (\n `created_at` datetime DEFAULT NULL\n) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin TTL = `created_at` + INTERVAL 3 DAY TTL_ENABLE = 'OFF'")) + tk.MustExec("DROP TABLE t") +} + +func TestAlterTTLInfo(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + + tk.MustExec("CREATE TABLE t (created_at datetime, updated_at datetime, wrong_type int) TTL = `created_at` + INTERVAL 5 DAY") + tk.MustExec("ALTER TABLE t TTL = `updated_at` + INTERVAL 2 YEAR") + tk.MustQuery("SHOW CREATE TABLE t").Check(testkit.Rows("t CREATE TABLE `t` (\n `created_at` datetime DEFAULT NULL,\n `updated_at` datetime DEFAULT NULL,\n `wrong_type` int(11) DEFAULT NULL\n) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin TTL = `updated_at` + INTERVAL 2 YEAR TTL_ENABLE = 'ON'")) + + tk.MustExec("ALTER TABLE t TTL_ENABLE = 'OFF'") + tk.MustQuery("SHOW CREATE TABLE t").Check(testkit.Rows("t CREATE TABLE `t` (\n `created_at` datetime DEFAULT NULL,\n `updated_at` datetime DEFAULT NULL,\n `wrong_type` int(11) DEFAULT NULL\n) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin TTL = `updated_at` + INTERVAL 2 YEAR TTL_ENABLE = 'OFF'")) + + tk.MustGetErrMsg("ALTER TABLE t TTL = `not_exist` + INTERVAL 2 YEAR", "[ddl:1054]Unknown column 'not_exist' in 'TTL config'") + + tk.MustGetErrMsg("ALTER TABLE t TTL = `wrong_type` + INTERVAL 2 YEAR", "[ddl:8148]Field 'wrong_type' is of a not supported type for TTL config, expect DATETIME, DATE or TIMESTAMP") + + tk.MustGetErrMsg("ALTER TABLE t DROP COLUMN updated_at", "[ddl:8149]Cannot drop column 'updated_at': needed in TTL config") + tk.MustGetErrMsg("ALTER TABLE t CHANGE updated_at updated_at_new INT", "[ddl:8148]Field 'updated_at_new' is of a not supported type for TTL config, expect DATETIME, DATE or TIMESTAMP") + + tk.MustExec("ALTER TABLE t RENAME COLUMN `updated_at` TO `updated_at_2`") + tk.MustQuery("SHOW CREATE TABLE t").Check(testkit.Rows("t CREATE TABLE `t` (\n `created_at` datetime DEFAULT NULL,\n `updated_at_2` datetime DEFAULT NULL,\n `wrong_type` int(11) DEFAULT NULL\n) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin TTL = `updated_at_2` + INTERVAL 2 YEAR TTL_ENABLE = 'OFF'")) + + tk.MustExec("ALTER TABLE t CHANGE `updated_at_2` `updated_at_3` date") + tk.MustQuery("SHOW CREATE TABLE t").Check(testkit.Rows("t CREATE TABLE `t` (\n `created_at` datetime DEFAULT NULL,\n `updated_at_3` date DEFAULT NULL,\n `wrong_type` int(11) DEFAULT NULL\n) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin TTL = `updated_at_3` + INTERVAL 2 YEAR TTL_ENABLE = 'OFF'")) + + tk.MustExec("ALTER TABLE t TTL = `updated_at_3` + INTERVAL 3 YEAR") + tk.MustQuery("SHOW CREATE TABLE t").Check(testkit.Rows("t CREATE TABLE `t` (\n `created_at` datetime DEFAULT NULL,\n `updated_at_3` date DEFAULT NULL,\n `wrong_type` int(11) DEFAULT NULL\n) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin TTL = `updated_at_3` + INTERVAL 3 YEAR TTL_ENABLE = 'OFF'")) + + tk.MustGetErrMsg("ALTER TABLE t TTL_ENABLE = 'OFF' REMOVE TTL", "[ddl:8200]Unsupported multi schema change for alter table ttl") + + tk.MustExec("ALTER TABLE t REMOVE TTL") + tk.MustQuery("SHOW CREATE TABLE t").Check(testkit.Rows("t CREATE TABLE `t` (\n `created_at` datetime DEFAULT NULL,\n `updated_at_3` date DEFAULT NULL,\n `wrong_type` int(11) DEFAULT NULL\n) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin")) + + tk.MustGetErrMsg("ALTER TABLE t TTL_ENABLE = 'OFF'", "[ddl:8150]Cannot set TTL_ENABLE on a table without TTL config") +} diff --git a/executor/show.go b/executor/show.go index 72bd63a785623..10799b1ffce0d 100644 --- a/executor/show.go +++ b/executor/show.go @@ -39,6 +39,7 @@ import ( "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/auth" "github.com/pingcap/tidb/parser/charset" + parserformat "github.com/pingcap/tidb/parser/format" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/parser/terror" @@ -1222,6 +1223,28 @@ func ConstructResultOfShowCreateTable(ctx sessionctx.Context, tableInfo *model.T // add partition info here. ddl.AppendPartitionInfo(tableInfo.Partition, buf, sqlMode) + + if tableInfo.TTLInfo != nil { + restoreFlags := parserformat.RestoreStringSingleQuotes | parserformat.RestoreNameBackQuotes + restoreCtx := parserformat.NewRestoreCtx(restoreFlags, buf) + + columnName := ast.ColumnName{Name: tableInfo.TTLInfo.ColumnName} + timeUnit := ast.TimeUnitExpr{Unit: ast.TimeUnitType(tableInfo.TTLInfo.IntervalTimeUnit)} + + restoreCtx.WriteKeyWord(" TTL ") + restoreCtx.WritePlain("= ") + restoreCtx.WriteName(columnName.String()) + restoreCtx.WritePlainf(" + INTERVAL %s ", tableInfo.TTLInfo.IntervalExprStr) + err = timeUnit.Restore(restoreCtx) + if err != nil { + return err + } + if tableInfo.TTLInfo.Enable { + fmt.Fprintf(buf, " TTL_ENABLE = 'ON'") + } else { + fmt.Fprintf(buf, " TTL_ENABLE = 'OFF'") + } + } return nil } diff --git a/executor/showtest/show_test.go b/executor/showtest/show_test.go index 3566b1589a271..ae63782a0b3b8 100644 --- a/executor/showtest/show_test.go +++ b/executor/showtest/show_test.go @@ -1989,3 +1989,24 @@ func TestShowLimitReturnRow(t *testing.T) { rows = result.Rows() require.Equal(t, rows[0][2], "idx_b") } + +func TestShowTTLOption(t *testing.T) { + store := testkit.CreateMockStore(t) + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(created_at datetime) ttl = `created_at` + INTERVAL 100 YEAR") + tk.MustQuery("show create table t").Check(testkit.Rows("t CREATE TABLE `t` (\n `created_at` datetime DEFAULT NULL\n) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin TTL = `created_at` + INTERVAL 100 YEAR TTL_ENABLE = 'ON'")) + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(created_at datetime) ttl = `created_at` + INTERVAL 100 YEAR ttl_enable = 'OFF'") + tk.MustQuery("show create table t").Check(testkit.Rows("t CREATE TABLE `t` (\n `created_at` datetime DEFAULT NULL\n) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin TTL = `created_at` + INTERVAL 100 YEAR TTL_ENABLE = 'OFF'")) + + tk.MustExec("drop table if exists t") + tk.MustExec("create table t (created_at datetime) TTL = created_at + INTERVAL 3.14159 HOUR_MINUTE") + tk.MustQuery("show create table t").Check(testkit.Rows("t CREATE TABLE `t` (\n `created_at` datetime DEFAULT NULL\n) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin TTL = `created_at` + INTERVAL 3.14159 HOUR_MINUTE TTL_ENABLE = 'ON'")) + + tk.MustExec("drop table if exists t") + tk.MustExec("create table t (created_at datetime) TTL = created_at + INTERVAL \"15:20\" HOUR_MINUTE") + tk.MustQuery("show create table t").Check(testkit.Rows("t CREATE TABLE `t` (\n `created_at` datetime DEFAULT NULL\n) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin TTL = `created_at` + INTERVAL _utf8mb4'15:20' HOUR_MINUTE TTL_ENABLE = 'ON'")) +} diff --git a/parser/model/ddl.go b/parser/model/ddl.go index 88e8d8f43bffa..c9b36a9e9ef3a 100644 --- a/parser/model/ddl.go +++ b/parser/model/ddl.go @@ -99,6 +99,9 @@ const ( ActionFlashbackCluster ActionType = 62 ActionRecoverSchema ActionType = 63 ActionReorganizePartition ActionType = 64 + + ActionAlterTTLInfo ActionType = 65 + ActionAlterTTLRemove ActionType = 67 ) var actionMap = map[ActionType]string{ @@ -162,7 +165,8 @@ var actionMap = map[ActionType]string{ ActionFlashbackCluster: "flashback cluster", ActionRecoverSchema: "flashback schema", ActionReorganizePartition: "alter table reorganize partition", - + ActionAlterTTLInfo: "alter table ttl", + ActionAlterTTLRemove: "alter table no_ttl", // `ActionAlterTableAlterPartition` is removed and will never be used. // Just left a tombstone here for compatibility. __DEPRECATED_ActionAlterTableAlterPartition: "alter partition", diff --git a/parser/model/model.go b/parser/model/model.go index dcc4cdd0af8bc..e38e5b368f4a1 100644 --- a/parser/model/model.go +++ b/parser/model/model.go @@ -548,6 +548,8 @@ type TableInfo struct { StatsOptions *StatsOptions `json:"stats_options"` ExchangePartitionInfo *ExchangePartitionInfo `json:"exchange_partition_info"` + + TTLInfo *TTLInfo `json:"ttl_info"` } // TableCacheStatusType is the type of the table cache status @@ -748,6 +750,10 @@ func (t *TableInfo) Clone() *TableInfo { nt.ForeignKeys[i] = t.ForeignKeys[i].Clone() } + if t.TTLInfo != nil { + nt.TTLInfo = t.TTLInfo.Clone() + } + return &nt } @@ -1739,6 +1745,21 @@ func (p *PolicyInfo) Clone() *PolicyInfo { return &cloned } +// TTLInfo records the TTL config +type TTLInfo struct { + ColumnName CIStr `json:"column"` + IntervalExprStr string `json:"interval_expr"` + // `IntervalTimeUnit` is actually ast.TimeUnitType. Use `int` to avoid cycle dependency + IntervalTimeUnit int `json:"interval_time_unit"` + Enable bool `json:"enable"` +} + +// Clone clones TTLInfo +func (t *TTLInfo) Clone() *TTLInfo { + cloned := *t + return &cloned +} + func writeSettingItemToBuilder(sb *strings.Builder, item string) { if sb.Len() != 0 { sb.WriteString(" ") diff --git a/parser/model/model_test.go b/parser/model/model_test.go index 47a8ecad6e4a4..6062df58aabec 100644 --- a/parser/model/model_test.go +++ b/parser/model/model_test.go @@ -785,3 +785,23 @@ func TestIsIndexPrefixCovered(t *testing.T) { require.Equal(t, true, IsIndexPrefixCovered(tbl, i1, NewCIStr("c_4"), NewCIStr("c_2"))) require.Equal(t, false, IsIndexPrefixCovered(tbl, i0, NewCIStr("c_2"))) } + +func TestTTLInfoClone(t *testing.T) { + ttlInfo := &TTLInfo{ + ColumnName: NewCIStr("test"), + IntervalExprStr: "test_expr", + IntervalTimeUnit: 5, + Enable: true, + } + + clonedTTLInfo := ttlInfo.Clone() + clonedTTLInfo.ColumnName = NewCIStr("test_2") + clonedTTLInfo.IntervalExprStr = "test_expr_2" + clonedTTLInfo.IntervalTimeUnit = 9 + clonedTTLInfo.Enable = false + + require.Equal(t, "test", ttlInfo.ColumnName.O) + require.Equal(t, "test_expr", ttlInfo.IntervalExprStr) + require.Equal(t, 5, ttlInfo.IntervalTimeUnit) + require.Equal(t, true, ttlInfo.Enable) +} diff --git a/util/dbterror/ddl_terror.go b/util/dbterror/ddl_terror.go index 65a02fb23da24..cce0cd23c8cb7 100644 --- a/util/dbterror/ddl_terror.go +++ b/util/dbterror/ddl_terror.go @@ -416,4 +416,11 @@ var ( ErrForeignKeyColumnCannotChangeChild = ClassDDL.NewStd(mysql.ErrForeignKeyColumnCannotChangeChild) // ErrNoReferencedRow2 returns when there are rows in child table don't have related foreign key value in refer table. ErrNoReferencedRow2 = ClassDDL.NewStd(mysql.ErrNoReferencedRow2) + + // ErrUnsupportedColumnInTTLConfig returns when a column type is not expected in TTL config + ErrUnsupportedColumnInTTLConfig = ClassDDL.NewStd(mysql.ErrUnsupportedColumnInTTLConfig) + // ErrTTLColumnCannotDrop returns when a column is dropped while referenced by TTL config + ErrTTLColumnCannotDrop = ClassDDL.NewStd(mysql.ErrTTLColumnCannotDrop) + // ErrSetTTLEnableForNonTTLTable returns when the `TTL_ENABLE` option is set on a non-TTL table + ErrSetTTLEnableForNonTTLTable = ClassDDL.NewStd(mysql.ErrSetTTLEnableForNonTTLTable) )