Skip to content

Commit

Permalink
ttl: make the TTL compatible with dumpling, lightning and BR (#40216)
Browse files Browse the repository at this point in the history
close #40215
  • Loading branch information
lcwangchao committed Dec 29, 2022
1 parent 92a936e commit cf34941
Show file tree
Hide file tree
Showing 13 changed files with 230 additions and 5 deletions.
2 changes: 1 addition & 1 deletion br/pkg/lightning/restore/tidb.go
Original file line number Diff line number Diff line change
Expand Up @@ -152,7 +152,7 @@ func createIfNotExistsStmt(p *parser.Parser, createTable, dbName, tblName string
}

var res strings.Builder
ctx := format.NewRestoreCtx(format.DefaultRestoreFlags|format.RestoreTiDBSpecialComment, &res)
ctx := format.NewRestoreCtx(format.DefaultRestoreFlags|format.RestoreTiDBSpecialComment|format.RestoreWithTTLEnableOff, &res)

retStmts := make([]string, 0, len(stmts))
for _, stmt := range stmts {
Expand Down
8 changes: 8 additions & 0 deletions br/pkg/restore/db.go
Original file line number Diff line number Diff line change
Expand Up @@ -308,6 +308,10 @@ func (db *DB) CreateTables(ctx context.Context, tables []*metautil.Table,
return errors.Trace(err)
}
}

if ttlInfo := table.Info.TTLInfo; ttlInfo != nil {
ttlInfo.Enable = false
}
}
if err := batchSession.CreateTables(ctx, m, db.tableIDAllocFilter()); err != nil {
return err
Expand Down Expand Up @@ -336,6 +340,10 @@ func (db *DB) CreateTable(ctx context.Context, table *metautil.Table,
}
}

if ttlInfo := table.Info.TTLInfo; ttlInfo != nil {
ttlInfo.Enable = false
}

err := db.se.CreateTable(ctx, table.DB.Name, table.Info, db.tableIDAllocFilter())
if err != nil {
log.Error("create table failed",
Expand Down
3 changes: 3 additions & 0 deletions br/pkg/stream/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -56,8 +56,11 @@ go_test(
"//br/pkg/storage",
"//br/pkg/streamhelper",
"//meta",
"//parser/ast",
"//parser/model",
"//parser/mysql",
"//tablecodec",
"//types",
"//util/codec",
"//util/table-filter",
"@com_github_pingcap_kvproto//pkg/brpb",
Expand Down
5 changes: 5 additions & 0 deletions br/pkg/stream/rewrite_meta_rawkv.go
Original file line number Diff line number Diff line change
Expand Up @@ -336,6 +336,11 @@ func (sr *SchemasReplace) rewriteTableInfo(value []byte, dbID int64) ([]byte, bo
}
}

// Force to disable TTL_ENABLE when restore
if newTableInfo.TTLInfo != nil {
newTableInfo.TTLInfo.Enable = false
}

if sr.AfterTableRewritten != nil {
sr.AfterTableRewritten(false, newTableInfo)
}
Expand Down
49 changes: 49 additions & 0 deletions br/pkg/stream/rewrite_meta_rawkv_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -7,7 +7,10 @@ import (
"encoding/json"
"testing"

"github.com/pingcap/tidb/parser/ast"
"github.com/pingcap/tidb/parser/model"
"github.com/pingcap/tidb/parser/mysql"
"github.com/pingcap/tidb/types"
filter "github.com/pingcap/tidb/util/table-filter"
"github.com/stretchr/testify/require"
)
Expand Down Expand Up @@ -312,6 +315,52 @@ func TestRewriteValueForExchangePartition(t *testing.T) {
require.Equal(t, tableInfo.ID, pt1ID+100)
}

func TestRewriteValueForTTLTable(t *testing.T) {
var (
dbId int64 = 40
tableID int64 = 100
colID int64 = 1000
colName = "t"
tableName = "t1"
tableInfo model.TableInfo
)

tbl := model.TableInfo{
ID: tableID,
Name: model.NewCIStr(tableName),
Columns: []*model.ColumnInfo{
{
ID: colID,
Name: model.NewCIStr(colName),
FieldType: *types.NewFieldType(mysql.TypeTimestamp),
},
},
TTLInfo: &model.TTLInfo{
ColumnName: model.NewCIStr(colName),
IntervalExprStr: "1",
IntervalTimeUnit: int(ast.TimeUnitDay),
Enable: true,
},
}
value, err := json.Marshal(&tbl)
require.Nil(t, err)

sr := MockEmptySchemasReplace(nil)
newValue, needRewrite, err := sr.rewriteTableInfo(value, dbId)
require.Nil(t, err)
require.True(t, needRewrite)

err = json.Unmarshal(newValue, &tableInfo)
require.Nil(t, err)
require.Equal(t, tableInfo.Name.String(), tableName)
require.Equal(t, tableInfo.ID, sr.DbMap[dbId].TableMap[tableID].NewTableID)
require.NotNil(t, tableInfo.TTLInfo)
require.Equal(t, colName, tableInfo.TTLInfo.ColumnName.O)
require.Equal(t, "1", tableInfo.TTLInfo.IntervalExprStr)
require.Equal(t, int(ast.TimeUnitDay), tableInfo.TTLInfo.IntervalTimeUnit)
require.False(t, tableInfo.TTLInfo.Enable)
}

// db:70->80 -
// | - t0:71->81 -
// | | - p0:72->82
Expand Down
54 changes: 54 additions & 0 deletions br/tests/br_ttl/run.sh
Original file line number Diff line number Diff line change
@@ -0,0 +1,54 @@
#!/bin/sh
#
# 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.

set -eu
DB="$TEST_NAME"

PROGRESS_FILE="$TEST_DIR/progress_file"
BACKUPMETAV1_LOG="$TEST_DIR/backup.log"
BACKUPMETAV2_LOG="$TEST_DIR/backupv2.log"
RESTORE_LOG="$TEST_DIR/restore.log"
rm -rf $PROGRESS_FILE

run_sql "create schema $DB;"
run_sql "create table $DB.ttl_test_tbl(id int primary key, t datetime) TTL=\`t\` + interval 1 day TTL_ENABLE='ON'"

# backup db
echo "full backup meta v2 start..."
unset BR_LOG_TO_TERM
rm -f $BACKUPMETAV2_LOG
run_br backup full --log-file $BACKUPMETAV2_LOG -s "local://$TEST_DIR/${DB}v2" --pd $PD_ADDR --use-backupmeta-v2

echo "full backup meta v1 start..."
rm -f $BACKUPMETAV1_LOG
run_br backup full --log-file $BACKUPMETAV1_LOG -s "local://$TEST_DIR/$DB" --pd $PD_ADDR

TTL_MARK='![ttl]'
CREATE_SQL_CONTAINS="/*T${TTL_MARK} TTL=\`t\` + INTERVAL 1 DAY */ /*T${TTL_MARK} TTL_ENABLE='OFF' */"

# restore v2
run_sql "DROP DATABASE $DB;"
echo "restore ttl table start v2..."
run_br restore db --db $DB -s "local://$TEST_DIR/${DB}v2" --pd $PD_ADDR
run_sql "show create table $DB.ttl_test_tbl;"
check_contains "$CREATE_SQL_CONTAINS"

# restore v1
run_sql "DROP DATABASE $DB;"
echo "restore ttl table start v1..."
run_br restore db --db $DB -s "local://$TEST_DIR/$DB" --pd $PD_ADDR
run_sql "show create table $DB.ttl_test_tbl;"
check_contains "$CREATE_SQL_CONTAINS"
2 changes: 2 additions & 0 deletions br/tests/lightning_ttl/config.toml
Original file line number Diff line number Diff line change
@@ -0,0 +1,2 @@
[tikv-importer]
backend = 'local'
1 change: 1 addition & 0 deletions br/tests/lightning_ttl/data/ttldb-schema-create.sql
Original file line number Diff line number Diff line change
@@ -0,0 +1 @@
CREATE DATABASE `ttldb`;
4 changes: 4 additions & 0 deletions br/tests/lightning_ttl/data/ttldb.t1-schema.sql
Original file line number Diff line number Diff line change
@@ -0,0 +1,4 @@
CREATE TABLE `t1` (
`id` int(11) PRIMARY KEY,
`t` datetime
) TTL = `t` + INTERVAL 1 DAY TTL_ENABLE = 'ON';
26 changes: 26 additions & 0 deletions br/tests/lightning_ttl/run.sh
Original file line number Diff line number Diff line change
@@ -0,0 +1,26 @@
#!/bin/sh
#
# 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.

set -eu

run_sql 'drop database if exists ttldb;'
run_lightning

TTL_MARK='![ttl]'
CREATE_SQL_CONTAINS="/*T${TTL_MARK} TTL=\`t\` + INTERVAL 1 DAY */ /*T${TTL_MARK} TTL_ENABLE='OFF' */"

run_sql 'show create table ttldb.t1'
check_contains "$CREATE_SQL_CONTAINS"
41 changes: 37 additions & 4 deletions parser/ast/ddl.go
Original file line number Diff line number Diff line change
Expand Up @@ -1096,7 +1096,8 @@ func (n *CreateTableStmt) Restore(ctx *format.RestoreCtx) error {
ctx.WritePlain(")")
}

for i, option := range n.Options {
options := tableOptionsWithRestoreTTLFlag(ctx.Flags, n.Options)
for i, option := range options {
ctx.WritePlain(" ")
if err := option.Restore(ctx); err != nil {
return errors.Annotatef(err, "An error occurred while splicing CreateTableStmt TableOption: [%v]", i)
Expand Down Expand Up @@ -3573,11 +3574,21 @@ func (n *AlterTableStmt) Restore(ctx *format.RestoreCtx) error {
if err := n.Table.Restore(ctx); err != nil {
return errors.Annotate(err, "An error occurred while restore AlterTableStmt.Table")
}
var specs []*AlterTableSpec
specs := make([]*AlterTableSpec, 0, len(n.Specs))
for _, spec := range n.Specs {
if !(spec.IsAllPlacementRule() && ctx.Flags.HasSkipPlacementRuleForRestoreFlag()) {
specs = append(specs, spec)
if spec.IsAllPlacementRule() && ctx.Flags.HasSkipPlacementRuleForRestoreFlag() {
continue
}
if spec.Tp == AlterTableOption {
newOptions := tableOptionsWithRestoreTTLFlag(ctx.Flags, spec.Options)
if len(newOptions) == 0 {
continue
}
newSpec := *spec
newSpec.Options = newOptions
spec = &newSpec
}
specs = append(specs, spec)
}
for i, spec := range specs {
if i == 0 || spec.Tp == AlterTablePartition || spec.Tp == AlterTableRemovePartitioning || spec.Tp == AlterTableImportTablespace || spec.Tp == AlterTableDiscardTablespace {
Expand Down Expand Up @@ -4510,3 +4521,25 @@ func restorePlacementStmtInSpecialComment(ctx *format.RestoreCtx, n DDLNode) err
return n.Restore(ctx)
})
}

func tableOptionsWithRestoreTTLFlag(flags format.RestoreFlags, options []*TableOption) []*TableOption {
if !flags.HasRestoreWithTTLEnableOff() {
return options
}

newOptions := make([]*TableOption, 0, len(options))
for _, opt := range options {
if opt.Tp == TableOptionTTLEnable {
continue
}

newOptions = append(newOptions, opt)
if opt.Tp == TableOptionTTL {
newOptions = append(newOptions, &TableOption{
Tp: TableOptionTTLEnable,
BoolValue: false,
})
}
}
return newOptions
}
34 changes: 34 additions & 0 deletions parser/ast/ddl_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -869,3 +869,37 @@ func TestTableOptionTTLRestore(t *testing.T) {
runNodeRestoreTestWithFlags(t, testCases, "%s", extractNodeFunc, ca.flags)
}
}

func TestTableOptionTTLRestoreWithTTLEnableOffFlag(t *testing.T) {
sourceSQL1 := "create table t (created_at datetime) ttl = created_at + INTERVAL 1 YEAR"
sourceSQL2 := "alter table t ttl_enable = 'ON'"
sourceSQL3 := "alter table t remove ttl"
sourceSQL4 := "create table t (created_at datetime) ttl = created_at + INTERVAL 1 YEAR ttl_enable = 'ON'"
sourceSQL5 := "alter table t ttl_enable = 'ON' placement policy p1"
cases := []struct {
sourceSQL string
flags format.RestoreFlags
expectSQL string
}{
{sourceSQL1, format.DefaultRestoreFlags | format.RestoreWithTTLEnableOff, "CREATE TABLE `t` (`created_at` DATETIME) TTL = `created_at` + INTERVAL 1 YEAR TTL_ENABLE = 'OFF'"},
{sourceSQL1, format.DefaultRestoreFlags | format.RestoreTiDBSpecialComment | format.RestoreWithTTLEnableOff, "CREATE TABLE `t` (`created_at` DATETIME) /*T![ttl] TTL = `created_at` + INTERVAL 1 YEAR */ /*T![ttl] TTL_ENABLE = 'OFF' */"},
{sourceSQL2, format.DefaultRestoreFlags | format.RestoreWithTTLEnableOff, "ALTER TABLE `t`"},
{sourceSQL2, format.DefaultRestoreFlags | format.RestoreTiDBSpecialComment | format.RestoreWithTTLEnableOff, "ALTER TABLE `t`"},
{sourceSQL3, format.DefaultRestoreFlags | format.RestoreWithTTLEnableOff, "ALTER TABLE `t` REMOVE TTL"},
{sourceSQL3, format.DefaultRestoreFlags | format.RestoreTiDBSpecialComment | format.RestoreWithTTLEnableOff, "ALTER TABLE `t` /*T![ttl] REMOVE TTL */"},
{sourceSQL4, format.DefaultRestoreFlags | format.RestoreWithTTLEnableOff, "CREATE TABLE `t` (`created_at` DATETIME) TTL = `created_at` + INTERVAL 1 YEAR TTL_ENABLE = 'OFF'"},
{sourceSQL4, format.DefaultRestoreFlags | format.RestoreTiDBSpecialComment | format.RestoreWithTTLEnableOff, "CREATE TABLE `t` (`created_at` DATETIME) /*T![ttl] TTL = `created_at` + INTERVAL 1 YEAR */ /*T![ttl] TTL_ENABLE = 'OFF' */"},
{sourceSQL5, format.DefaultRestoreFlags | format.RestoreTiDBSpecialComment | format.RestoreWithTTLEnableOff, "ALTER TABLE `t` /*T![placement] PLACEMENT POLICY = `p1` */"},
}

extractNodeFunc := func(node Node) Node {
return node
}

for _, ca := range cases {
testCases := []NodeRestoreTestCase{
{ca.sourceSQL, ca.expectSQL},
}
runNodeRestoreTestWithFlagsStmtChange(t, testCases, "%s", extractNodeFunc, ca.flags)
}
}
6 changes: 6 additions & 0 deletions parser/format/format.go
Original file line number Diff line number Diff line change
Expand Up @@ -235,6 +235,7 @@ const (

RestoreTiDBSpecialComment
SkipPlacementRuleForRestore
RestoreWithTTLEnableOff
)

const (
Expand Down Expand Up @@ -321,6 +322,11 @@ func (rf RestoreFlags) HasSkipPlacementRuleForRestoreFlag() bool {
return rf.has(SkipPlacementRuleForRestore)
}

// HasRestoreWithTTLEnableOff returns a boolean indicating whether to force set TTL_ENABLE='OFF' when restoring a TTL table
func (rf RestoreFlags) HasRestoreWithTTLEnableOff() bool {
return rf.has(RestoreWithTTLEnableOff)
}

// RestoreCtx is `Restore` context to hold flags and writer.
type RestoreCtx struct {
Flags RestoreFlags
Expand Down

0 comments on commit cf34941

Please sign in to comment.