diff --git a/br/cmd/br/cmd.go b/br/cmd/br/cmd.go index 695d9975717a9..9c73c53177fac 100644 --- a/br/cmd/br/cmd.go +++ b/br/cmd/br/cmd.go @@ -37,7 +37,7 @@ var ( tidbGlue = gluetidb.New() envLogToTermKey = "BR_LOG_TO_TERM" - filterOutSysAndMemTables = []string{ + filterOutSysAndMemKeepAuthAndBind = []string{ "*.*", fmt.Sprintf("!%s.*", utils.TemporaryDBName("*")), "!mysql.*", diff --git a/br/cmd/br/restore.go b/br/cmd/br/restore.go index a37f55b904ca4..5ea2a99333afb 100644 --- a/br/cmd/br/restore.go +++ b/br/cmd/br/restore.go @@ -186,7 +186,7 @@ func newFullRestoreCommand() *cobra.Command { return runRestoreCommand(cmd, task.FullRestoreCmd) }, } - task.DefineFilterFlags(command, filterOutSysAndMemTables, false) + task.DefineFilterFlags(command, filterOutSysAndMemKeepAuthAndBind, false) task.DefineRestoreSnapshotFlags(command) return command } @@ -254,7 +254,7 @@ func newStreamRestoreCommand() *cobra.Command { return runRestoreCommand(command, task.PointRestoreCmd) }, } - task.DefineFilterFlags(command, filterOutSysAndMemTables, true) + task.DefineFilterFlags(command, filterOutSysAndMemKeepAuthAndBind, true) task.DefineStreamRestoreFlags(command) return command } diff --git a/br/pkg/backup/client.go b/br/pkg/backup/client.go index 57b5c4afb526e..2b08f40b03ef0 100644 --- a/br/pkg/backup/client.go +++ b/br/pkg/backup/client.go @@ -692,7 +692,7 @@ func (bc *Client) SetApiVersion(v kvrpcpb.APIVersion) { bc.apiVersion = v } -// Client.BuildBackupRangeAndSchema calls BuildBackupRangeAndSchema, +// BuildBackupRangeAndSchema calls BuildBackupRangeAndInitSchema, // if the checkpoint mode is used, return the ranges from checkpoint meta func (bc *Client) BuildBackupRangeAndSchema( storage kv.Storage, @@ -731,7 +731,7 @@ func CheckBackupStorageIsLocked(ctx context.Context, s storage.ExternalStorage) return nil } -// BuildBackupRangeAndSchema gets KV range and schema of tables. +// BuildBackupRangeAndInitSchema gets KV range and schema of tables. // KV ranges are separated by Table IDs. // Also, KV ranges are separated by Index IDs in the same table. func BuildBackupRangeAndInitSchema( diff --git a/br/pkg/checkpoint/checkpoint_test.go b/br/pkg/checkpoint/checkpoint_test.go index f2534f26c5f52..c7412c40b269e 100644 --- a/br/pkg/checkpoint/checkpoint_test.go +++ b/br/pkg/checkpoint/checkpoint_test.go @@ -105,12 +105,12 @@ func TestCheckpointMetaForRestore(t *testing.T) { exists := checkpoint.ExistsCheckpointProgress(ctx, dom) require.False(t, exists) err = checkpoint.SaveCheckpointProgress(ctx, se, &checkpoint.CheckpointProgress{ - Progress: checkpoint.InLogRestoreAndIdMapPersist, + Progress: checkpoint.InLogRestoreAndIdMapPersisted, }) require.NoError(t, err) progress, err := checkpoint.LoadCheckpointProgress(ctx, se.GetSessionCtx().GetRestrictedSQLExecutor()) require.NoError(t, err) - require.Equal(t, checkpoint.InLogRestoreAndIdMapPersist, progress.Progress) + require.Equal(t, checkpoint.InLogRestoreAndIdMapPersisted, progress.Progress) taskInfo, err := checkpoint.TryToGetCheckpointTaskInfo(ctx, s.Mock.Domain, se.GetSessionCtx().GetRestrictedSQLExecutor()) require.NoError(t, err) @@ -120,7 +120,7 @@ func TestCheckpointMetaForRestore(t *testing.T) { require.Equal(t, uint64(333), taskInfo.Metadata.RewriteTS) require.Equal(t, "1.0", taskInfo.Metadata.GcRatio) require.Equal(t, true, taskInfo.HasSnapshotMetadata) - require.Equal(t, checkpoint.InLogRestoreAndIdMapPersist, taskInfo.Progress) + require.Equal(t, checkpoint.InLogRestoreAndIdMapPersisted, taskInfo.Progress) exists = checkpoint.ExistsCheckpointIngestIndexRepairSQLs(ctx, dom) require.False(t, exists) diff --git a/br/pkg/checkpoint/log_restore.go b/br/pkg/checkpoint/log_restore.go index 15f2f995aec5c..2b21fb82a19c8 100644 --- a/br/pkg/checkpoint/log_restore.go +++ b/br/pkg/checkpoint/log_restore.go @@ -194,14 +194,14 @@ func ExistsLogRestoreCheckpointMetadata( TableExists(ast.NewCIStr(LogRestoreCheckpointDatabaseName), ast.NewCIStr(checkpointMetaTableName)) } -// A progress type for snapshot + log restore. +// RestoreProgress is a progress type for snapshot + log restore. // -// Before the id-maps is persist into external storage, the snapshot restore and -// id-maps constructure can be repeated. So if the progress is in `InSnapshotRestore`, +// Before the id-maps is persisted into external storage, the snapshot restore and +// id-maps building can be retried. So if the progress is in `InSnapshotRestore`, // it can retry from snapshot restore. // -// After the id-maps is persist into external storage, there are some meta-kvs has -// been restored into the cluster, such as `rename ddl`. Where would be a situation: +// After the id-maps is persisted into external storage, there are some meta-kvs has +// been restored into the cluster, such as `rename ddl`. A situation could be: // // the first execution: // @@ -209,7 +209,7 @@ func ExistsLogRestoreCheckpointMetadata( // table A (id 80) --------------> table B (id 80) // ( snapshot restore ) ( log restore ) // -// the second execution if don't skip snasphot restore: +// the second execution if don't skip snapshot restore: // // table A is created again in snapshot restore, because there is no table named A // table A (id 81) --------------> [not in id-maps, so ignored] @@ -221,8 +221,8 @@ type RestoreProgress int const ( InSnapshotRestore RestoreProgress = iota - // Only when the id-maps is persist, status turns into it. - InLogRestoreAndIdMapPersist + // Only when the id-maps is persisted, status turns into it. + InLogRestoreAndIdMapPersisted ) type CheckpointProgress struct { @@ -254,8 +254,8 @@ func ExistsCheckpointProgress( TableExists(ast.NewCIStr(LogRestoreCheckpointDatabaseName), ast.NewCIStr(checkpointProgressTableName)) } -// CheckpointTaskInfo is unique information within the same cluster id. It represents the last -// restore task executed for this cluster. +// CheckpointTaskInfoForLogRestore is tied to a specific cluster. +// It represents the last restore task executed in this cluster. type CheckpointTaskInfoForLogRestore struct { Metadata *CheckpointMetadataForLogRestore HasSnapshotMetadata bool diff --git a/br/pkg/restore/import_mode_switcher.go b/br/pkg/restore/import_mode_switcher.go index 33552ca0734bd..c38b97ba23608 100644 --- a/br/pkg/restore/import_mode_switcher.go +++ b/br/pkg/restore/import_mode_switcher.go @@ -176,7 +176,7 @@ func (switcher *ImportModeSwitcher) GoSwitchToImportMode( return nil } -// RestorePreWork executes some prepare work before restore. +// RestorePreWork switches to import mode and removes pd schedulers if needed // TODO make this function returns a restore post work. func RestorePreWork( ctx context.Context, diff --git a/br/pkg/restore/internal/prealloc_db/db.go b/br/pkg/restore/internal/prealloc_db/db.go index af5cbb30ac402..53bb9cf1dfd85 100644 --- a/br/pkg/restore/internal/prealloc_db/db.go +++ b/br/pkg/restore/internal/prealloc_db/db.go @@ -287,7 +287,7 @@ func (db *DB) canReuseTableID(ti *model.TableInfo) bool { } prealloced := db.preallocedIDs.PreallocedFor(ti) if prealloced { - log.Info("reusing table ID", zap.Stringer("table", ti.Name)) + log.Info("reusing table ID", zap.Stringer("table", ti.Name), zap.Int64("tableID", ti.ID)) } return prealloced } diff --git a/br/pkg/restore/log_client/BUILD.bazel b/br/pkg/restore/log_client/BUILD.bazel index 73b70fcbae739..3705ffef88930 100644 --- a/br/pkg/restore/log_client/BUILD.bazel +++ b/br/pkg/restore/log_client/BUILD.bazel @@ -3,6 +3,7 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") go_library( name = "log_client", srcs = [ + "batch_meta_processor.go", "client.go", "compacted_file_strategy.go", "import.go", @@ -37,6 +38,7 @@ go_library( "//br/pkg/stream", "//br/pkg/summary", "//br/pkg/utils", + "//br/pkg/utils/consts", "//br/pkg/utils/iter", "//br/pkg/version", "//pkg/ddl/util", @@ -49,7 +51,6 @@ go_library( "//pkg/util/codec", "//pkg/util/redact", "//pkg/util/sqlexec", - "//pkg/util/table-filter", "@com_github_docker_go_units//:go-units", "@com_github_fatih_color//:color", "@com_github_gogo_protobuf//proto", @@ -107,6 +108,7 @@ go_test( "//br/pkg/storage", "//br/pkg/stream", "//br/pkg/utils", + "//br/pkg/utils/consts", "//br/pkg/utils/iter", "//br/pkg/utiltest", "//pkg/domain", @@ -121,7 +123,6 @@ go_test( "//pkg/util/chunk", "//pkg/util/codec", "//pkg/util/sqlexec", - "//pkg/util/table-filter", "@com_github_docker_go_units//:go-units", "@com_github_google_uuid//:uuid", "@com_github_pingcap_errors//:errors", diff --git a/br/pkg/restore/log_client/batch_meta_processor.go b/br/pkg/restore/log_client/batch_meta_processor.go new file mode 100644 index 0000000000000..aa4ecd6e695f5 --- /dev/null +++ b/br/pkg/restore/log_client/batch_meta_processor.go @@ -0,0 +1,229 @@ +// Copyright 2024 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 logclient + +import ( + "context" + "encoding/json" + + "github.com/pingcap/errors" + backuppb "github.com/pingcap/kvproto/pkg/brpb" + "github.com/pingcap/log" + "github.com/pingcap/tidb/br/pkg/stream" + "github.com/pingcap/tidb/br/pkg/utils" + "github.com/pingcap/tidb/pkg/meta" + "github.com/pingcap/tidb/pkg/meta/model" + "go.uber.org/zap" +) + +// BatchMetaKVProcessor defines how to process a batch of files +type BatchMetaKVProcessor interface { + // ProcessBatch processes a batch of files and with a filterTS and return what's not processed for next iteration + ProcessBatch( + ctx context.Context, + files []*backuppb.DataFileInfo, + entries []*KvEntryWithTS, + filterTS uint64, + cf string, + ) ([]*KvEntryWithTS, error) +} + +// RestoreMetaKVProcessor implements BatchMetaKVProcessor for restoring files in batches +type RestoreMetaKVProcessor struct { + client *LogClient + schemasReplace *stream.SchemasReplace + updateStats func(kvCount uint64, size uint64) + progressInc func() +} + +func NewRestoreMetaKVProcessor(client *LogClient, schemasReplace *stream.SchemasReplace, + updateStats func(kvCount uint64, size uint64), + progressInc func()) *RestoreMetaKVProcessor { + return &RestoreMetaKVProcessor{ + client: client, + schemasReplace: schemasReplace, + updateStats: updateStats, + progressInc: progressInc, + } +} + +// RestoreAndRewriteMetaKVFiles tries to restore files about meta kv-event from stream-backup. +func (rp *RestoreMetaKVProcessor) RestoreAndRewriteMetaKVFiles( + ctx context.Context, + files []*backuppb.DataFileInfo, +) error { + // starts gc row collector + rp.client.RunGCRowsLoader(ctx) + + // separate the files by CF and sort each group by TS + filesInDefaultCF, filesInWriteCF := SeparateAndSortFilesByCF(files) + + log.Info("start to restore meta files", + zap.Int("total files", len(files)), + zap.Int("default files", len(filesInDefaultCF)), + zap.Int("write files", len(filesInWriteCF))) + + if err := LoadAndProcessMetaKVFilesInBatch( + ctx, + filesInDefaultCF, + filesInWriteCF, + rp, + ); err != nil { + return errors.Trace(err) + } + + // AddPhysicalId global schema version to trigger a full reload so every TiDB node in the cluster will get synced with + // the latest schema update. + if err := rp.client.UpdateSchemaVersionFullReload(ctx); err != nil { + return errors.Trace(err) + } + return nil +} + +func (rp *RestoreMetaKVProcessor) ProcessBatch( + ctx context.Context, + files []*backuppb.DataFileInfo, + entries []*KvEntryWithTS, + filterTS uint64, + cf string, +) ([]*KvEntryWithTS, error) { + return rp.client.RestoreBatchMetaKVFiles( + ctx, files, rp.schemasReplace, entries, + filterTS, rp.updateStats, rp.progressInc, cf, + ) +} + +// MetaKVInfoProcessor implements BatchMetaKVProcessor to iterate meta kv and collect information. +// +// 1. It collects table renaming information. The table rename operation will not change the table id, and the process +// will drop the original table and create a new one with the same table id, so in DDL history there will be two events +// that corresponds to the same table id. +// +// 2. It builds the id mapping from upstream to downstream. This logic was nested into table rewrite previously and now +// separated out to its own component. +type MetaKVInfoProcessor struct { + client *LogClient + tableHistoryManager *stream.LogBackupTableHistoryManager + tableMappingManager *stream.TableMappingManager +} + +func NewMetaKVInfoProcessor(client *LogClient) *MetaKVInfoProcessor { + return &MetaKVInfoProcessor{ + client: client, + tableHistoryManager: stream.NewTableHistoryManager(), + tableMappingManager: stream.NewTableMappingManager(), + } +} + +func (mp *MetaKVInfoProcessor) ReadMetaKVFilesAndBuildInfo( + ctx context.Context, + files []*backuppb.DataFileInfo, +) error { + // separate the files by CF and sort each group by TS + filesInDefaultCF, filesInWriteCF := SeparateAndSortFilesByCF(files) + + if err := LoadAndProcessMetaKVFilesInBatch( + ctx, + filesInDefaultCF, + filesInWriteCF, + mp, + ); err != nil { + return errors.Trace(err) + } + return nil +} + +func (mp *MetaKVInfoProcessor) ProcessBatch( + ctx context.Context, + files []*backuppb.DataFileInfo, + entries []*KvEntryWithTS, + filterTS uint64, + cf string, +) ([]*KvEntryWithTS, error) { + curSortedEntries, filteredEntries, err := mp.client.filterAndSortKvEntriesFromFiles(ctx, files, entries, filterTS) + if err != nil { + return nil, errors.Trace(err) + } + + // process entries to collect table IDs + for _, entry := range curSortedEntries { + // parse entry and do the table mapping + if err = mp.tableMappingManager.ParseMetaKvAndUpdateIdMapping(&entry.E, cf); err != nil { + return nil, errors.Trace(err) + } + + // collect rename/partition exchange history + // get value from default cf and get the short value if possible from write cf + value, err := stream.ExtractValue(&entry.E, cf) + if err != nil { + return nil, errors.Trace(err) + } + + // write cf doesn't have short value in it + if value == nil { + continue + } + + if utils.IsMetaDBKey(entry.E.Key) { + rawKey, err := stream.ParseTxnMetaKeyFrom(entry.E.Key) + if err != nil { + return nil, errors.Trace(err) + } + + if meta.IsDBkey(rawKey.Field) { + var dbInfo model.DBInfo + if err := json.Unmarshal(value, &dbInfo); err != nil { + return nil, errors.Trace(err) + } + // collect db id -> name mapping during log backup, it will contain information about newly created db + mp.tableHistoryManager.RecordDBIdToName(dbInfo.ID, dbInfo.Name.O) + } else if !meta.IsDBkey(rawKey.Key) { + // also see RewriteMetaKvEntry + continue + } else if meta.IsTableKey(rawKey.Field) { + // collect table history indexed by table id, same id may have different table names in history + var tableInfo model.TableInfo + if err := json.Unmarshal(value, &tableInfo); err != nil { + return nil, errors.Trace(err) + } + // cannot use dbib in the parsed table info cuz it might not set so default to 0 + dbID, err := meta.ParseDBKey(rawKey.Key) + if err != nil { + return nil, errors.Trace(err) + } + + // add to table rename history + mp.tableHistoryManager.AddTableHistory(tableInfo.ID, tableInfo.Name.String(), dbID) + + // track partition information if table is partitioned + // this is needed since a table might get exchanged + if tableInfo.Partition != nil { + for _, def := range tableInfo.Partition.Definitions { + mp.tableHistoryManager.AddPartitionHistory(def.ID, dbID, tableInfo.ID, tableInfo.Name.O) + } + } + } + } + } + return filteredEntries, nil +} + +func (mp *MetaKVInfoProcessor) GetTableMappingManager() *stream.TableMappingManager { + return mp.tableMappingManager +} + +func (mp *MetaKVInfoProcessor) GetTableHistoryManager() *stream.LogBackupTableHistoryManager { + return mp.tableHistoryManager +} diff --git a/br/pkg/restore/log_client/client.go b/br/pkg/restore/log_client/client.go index 41f86f16ac084..0abe0054be107 100644 --- a/br/pkg/restore/log_client/client.go +++ b/br/pkg/restore/log_client/client.go @@ -58,6 +58,7 @@ import ( "github.com/pingcap/tidb/br/pkg/stream" "github.com/pingcap/tidb/br/pkg/summary" "github.com/pingcap/tidb/br/pkg/utils" + "github.com/pingcap/tidb/br/pkg/utils/consts" "github.com/pingcap/tidb/br/pkg/utils/iter" "github.com/pingcap/tidb/br/pkg/version" ddlutil "github.com/pingcap/tidb/pkg/ddl/util" @@ -67,7 +68,6 @@ import ( "github.com/pingcap/tidb/pkg/meta/model" tidbutil "github.com/pingcap/tidb/pkg/util" "github.com/pingcap/tidb/pkg/util/sqlexec" - filter "github.com/pingcap/tidb/pkg/util/table-filter" "github.com/tikv/client-go/v2/config" kvutil "github.com/tikv/client-go/v2/util" pd "github.com/tikv/pd/client" @@ -231,8 +231,8 @@ type restoreStatistics struct { restoreSSTTakes uint64 } -// NewRestoreClient returns a new RestoreClient. -func NewRestoreClient( +// NewLogClient returns a new LogClient. +func NewLogClient( pdClient pd.Client, pdHTTPCli pdhttp.Client, tlsConf *tls.Config, @@ -250,24 +250,27 @@ func NewRestoreClient( // Close a client. func (rc *LogClient) Close(ctx context.Context) { - defer func() { - if rc.logRestoreManager != nil { - rc.logRestoreManager.Close(ctx) - } - if rc.sstRestoreManager != nil { - rc.sstRestoreManager.Close(ctx) - } - }() - - // close the connection, and it must be succeed when in SQL mode. + // close the connection, and it must be succeeded when in SQL mode. if rc.unsafeSession != nil { rc.unsafeSession.Close() } + if rc.LogFileManager != nil { + rc.LogFileManager.Close() + } + if rc.rawKVClient != nil { rc.rawKVClient.Close() } - log.Info("Restore client closed") + + if rc.logRestoreManager != nil { + rc.logRestoreManager.Close(ctx) + } + + if rc.sstRestoreManager != nil { + rc.sstRestoreManager.Close(ctx) + } + log.Info("Log client closed") } func (rc *LogClient) rewriteRulesFor(sst SSTs, rules *restoreutils.RewriteRules) (*restoreutils.RewriteRules, error) { @@ -551,7 +554,7 @@ func (rc *LogClient) InitCheckpointMetadataForCompactedSstRestore( return sstCheckpointSets, nil } -func (rc *LogClient) InitCheckpointMetadataForLogRestore( +func (rc *LogClient) LoadOrCreateCheckpointMetadataForLogRestore( ctx context.Context, startTS, restoredTS uint64, gcRatio string, @@ -568,7 +571,8 @@ func (rc *LogClient) InitCheckpointMetadataForLogRestore( return "", errors.Trace(err) } - log.Info("reuse gc ratio from checkpoint metadata", zap.String("gc-ratio", gcRatio)) + log.Info("reuse gc ratio from checkpoint metadata", zap.String("old-gc-ratio", gcRatio), + zap.String("checkpoint-gc-ratio", meta.GcRatio)) return meta.GcRatio, nil } @@ -702,7 +706,7 @@ func ApplyKVFilesWithBatchMethod( } fs.deleteFiles = append(fs.deleteFiles, f) } else { - if f.GetCf() == stream.DefaultCF { + if f.GetCf() == consts.DefaultCF { if fs.defaultFiles == nil { fs.defaultFiles = make([]*LogDataFileInfo, 0, batchCount) } @@ -827,6 +831,7 @@ func (rc *LogClient) RestoreKVFiles( } }() + log.Info("starting to restore kv files") if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil { span1 := span.Tracer().StartSpan("Client.RestoreKVFiles", opentracing.ChildOf(span.Context())) defer span1.Finish() @@ -843,10 +848,6 @@ func (rc *LogClient) RestoreKVFiles( // because the tableID of files is the same. rule, ok := rules[files[0].TableId] if !ok { - // TODO handle new created table - // For this version we do not handle new created table after full backup. - // in next version we will perform rewrite and restore meta key to restore new created tables. - // so we can simply skip the file that doesn't have the rule here. onProgress(kvCount) summary.CollectInt("FileSkip", len(files)) log.Debug("skip file due to table id not matched", zap.Int64("table-id", files[0].TableId)) @@ -905,7 +906,7 @@ func (rc *LogClient) RestoreKVFiles( return errors.Trace(err) } -func (rc *LogClient) initSchemasMap( +func (rc *LogClient) loadSchemasMap( ctx context.Context, restoreTS uint64, ) ([]*backuppb.PitrDBMap, error) { @@ -945,12 +946,15 @@ func (rc *LogClient) initSchemasMap( return backupMeta.GetDbMaps(), nil } -func initFullBackupTables( +func readFilteredFullBackupTables( ctx context.Context, s storage.ExternalStorage, - tableFilter filter.Filter, + piTRIdTracker *utils.PiTRIdTracker, cipherInfo *backuppb.CipherInfo, ) (map[int64]*metautil.Table, error) { + if piTRIdTracker == nil { + return nil, errors.Errorf("missing pitr table tracker information") + } metaData, err := s.ReadFile(ctx, metautil.MetaFile) if err != nil { return nil, errors.Trace(err) @@ -976,25 +980,31 @@ func initFullBackupTables( tables := make(map[int64]*metautil.Table) for _, db := range databases { - dbName := db.Info.Name.O - if name, ok := utils.GetSysDBName(db.Info.Name); utils.IsSysDB(name) && ok { - dbName = name - } - - if !tableFilter.MatchSchema(dbName) { + if !piTRIdTracker.ContainsDB(db.Info.ID) { continue } + tableAdded := false for _, table := range db.Tables { // check this db is empty. if table.Info == nil { tables[db.Info.ID] = table + tableAdded = true continue } - if !tableFilter.MatchTable(dbName, table.Info.Name.O) { + if !piTRIdTracker.ContainsPhysicalId(db.Info.ID, table.Info.ID) { continue } tables[table.Info.ID] = table + tableAdded = true + } + // all tables in this db are filtered out, but we still need to keep this db since it passed the filter check + // and tables might get created later during log backup, if not keeping this db, those tables will be mapped to + // a new db id and thus will become data corruption. + if !tableAdded { + tables[db.Info.ID] = &metautil.Table{ + DB: db.Info, + } } } @@ -1006,25 +1016,28 @@ type FullBackupStorageConfig struct { Opts *storage.ExternalStorageOptions } -type BuildTableMappingManagerConfig struct { +type GetIDMapConfig struct { // required - CurrentIdMapSaved bool - TableFilter filter.Filter + LoadSavedIDMap bool // optional - FullBackupStorage *FullBackupStorageConfig - CipherInfo *backuppb.CipherInfo - Files []*backuppb.DataFileInfo + FullBackupStorageConfig *FullBackupStorageConfig + CipherInfo *backuppb.CipherInfo + // generated at full restore step that contains all the table ids that need to restore + PiTRTableTracker *utils.PiTRIdTracker } const UnsafePITRLogRestoreStartBeforeAnyUpstreamUserDDL = "UNSAFE_PITR_LOG_RESTORE_START_BEFORE_ANY_UPSTREAM_USER_DDL" +// generateDBReplacesFromFullBackupStorage reads the full backup schema and creates the mapping from upstream table id +// to downstream table id. The downstream tables have been created in the previous snapshot restore step, so we +// can build the mapping by looking at the table names. The current table information is in domain.InfoSchema. func (rc *LogClient) generateDBReplacesFromFullBackupStorage( ctx context.Context, - cfg *BuildTableMappingManagerConfig, + cfg *GetIDMapConfig, ) (map[stream.UpstreamID]*stream.DBReplace, error) { dbReplaces := make(map[stream.UpstreamID]*stream.DBReplace) - if cfg.FullBackupStorage == nil { + if cfg.FullBackupStorageConfig == nil { envVal, ok := os.LookupEnv(UnsafePITRLogRestoreStartBeforeAnyUpstreamUserDDL) if ok && len(envVal) > 0 { log.Info(fmt.Sprintf("the environment variable %s is active, skip loading the base schemas.", UnsafePITRLogRestoreStartBeforeAnyUpstreamUserDDL)) @@ -1032,19 +1045,19 @@ func (rc *LogClient) generateDBReplacesFromFullBackupStorage( } return nil, errors.Errorf("miss upstream table information at `start-ts`(%d) but the full backup path is not specified", rc.startTS) } - s, err := storage.New(ctx, cfg.FullBackupStorage.Backend, cfg.FullBackupStorage.Opts) + s, err := storage.New(ctx, cfg.FullBackupStorageConfig.Backend, cfg.FullBackupStorageConfig.Opts) if err != nil { return nil, errors.Trace(err) } - fullBackupTables, err := initFullBackupTables(ctx, s, cfg.TableFilter, cfg.CipherInfo) + filteredFullBackupTables, err := readFilteredFullBackupTables(ctx, s, cfg.PiTRTableTracker, cfg.CipherInfo) if err != nil { return nil, errors.Trace(err) } - for _, t := range fullBackupTables { + for _, t := range filteredFullBackupTables { dbName, _ := utils.GetSysDBCIStrName(t.DB.Name) newDBInfo, exist := rc.dom.InfoSchema().SchemaByName(dbName) if !exist { - log.Info("db not existed", zap.String("dbname", dbName.String())) + log.Info("db does not exist", zap.String("dbName", dbName.String())) continue } @@ -1060,7 +1073,7 @@ func (rc *LogClient) generateDBReplacesFromFullBackupStorage( } newTableInfo, err := restore.GetTableSchema(rc.GetDomain(), dbName, t.Info.Name) if err != nil { - log.Info("table not existed", zap.String("tablename", dbName.String()+"."+t.Info.Name.String())) + log.Info("table doesn't exist", zap.String("tableName", dbName.String()+"."+t.Info.Name.String())) continue } @@ -1074,26 +1087,25 @@ func (rc *LogClient) generateDBReplacesFromFullBackupStorage( return dbReplaces, nil } -// BuildTableMappingManager builds the table mapping manager. It reads the full backup storage to get the full backup -// table info to initialize the manager, or it reads the id map from previous task, -// or it loads the saved mapping from last time of run of the same task. -func (rc *LogClient) BuildTableMappingManager( +// GetBaseIDMap get the id map from following ways +// 1. from previously saved id map if the same task has been running and built/saved id map already but failed later +// 2. from previous different task. A PiTR job might be split into multiple runs/tasks and each task only restores +// a subset of the entire job. +// 3. from full backup snapshot if specified. +func (rc *LogClient) GetBaseIDMap( ctx context.Context, - cfg *BuildTableMappingManagerConfig, -) (*stream.TableMappingManager, error) { + cfg *GetIDMapConfig, +) (map[stream.UpstreamID]*stream.DBReplace, error) { var ( - err error - dbMaps []*backuppb.PitrDBMap - // the id map doesn't need to construct only when it is not the first execution - needConstructIdMap bool - dbReplaces map[stream.UpstreamID]*stream.DBReplace + err error + dbMaps []*backuppb.PitrDBMap + dbReplaces map[stream.UpstreamID]*stream.DBReplace ) // this is a retry, id map saved last time, load it from external storage - if cfg.CurrentIdMapSaved { + if cfg.LoadSavedIDMap { log.Info("try to load previously saved pitr id maps") - needConstructIdMap = false - dbMaps, err = rc.initSchemasMap(ctx, rc.restoreTS) + dbMaps, err = rc.loadSchemasMap(ctx, rc.restoreTS) if err != nil { return nil, errors.Trace(err) } @@ -1101,28 +1113,20 @@ func (rc *LogClient) BuildTableMappingManager( // a new task, but without full snapshot restore, tries to load // schemas map whose `restore-ts`` is the task's `start-ts`. - if len(dbMaps) <= 0 && cfg.FullBackupStorage == nil { + if len(dbMaps) <= 0 && cfg.FullBackupStorageConfig == nil { log.Info("try to load pitr id maps of the previous task", zap.Uint64("start-ts", rc.startTS)) - needConstructIdMap = true - dbMaps, err = rc.initSchemasMap(ctx, rc.startTS) + dbMaps, err = rc.loadSchemasMap(ctx, rc.startTS) if err != nil { return nil, errors.Trace(err) } - existTiFlashTable := false - rc.dom.InfoSchema().ListTablesWithSpecialAttribute(func(tableInfo *model.TableInfo) bool { - if tableInfo.TiFlashReplica != nil && tableInfo.TiFlashReplica.Count > 0 { - existTiFlashTable = true - } - return false - }) - if existTiFlashTable { - return nil, errors.Errorf("exist table(s) have tiflash replica, please remove it before restore") + err := rc.validateNoTiFlashReplica() + if err != nil { + return nil, errors.Trace(err) } } if len(dbMaps) <= 0 { log.Info("no id maps, build the table replaces from cluster and full backup schemas") - needConstructIdMap = true dbReplaces, err = rc.generateDBReplacesFromFullBackupStorage(ctx, cfg) if err != nil { return nil, errors.Trace(err) @@ -1131,33 +1135,9 @@ func (rc *LogClient) BuildTableMappingManager( dbReplaces = stream.FromDBMapProto(dbMaps) } - for oldDBID, dbReplace := range dbReplaces { - log.Info("base replace info", func() []zapcore.Field { - fields := make([]zapcore.Field, 0, (len(dbReplace.TableMap)+1)*3) - fields = append(fields, - zap.String("dbName", dbReplace.Name), - zap.Int64("oldID", oldDBID), - zap.Int64("newID", dbReplace.DbID)) - for oldTableID, tableReplace := range dbReplace.TableMap { - fields = append(fields, - zap.String("table", tableReplace.Name), - zap.Int64("oldID", oldTableID), - zap.Int64("newID", tableReplace.TableID)) - } - return fields - }()...) - } - - tableMappingManager := stream.NewTableMappingManager(dbReplaces, rc.GenGlobalID) - - // not loaded from previously saved, need to iter meta kv and build and save the map - if needConstructIdMap { - if err = rc.IterMetaKVToBuildAndSaveIdMap(ctx, tableMappingManager, cfg.Files); err != nil { - return nil, errors.Trace(err) - } - } + stream.LogDBReplaceMap("base db replace info", dbReplaces) - return tableMappingManager, nil + return dbReplaces, nil } func SortMetaKVFiles(files []*backuppb.DataFileInfo) []*backuppb.DataFileInfo { @@ -1173,14 +1153,24 @@ func SortMetaKVFiles(files []*backuppb.DataFileInfo) []*backuppb.DataFileInfo { return files } -// RestoreAndRewriteMetaKVFiles tries to restore files about meta kv-event from stream-backup. -func (rc *LogClient) RestoreAndRewriteMetaKVFiles( - ctx context.Context, - files []*backuppb.DataFileInfo, - schemasReplace *stream.SchemasReplace, - updateStats func(kvCount uint64, size uint64), - progressInc func(), -) error { +// validateNoTiFlashReplica makes sure no table contains TiFlash replica +func (rc *LogClient) validateNoTiFlashReplica() error { + existTiFlashTable := false + rc.dom.InfoSchema().ListTablesWithSpecialAttribute(func(tableInfo *model.TableInfo) bool { + if tableInfo.TiFlashReplica != nil && tableInfo.TiFlashReplica.Count > 0 { + existTiFlashTable = true + } + return false + }) + if existTiFlashTable { + return errors.Errorf("exist table(s) have tiflash replica, please remove it before restore") + } + return nil +} + +// SeparateAndSortFilesByCF filters and sorts files by column family. +// It separates files into write CF and default CF groups and then sorts them within each CF group. +func SeparateAndSortFilesByCF(files []*backuppb.DataFileInfo) ([]*backuppb.DataFileInfo, []*backuppb.DataFileInfo) { filesInWriteCF := make([]*backuppb.DataFileInfo, 0, len(files)) filesInDefaultCF := make([]*backuppb.DataFileInfo, 0, len(files)) @@ -1188,70 +1178,15 @@ func (rc *LogClient) RestoreAndRewriteMetaKVFiles( // The error of transactions of meta could happen if restore write CF events successfully, // but failed to restore default CF events. for _, f := range files { - if f.Cf == stream.WriteCF { + if f.Cf == consts.WriteCF { filesInWriteCF = append(filesInWriteCF, f) continue } if f.Type == backuppb.FileType_Delete { - // this should happen abnormally. - // only do some preventive checks here. - log.Warn("detected delete file of meta key, skip it", zap.Any("file", f)) - continue - } - if f.Cf == stream.DefaultCF { - filesInDefaultCF = append(filesInDefaultCF, f) - } - } - filesInDefaultCF = SortMetaKVFiles(filesInDefaultCF) - filesInWriteCF = SortMetaKVFiles(filesInWriteCF) - - log.Info("start to restore meta files", - zap.Int("total files", len(files)), - zap.Int("default files", len(filesInDefaultCF)), - zap.Int("write files", len(filesInWriteCF))) - - // run the rewrite and restore meta-kv into TiKV cluster. - if err := RestoreMetaKVFilesWithBatchMethod( - ctx, - filesInDefaultCF, - filesInWriteCF, - schemasReplace, - updateStats, - progressInc, - rc.RestoreBatchMetaKVFiles, - ); err != nil { - return errors.Trace(err) - } - - // Update global schema version and report all of TiDBs. - if err := rc.UpdateSchemaVersion(ctx); err != nil { - return errors.Trace(err) - } - return nil -} - -// IterMetaKVToBuildAndSaveIdMap iterates meta kv and builds id mapping and saves it to storage. -func (rc *LogClient) IterMetaKVToBuildAndSaveIdMap( - ctx context.Context, - tableMappingManager *stream.TableMappingManager, - files []*backuppb.DataFileInfo, -) error { - filesInDefaultCF := make([]*backuppb.DataFileInfo, 0, len(files)) - // need to look at write cf for "short value", which inlines the actual values without redirecting to default cf - filesInWriteCF := make([]*backuppb.DataFileInfo, 0, len(files)) - - for _, f := range files { - if f.Type == backuppb.FileType_Delete { - // it should not happen - // only do some preventive checks here. log.Warn("internal error: detected delete file of meta key, skip it", zap.Any("file", f)) continue } - if f.Cf == stream.WriteCF { - filesInWriteCF = append(filesInWriteCF, f) - continue - } - if f.Cf == stream.DefaultCF { + if f.Cf == consts.DefaultCF { filesInDefaultCF = append(filesInDefaultCF, f) } } @@ -1259,88 +1194,16 @@ func (rc *LogClient) IterMetaKVToBuildAndSaveIdMap( filesInDefaultCF = SortMetaKVFiles(filesInDefaultCF) filesInWriteCF = SortMetaKVFiles(filesInWriteCF) - failpoint.Inject("failed-before-id-maps-saved", func(_ failpoint.Value) { - failpoint.Return(errors.New("failpoint: failed before id maps saved")) - }) - - log.Info("start to iterate meta kv and build id map", - zap.Int("total files", len(files)), - zap.Int("default files", len(filesInDefaultCF)), - zap.Int("write files", len(filesInWriteCF))) - - // build the map and save it into external storage. - if err := rc.buildAndSaveIDMap( - ctx, - filesInDefaultCF, - filesInWriteCF, - tableMappingManager, - ); err != nil { - return errors.Trace(err) - } - failpoint.Inject("failed-after-id-maps-saved", func(_ failpoint.Value) { - failpoint.Return(errors.New("failpoint: failed after id maps saved")) - }) - return nil -} - -// buildAndSaveIDMap build id mapping and save it. -func (rc *LogClient) buildAndSaveIDMap( - ctx context.Context, - fsInDefaultCF []*backuppb.DataFileInfo, - fsInWriteCF []*backuppb.DataFileInfo, - tableMappingManager *stream.TableMappingManager, -) error { - if err := rc.iterAndBuildIDMap(ctx, fsInWriteCF, tableMappingManager); err != nil { - return errors.Trace(err) - } - - if err := rc.iterAndBuildIDMap(ctx, fsInDefaultCF, tableMappingManager); err != nil { - return errors.Trace(err) - } - - if err := rc.saveIDMap(ctx, tableMappingManager); err != nil { - return errors.Trace(err) - } - return nil -} - -func (rc *LogClient) iterAndBuildIDMap( - ctx context.Context, - fs []*backuppb.DataFileInfo, - tableMappingManager *stream.TableMappingManager, -) error { - for _, f := range fs { - entries, _, err := rc.ReadAllEntries(ctx, f, math.MaxUint64) - if err != nil { - return errors.Trace(err) - } - - for _, entry := range entries { - if err := tableMappingManager.ParseMetaKvAndUpdateIdMapping(&entry.E, f.GetCf()); err != nil { - return errors.Trace(err) - } - } - } - return nil + return filesInDefaultCF, filesInWriteCF } -func RestoreMetaKVFilesWithBatchMethod( +// LoadAndProcessMetaKVFilesInBatch restores meta kv files to TiKV in strict TS order. It does so in batch and after +// success it triggers an update so every TiDB node can pick up the restored content. +func LoadAndProcessMetaKVFilesInBatch( ctx context.Context, defaultFiles []*backuppb.DataFileInfo, writeFiles []*backuppb.DataFileInfo, - schemasReplace *stream.SchemasReplace, - updateStats func(kvCount uint64, size uint64), - progressInc func(), - restoreBatch func( - ctx context.Context, - files []*backuppb.DataFileInfo, - schemasReplace *stream.SchemasReplace, - kvEntries []*KvEntryWithTS, - filterTS uint64, - updateStats func(kvCount uint64, size uint64), - progressInc func(), - cf string, - ) ([]*KvEntryWithTS, error), + processor BatchMetaKVProcessor, ) error { // the average size of each KV is 2560 Bytes // kvEntries is kvs left by the previous batch @@ -1371,7 +1234,7 @@ func RestoreMetaKVFilesWithBatchMethod( } else { // Either f.MinTS > rangeMax or f.MinTs is the filterTs we need. // So it is ok to pass f.MinTs as filterTs. - defaultKvEntries, err = restoreBatch(ctx, defaultFiles[defaultIdx:i], schemasReplace, defaultKvEntries, f.MinTs, updateStats, progressInc, stream.DefaultCF) + defaultKvEntries, err = processor.ProcessBatch(ctx, defaultFiles[defaultIdx:i], defaultKvEntries, f.MinTs, consts.DefaultCF) if err != nil { return errors.Trace(err) } @@ -1388,7 +1251,7 @@ func RestoreMetaKVFilesWithBatchMethod( break } } - writeKvEntries, err = restoreBatch(ctx, writeFiles[writeIdx:toWriteIdx], schemasReplace, writeKvEntries, f.MinTs, updateStats, progressInc, stream.WriteCF) + writeKvEntries, err = processor.ProcessBatch(ctx, writeFiles[writeIdx:toWriteIdx], writeKvEntries, f.MinTs, consts.WriteCF) if err != nil { return errors.Trace(err) } @@ -1400,11 +1263,11 @@ func RestoreMetaKVFilesWithBatchMethod( // restore the left meta kv files and entries // Notice: restoreBatch needs to realize the parameter `files` and `kvEntries` might be empty // Assert: defaultIdx <= len(defaultFiles) && writeIdx <= len(writeFiles) - _, err = restoreBatch(ctx, defaultFiles[defaultIdx:], schemasReplace, defaultKvEntries, math.MaxUint64, updateStats, progressInc, stream.DefaultCF) + _, err = processor.ProcessBatch(ctx, defaultFiles[defaultIdx:], defaultKvEntries, math.MaxUint64, consts.DefaultCF) if err != nil { return errors.Trace(err) } - _, err = restoreBatch(ctx, writeFiles[writeIdx:], schemasReplace, writeKvEntries, math.MaxUint64, updateStats, progressInc, stream.WriteCF) + _, err = processor.ProcessBatch(ctx, writeFiles[writeIdx:], writeKvEntries, math.MaxUint64, consts.WriteCF) if err != nil { return errors.Trace(err) } @@ -1412,6 +1275,9 @@ func RestoreMetaKVFilesWithBatchMethod( return nil } +// RestoreBatchMetaKVFiles tries to restore and rewrite meta kv to TiKV from external storage. It reads out entries +// from the given files and only restores ones that's in filter range, then it returns those entries out of the filter +// range back to caller for next iteration of restore. func (rc *LogClient) RestoreBatchMetaKVFiles( ctx context.Context, files []*backuppb.DataFileInfo, @@ -1422,10 +1288,38 @@ func (rc *LogClient) RestoreBatchMetaKVFiles( progressInc func(), cf string, ) ([]*KvEntryWithTS, error) { - nextKvEntries := make([]*KvEntryWithTS, 0) + curSortedKvEntries, filteredOutKvEntries, err := rc.filterAndSortKvEntriesFromFiles(ctx, files, kvEntries, filterTS) + if err != nil { + return nil, errors.Trace(err) + } + + if len(curSortedKvEntries) == 0 { + return filteredOutKvEntries, nil + } + + // restore and rewrite these entries to TiKV with rawPut() method. + kvCount, size, err := rc.restoreAndRewriteMetaKvEntries(ctx, schemasReplace, curSortedKvEntries, cf) + if err != nil { + return nil, errors.Trace(err) + } + + updateStats(kvCount, size) + for i := 0; i < len(files); i++ { + progressInc() + } + return filteredOutKvEntries, nil +} + +func (rc *LogClient) filterAndSortKvEntriesFromFiles( + ctx context.Context, + files []*backuppb.DataFileInfo, + kvEntries []*KvEntryWithTS, + filterTS uint64, +) ([]*KvEntryWithTS, []*KvEntryWithTS, error) { + filteredOutKvEntries := make([]*KvEntryWithTS, 0) curKvEntries := make([]*KvEntryWithTS, 0) if len(files) == 0 && len(kvEntries) == 0 { - return nextKvEntries, nil + return curKvEntries, filteredOutKvEntries, nil } // filter the kv from kvEntries again. @@ -1433,40 +1327,29 @@ func (rc *LogClient) RestoreBatchMetaKVFiles( if kv.Ts < filterTS { curKvEntries = append(curKvEntries, kv) } else { - nextKvEntries = append(nextKvEntries, kv) + filteredOutKvEntries = append(filteredOutKvEntries, kv) } } - // read all of entries from files. + // read all entries from files. for _, f := range files { - es, nextEs, err := rc.ReadAllEntries(ctx, f, filterTS) + es, filteredOutEs, err := rc.ReadFilteredEntriesFromFiles(ctx, f, filterTS) if err != nil { - return nextKvEntries, errors.Trace(err) + return nil, nil, errors.Trace(err) } curKvEntries = append(curKvEntries, es...) - nextKvEntries = append(nextKvEntries, nextEs...) + filteredOutKvEntries = append(filteredOutKvEntries, filteredOutEs...) } // sort these entries. slices.SortFunc(curKvEntries, func(i, j *KvEntryWithTS) int { return cmp.Compare(i.Ts, j.Ts) }) - - // restore these entries with rawPut() method. - kvCount, size, err := rc.restoreMetaKvEntries(ctx, schemasReplace, curKvEntries, cf) - if err != nil { - return nextKvEntries, errors.Trace(err) - } - - updateStats(kvCount, size) - for i := 0; i < len(files); i++ { - progressInc() - } - return nextKvEntries, nil + return curKvEntries, filteredOutKvEntries, nil } -func (rc *LogClient) restoreMetaKvEntries( +func (rc *LogClient) restoreAndRewriteMetaKvEntries( ctx context.Context, sr *stream.SchemasReplace, entries []*KvEntryWithTS, @@ -1480,10 +1363,10 @@ func (rc *LogClient) restoreMetaKvEntries( rc.rawKVClient.SetColumnFamily(columnFamily) for _, entry := range entries { - log.Debug("before rewrte entry", zap.Uint64("key-ts", entry.Ts), zap.Int("key-len", len(entry.E.Key)), + log.Debug("before rewriting entry", zap.Uint64("key-ts", entry.Ts), zap.Int("key-len", len(entry.E.Key)), zap.Int("value-len", len(entry.E.Value)), zap.ByteString("key", entry.E.Key)) - newEntry, err := sr.RewriteKvEntry(&entry.E, columnFamily) + newEntry, err := sr.RewriteMetaKvEntry(&entry.E, columnFamily) if err != nil { log.Error("rewrite txn entry failed", zap.Int("klen", len(entry.E.Key)), logutil.Key("txn-key", entry.E.Key)) @@ -1553,8 +1436,8 @@ func (rc *LogClient) GenGlobalIDs(ctx context.Context, n int) ([]int64, error) { return ids, err } -// UpdateSchemaVersion updates schema version by transaction way. -func (rc *LogClient) UpdateSchemaVersion(ctx context.Context) error { +// UpdateSchemaVersionFullReload updates schema version to trigger a full reload in transaction way. +func (rc *LogClient) UpdateSchemaVersionFullReload(ctx context.Context) error { storage := rc.GetDomain().Store() var schemaVersion int64 @@ -1620,7 +1503,7 @@ func (rc *LogClient) WrapCompactedFilesIterWithSplitHelper( return wrapper.WithSplit(ctx, compactedIter, strategy), nil } -// WrapLogFilesIteratorWithSplit applies a splitting strategy to the log files iterator. +// WrapLogFilesIterWithSplitHelper applies a splitting strategy to the log files iterator. // It uses a region splitter to handle the splitting logic based on the provided rules. func (rc *LogClient) WrapLogFilesIterWithSplitHelper( ctx context.Context, @@ -1909,6 +1792,24 @@ func (rc *LogClient) GetGCRows() []*stream.PreDelRangeQuery { const PITRIdMapBlockSize int = 524288 +func (rc *LogClient) SaveIdMapWithFailPoints( + ctx context.Context, + manager *stream.TableMappingManager, +) error { + failpoint.Inject("failed-before-id-maps-saved", func(_ failpoint.Value) { + failpoint.Return(errors.New("failpoint: failed before id maps saved")) + }) + + if err := rc.saveIDMap(ctx, manager); err != nil { + return errors.Trace(err) + } + + failpoint.Inject("failed-after-id-maps-saved", func(_ failpoint.Value) { + failpoint.Return(errors.New("failpoint: failed after id maps saved")) + }) + return nil +} + // saveIDMap saves the id mapping information. func (rc *LogClient) saveIDMap( ctx context.Context, @@ -1940,7 +1841,7 @@ func (rc *LogClient) saveIDMap( if rc.useCheckpoint { log.Info("save checkpoint task info with InLogRestoreAndIdMapPersist status") if err := checkpoint.SaveCheckpointProgress(ctx, rc.unsafeSession, &checkpoint.CheckpointProgress{ - Progress: checkpoint.InLogRestoreAndIdMapPersist, + Progress: checkpoint.InLogRestoreAndIdMapPersisted, }); err != nil { return errors.Trace(err) } @@ -2067,3 +1968,13 @@ func PutRawKvWithRetry(ctx context.Context, client *rawkv.RawKVBatchClient, key, } return nil } + +// DropTable drops a table with the given database and table name +func (rc *LogClient) DropTable(ctx context.Context, dbName, tableName string) error { + dropSQL := "DROP TABLE IF EXISTS %n.%n" + if err := rc.unsafeSession.ExecuteInternal(ctx, dropSQL, dbName, tableName); err != nil { + return errors.Annotatef(err, "failed to drop table %s.%s", dbName, tableName) + } + log.Info("dropped table", zap.String("db", dbName), zap.String("table", tableName)) + return nil +} diff --git a/br/pkg/restore/log_client/client_test.go b/br/pkg/restore/log_client/client_test.go index ecf9452fe4f7c..dbe96b2dedf7d 100644 --- a/br/pkg/restore/log_client/client_test.go +++ b/br/pkg/restore/log_client/client_test.go @@ -36,6 +36,7 @@ import ( "github.com/pingcap/tidb/br/pkg/restore/split" "github.com/pingcap/tidb/br/pkg/restore/utils" "github.com/pingcap/tidb/br/pkg/stream" + "github.com/pingcap/tidb/br/pkg/utils/consts" "github.com/pingcap/tidb/br/pkg/utils/iter" "github.com/pingcap/tidb/br/pkg/utiltest" "github.com/pingcap/tidb/pkg/domain" @@ -48,7 +49,6 @@ import ( "github.com/pingcap/tidb/pkg/util/chunk" "github.com/pingcap/tidb/pkg/util/codec" "github.com/pingcap/tidb/pkg/util/sqlexec" - filter "github.com/pingcap/tidb/pkg/util/table-filter" "github.com/stretchr/testify/require" "github.com/tikv/client-go/v2/rawkv" "google.golang.org/grpc/keepalive" @@ -97,7 +97,7 @@ func TestDeleteRangeQueryExec(t *testing.T) { ctx := context.Background() m := mc g := gluetidb.New() - client := logclient.NewRestoreClient( + client := logclient.NewLogClient( split.NewFakePDClient(nil, false, nil), nil, nil, keepalive.ClientParameters{}) err := client.Init(ctx, g, m.Storage) require.NoError(t, err) @@ -116,7 +116,7 @@ func TestDeleteRangeQuery(t *testing.T) { m := mc g := gluetidb.New() - client := logclient.NewRestoreClient( + client := logclient.NewLogClient( split.NewFakePDClient(nil, false, nil), nil, nil, keepalive.ClientParameters{}) err := client.Init(ctx, g, m.Storage) require.NoError(t, err) @@ -138,19 +138,8 @@ func TestDeleteRangeQuery(t *testing.T) { } } -func MockEmptySchemasReplace() *stream.SchemasReplace { - dbMap := make(map[stream.UpstreamID]*stream.DBReplace) - return stream.NewSchemasReplace( - dbMap, - nil, - 1, - filter.All(), - nil, - ) -} - func TestRestoreBatchMetaKVFiles(t *testing.T) { - client := logclient.NewRestoreClient(nil, nil, nil, keepalive.ClientParameters{}) + client := logclient.NewLogClient(nil, nil, nil, keepalive.ClientParameters{}) files := []*backuppb.DataFileInfo{} // test empty files and entries next, err := client.RestoreBatchMetaKVFiles(context.Background(), files[0:], nil, make([]*logclient.KvEntryWithTS, 0), math.MaxUint64, nil, nil, "") @@ -159,41 +148,35 @@ func TestRestoreBatchMetaKVFiles(t *testing.T) { } func TestRestoreMetaKVFilesWithBatchMethod1(t *testing.T) { - files_default := []*backuppb.DataFileInfo{} - files_write := []*backuppb.DataFileInfo{} + var filesDefault []*backuppb.DataFileInfo + var filesWrite []*backuppb.DataFileInfo batchCount := 0 - sr := MockEmptySchemasReplace() - err := logclient.RestoreMetaKVFilesWithBatchMethod( - context.Background(), - files_default, - files_write, - sr, - nil, - nil, - func( - ctx context.Context, + mockProcessor := &mockBatchProcessor{ + processFunc: func(ctx context.Context, files []*backuppb.DataFileInfo, - schemasReplace *stream.SchemasReplace, entries []*logclient.KvEntryWithTS, filterTS uint64, - updateStats func(kvCount uint64, size uint64), - progressInc func(), - cf string, - ) ([]*logclient.KvEntryWithTS, error) { + cf string) ([]*logclient.KvEntryWithTS, error) { require.Equal(t, 0, len(entries)) require.Equal(t, 0, len(files)) batchCount++ return nil, nil }, + } + err := logclient.LoadAndProcessMetaKVFilesInBatch( + context.Background(), + filesDefault, + filesWrite, + mockProcessor, ) require.Nil(t, err) require.Equal(t, batchCount, 2) } func TestRestoreMetaKVFilesWithBatchMethod2_default_empty(t *testing.T) { - files_default := []*backuppb.DataFileInfo{} - files_write := []*backuppb.DataFileInfo{ + var filesDefault []*backuppb.DataFileInfo + filesWrite := []*backuppb.DataFileInfo{ { Path: "f1", MinTs: 100, @@ -202,89 +185,78 @@ func TestRestoreMetaKVFilesWithBatchMethod2_default_empty(t *testing.T) { } batchCount := 0 - sr := MockEmptySchemasReplace() - err := logclient.RestoreMetaKVFilesWithBatchMethod( - context.Background(), - files_default, - files_write, - sr, - nil, - nil, - func( - ctx context.Context, + mockProcessor := &mockBatchProcessor{ + processFunc: func(ctx context.Context, files []*backuppb.DataFileInfo, - schemasReplace *stream.SchemasReplace, entries []*logclient.KvEntryWithTS, filterTS uint64, - updateStats func(kvCount uint64, size uint64), - progressInc func(), - cf string, - ) ([]*logclient.KvEntryWithTS, error) { + cf string) ([]*logclient.KvEntryWithTS, error) { if len(entries) == 0 && len(files) == 0 { - require.Equal(t, stream.DefaultCF, cf) + require.Equal(t, consts.DefaultCF, cf) batchCount++ } else { require.Equal(t, 0, len(entries)) require.Equal(t, 1, len(files)) require.Equal(t, uint64(100), files[0].MinTs) - require.Equal(t, stream.WriteCF, cf) + require.Equal(t, consts.WriteCF, cf) } require.Equal(t, uint64(math.MaxUint64), filterTS) return nil, nil }, + } + err := logclient.LoadAndProcessMetaKVFilesInBatch( + context.Background(), + filesDefault, + filesWrite, + mockProcessor, ) require.Nil(t, err) require.Equal(t, batchCount, 1) } func TestRestoreMetaKVFilesWithBatchMethod2_write_empty_1(t *testing.T) { - files_default := []*backuppb.DataFileInfo{ + filesDefault := []*backuppb.DataFileInfo{ { Path: "f1", MinTs: 100, MaxTs: 120, }, } - files_write := []*backuppb.DataFileInfo{} + var filesWrite []*backuppb.DataFileInfo batchCount := 0 - sr := MockEmptySchemasReplace() - err := logclient.RestoreMetaKVFilesWithBatchMethod( - context.Background(), - files_default, - files_write, - sr, - nil, - nil, - func( - ctx context.Context, + mockProcessor := &mockBatchProcessor{ + processFunc: func(ctx context.Context, files []*backuppb.DataFileInfo, - schemasReplace *stream.SchemasReplace, entries []*logclient.KvEntryWithTS, filterTS uint64, - updateStats func(kvCount uint64, size uint64), - progressInc func(), - cf string, - ) ([]*logclient.KvEntryWithTS, error) { + cf string) ([]*logclient.KvEntryWithTS, error) { if len(entries) == 0 && len(files) == 0 { - require.Equal(t, stream.WriteCF, cf) + require.Equal(t, consts.WriteCF, cf) batchCount++ } else { require.Equal(t, 0, len(entries)) require.Equal(t, 1, len(files)) require.Equal(t, uint64(100), files[0].MinTs) - require.Equal(t, stream.DefaultCF, cf) + require.Equal(t, consts.DefaultCF, cf) } require.Equal(t, uint64(math.MaxUint64), filterTS) return nil, nil }, + } + + err := logclient.LoadAndProcessMetaKVFilesInBatch( + context.Background(), + filesDefault, + filesWrite, + mockProcessor, ) require.Nil(t, err) require.Equal(t, batchCount, 1) } func TestRestoreMetaKVFilesWithBatchMethod2_write_empty_2(t *testing.T) { - files_default := []*backuppb.DataFileInfo{ + filesDefault := []*backuppb.DataFileInfo{ { Path: "f1", MinTs: 100, @@ -298,31 +270,19 @@ func TestRestoreMetaKVFilesWithBatchMethod2_write_empty_2(t *testing.T) { Length: logclient.MetaKVBatchSize, }, } - files_write := []*backuppb.DataFileInfo{} + var filesWrite []*backuppb.DataFileInfo emptyCount := 0 batchCount := 0 - sr := MockEmptySchemasReplace() - err := logclient.RestoreMetaKVFilesWithBatchMethod( - context.Background(), - files_default, - files_write, - sr, - nil, - nil, - func( - ctx context.Context, + mockProcessor := &mockBatchProcessor{ + processFunc: func(ctx context.Context, files []*backuppb.DataFileInfo, - schemasReplace *stream.SchemasReplace, entries []*logclient.KvEntryWithTS, filterTS uint64, - updateStats func(kvCount uint64, size uint64), - progressInc func(), - cf string, - ) ([]*logclient.KvEntryWithTS, error) { + cf string) ([]*logclient.KvEntryWithTS, error) { if len(entries) == 0 && len(files) == 0 { // write - write - require.Equal(t, stream.WriteCF, cf) + require.Equal(t, consts.WriteCF, cf) emptyCount++ if emptyCount == 1 { require.Equal(t, uint64(110), filterTS) @@ -333,7 +293,7 @@ func TestRestoreMetaKVFilesWithBatchMethod2_write_empty_2(t *testing.T) { // default - default batchCount++ require.Equal(t, 1, len(files)) - require.Equal(t, stream.DefaultCF, cf) + require.Equal(t, consts.DefaultCF, cf) if batchCount == 1 { require.Equal(t, uint64(100), files[0].MinTs) require.Equal(t, uint64(110), filterTS) @@ -343,6 +303,13 @@ func TestRestoreMetaKVFilesWithBatchMethod2_write_empty_2(t *testing.T) { } return nil, nil }, + } + + err := logclient.LoadAndProcessMetaKVFilesInBatch( + context.Background(), + filesDefault, + filesWrite, + mockProcessor, ) require.Nil(t, err) require.Equal(t, batchCount, 2) @@ -350,7 +317,7 @@ func TestRestoreMetaKVFilesWithBatchMethod2_write_empty_2(t *testing.T) { } func TestRestoreMetaKVFilesWithBatchMethod_with_entries(t *testing.T) { - files_default := []*backuppb.DataFileInfo{ + filesDefault := []*backuppb.DataFileInfo{ { Path: "f1", MinTs: 100, @@ -364,31 +331,19 @@ func TestRestoreMetaKVFilesWithBatchMethod_with_entries(t *testing.T) { Length: logclient.MetaKVBatchSize, }, } - files_write := []*backuppb.DataFileInfo{} + var filesWrite []*backuppb.DataFileInfo emptyCount := 0 batchCount := 0 - sr := MockEmptySchemasReplace() - err := logclient.RestoreMetaKVFilesWithBatchMethod( - context.Background(), - files_default, - files_write, - sr, - nil, - nil, - func( - ctx context.Context, + mockProcessor := &mockBatchProcessor{ + processFunc: func(ctx context.Context, files []*backuppb.DataFileInfo, - schemasReplace *stream.SchemasReplace, entries []*logclient.KvEntryWithTS, filterTS uint64, - updateStats func(kvCount uint64, size uint64), - progressInc func(), - cf string, - ) ([]*logclient.KvEntryWithTS, error) { + cf string) ([]*logclient.KvEntryWithTS, error) { if len(entries) == 0 && len(files) == 0 { // write - write - require.Equal(t, stream.WriteCF, cf) + require.Equal(t, consts.WriteCF, cf) emptyCount++ if emptyCount == 1 { require.Equal(t, uint64(110), filterTS) @@ -399,7 +354,7 @@ func TestRestoreMetaKVFilesWithBatchMethod_with_entries(t *testing.T) { // default - default batchCount++ require.Equal(t, 1, len(files)) - require.Equal(t, stream.DefaultCF, cf) + require.Equal(t, consts.DefaultCF, cf) if batchCount == 1 { require.Equal(t, uint64(100), files[0].MinTs) require.Equal(t, uint64(110), filterTS) @@ -409,6 +364,13 @@ func TestRestoreMetaKVFilesWithBatchMethod_with_entries(t *testing.T) { } return nil, nil }, + } + + err := logclient.LoadAndProcessMetaKVFilesInBatch( + context.Background(), + filesDefault, + filesWrite, + mockProcessor, ) require.Nil(t, err) require.Equal(t, batchCount, 2) @@ -475,31 +437,27 @@ func TestRestoreMetaKVFilesWithBatchMethod3(t *testing.T) { result := make(map[int][]*backuppb.DataFileInfo) resultKV := make(map[int]int) - sr := MockEmptySchemasReplace() - err := logclient.RestoreMetaKVFilesWithBatchMethod( - context.Background(), - defaultFiles, - writeFiles, - sr, - nil, - nil, - func( - ctx context.Context, - fs []*backuppb.DataFileInfo, - schemasReplace *stream.SchemasReplace, + mockProcessor := &mockBatchProcessor{ + processFunc: func(ctx context.Context, + files []*backuppb.DataFileInfo, entries []*logclient.KvEntryWithTS, filterTS uint64, - updateStats func(kvCount uint64, size uint64), - progressInc func(), - cf string, - ) ([]*logclient.KvEntryWithTS, error) { - result[batchCount] = fs + cf string) ([]*logclient.KvEntryWithTS, error) { + result[batchCount] = files t.Log(filterTS) resultKV[batchCount] = len(entries) batchCount++ return make([]*logclient.KvEntryWithTS, batchCount), nil }, + } + + err := logclient.LoadAndProcessMetaKVFilesInBatch( + context.Background(), + defaultFiles, + writeFiles, + mockProcessor, ) + require.Nil(t, err) require.Equal(t, len(result), 4) require.Equal(t, result[0], defaultFiles[0:3]) @@ -561,29 +519,25 @@ func TestRestoreMetaKVFilesWithBatchMethod4(t *testing.T) { batchCount := 0 result := make(map[int][]*backuppb.DataFileInfo) - sr := MockEmptySchemasReplace() - err := logclient.RestoreMetaKVFilesWithBatchMethod( - context.Background(), - defaultFiles, - writeFiles, - sr, - nil, - nil, - func( - ctx context.Context, - fs []*backuppb.DataFileInfo, - schemasReplace *stream.SchemasReplace, + mockProcessor := &mockBatchProcessor{ + processFunc: func(ctx context.Context, + files []*backuppb.DataFileInfo, entries []*logclient.KvEntryWithTS, filterTS uint64, - updateStats func(kvCount uint64, size uint64), - progressInc func(), - cf string, - ) ([]*logclient.KvEntryWithTS, error) { - result[batchCount] = fs + cf string) ([]*logclient.KvEntryWithTS, error) { + result[batchCount] = files batchCount++ return nil, nil }, + } + + err := logclient.LoadAndProcessMetaKVFilesInBatch( + context.Background(), + defaultFiles, + writeFiles, + mockProcessor, ) + require.Nil(t, err) require.Equal(t, len(result), 4) require.Equal(t, result[0], defaultFiles[0:2]) @@ -641,28 +595,22 @@ func TestRestoreMetaKVFilesWithBatchMethod5(t *testing.T) { batchCount := 0 result := make(map[int][]*backuppb.DataFileInfo) - sr := MockEmptySchemasReplace() - err := logclient.RestoreMetaKVFilesWithBatchMethod( - context.Background(), - defaultFiles, - writeFiles, - sr, - nil, - nil, - func( - ctx context.Context, - fs []*backuppb.DataFileInfo, - schemasReplace *stream.SchemasReplace, + mockProcessor := &mockBatchProcessor{ + processFunc: func(ctx context.Context, + files []*backuppb.DataFileInfo, entries []*logclient.KvEntryWithTS, filterTS uint64, - updateStats func(kvCount uint64, size uint64), - progressInc func(), - cf string, - ) ([]*logclient.KvEntryWithTS, error) { - result[batchCount] = fs + cf string) ([]*logclient.KvEntryWithTS, error) { + result[batchCount] = files batchCount++ return nil, nil }, + } + err := logclient.LoadAndProcessMetaKVFilesInBatch( + context.Background(), + defaultFiles, + writeFiles, + mockProcessor, ) require.Nil(t, err) require.Equal(t, len(result), 4) @@ -738,30 +686,24 @@ func TestRestoreMetaKVFilesWithBatchMethod6(t *testing.T) { result := make(map[int][]*backuppb.DataFileInfo) resultKV := make(map[int]int) - sr := MockEmptySchemasReplace() - err := logclient.RestoreMetaKVFilesWithBatchMethod( - context.Background(), - defaultFiles, - writeFiles, - sr, - nil, - nil, - func( - ctx context.Context, - fs []*backuppb.DataFileInfo, - schemasReplace *stream.SchemasReplace, + mockProcessor := &mockBatchProcessor{ + processFunc: func(ctx context.Context, + files []*backuppb.DataFileInfo, entries []*logclient.KvEntryWithTS, filterTS uint64, - updateStats func(kvCount uint64, size uint64), - progressInc func(), - cf string, - ) ([]*logclient.KvEntryWithTS, error) { - result[batchCount] = fs + cf string) ([]*logclient.KvEntryWithTS, error) { + result[batchCount] = files t.Log(filterTS) resultKV[batchCount] = len(entries) batchCount++ return make([]*logclient.KvEntryWithTS, batchCount), nil }, + } + err := logclient.LoadAndProcessMetaKVFilesInBatch( + context.Background(), + defaultFiles, + writeFiles, + mockProcessor, ) require.Nil(t, err) require.Equal(t, len(result), 6) @@ -841,20 +783,20 @@ func TestApplyKVFilesWithSingelMethod(t *testing.T) { Path: "log3", NumberOfEntries: 5, Length: 100, - Cf: stream.WriteCF, + Cf: consts.WriteCF, Type: backuppb.FileType_Delete, }, { Path: "log1", NumberOfEntries: 5, Length: 100, - Cf: stream.DefaultCF, + Cf: consts.DefaultCF, Type: backuppb.FileType_Put, }, { Path: "log2", NumberOfEntries: 5, Length: 100, - Cf: stream.WriteCF, + Cf: consts.WriteCF, Type: backuppb.FileType_Put, }, } @@ -896,28 +838,28 @@ func TestApplyKVFilesWithBatchMethod1(t *testing.T) { Path: "log5", NumberOfEntries: 5, Length: 100, - Cf: stream.WriteCF, + Cf: consts.WriteCF, Type: backuppb.FileType_Delete, RegionId: 1, }, { Path: "log3", NumberOfEntries: 5, Length: 100, - Cf: stream.WriteCF, + Cf: consts.WriteCF, Type: backuppb.FileType_Put, RegionId: 1, }, { Path: "log4", NumberOfEntries: 5, Length: 100, - Cf: stream.WriteCF, + Cf: consts.WriteCF, Type: backuppb.FileType_Put, RegionId: 1, }, { Path: "log1", NumberOfEntries: 5, Length: 800, - Cf: stream.DefaultCF, + Cf: consts.DefaultCF, Type: backuppb.FileType_Put, RegionId: 1, }, @@ -925,7 +867,7 @@ func TestApplyKVFilesWithBatchMethod1(t *testing.T) { Path: "log2", NumberOfEntries: 5, Length: 200, - Cf: stream.DefaultCF, + Cf: consts.DefaultCF, Type: backuppb.FileType_Put, RegionId: 1, }, @@ -979,35 +921,35 @@ func TestApplyKVFilesWithBatchMethod2(t *testing.T) { Path: "log1", NumberOfEntries: 5, Length: 100, - Cf: stream.WriteCF, + Cf: consts.WriteCF, Type: backuppb.FileType_Delete, RegionId: 1, }, { Path: "log2", NumberOfEntries: 5, Length: 100, - Cf: stream.WriteCF, + Cf: consts.WriteCF, Type: backuppb.FileType_Put, RegionId: 1, }, { Path: "log3", NumberOfEntries: 5, Length: 100, - Cf: stream.WriteCF, + Cf: consts.WriteCF, Type: backuppb.FileType_Put, RegionId: 1, }, { Path: "log4", NumberOfEntries: 5, Length: 100, - Cf: stream.WriteCF, + Cf: consts.WriteCF, Type: backuppb.FileType_Put, RegionId: 1, }, { Path: "log5", NumberOfEntries: 5, Length: 800, - Cf: stream.DefaultCF, + Cf: consts.DefaultCF, Type: backuppb.FileType_Put, RegionId: 1, }, @@ -1015,7 +957,7 @@ func TestApplyKVFilesWithBatchMethod2(t *testing.T) { Path: "log6", NumberOfEntries: 5, Length: 200, - Cf: stream.DefaultCF, + Cf: consts.DefaultCF, Type: backuppb.FileType_Put, RegionId: 1, }, @@ -1070,28 +1012,28 @@ func TestApplyKVFilesWithBatchMethod3(t *testing.T) { Path: "log1", NumberOfEntries: 5, Length: 2000, - Cf: stream.WriteCF, + Cf: consts.WriteCF, Type: backuppb.FileType_Delete, RegionId: 1, }, { Path: "log2", NumberOfEntries: 5, Length: 2000, - Cf: stream.WriteCF, + Cf: consts.WriteCF, Type: backuppb.FileType_Put, RegionId: 1, }, { Path: "log3", NumberOfEntries: 5, Length: 100, - Cf: stream.WriteCF, + Cf: consts.WriteCF, Type: backuppb.FileType_Put, RegionId: 1, }, { Path: "log5", NumberOfEntries: 5, Length: 800, - Cf: stream.DefaultCF, + Cf: consts.DefaultCF, Type: backuppb.FileType_Put, RegionId: 3, }, @@ -1099,7 +1041,7 @@ func TestApplyKVFilesWithBatchMethod3(t *testing.T) { Path: "log6", NumberOfEntries: 5, Length: 200, - Cf: stream.DefaultCF, + Cf: consts.DefaultCF, Type: backuppb.FileType_Put, RegionId: 3, }, @@ -1153,35 +1095,35 @@ func TestApplyKVFilesWithBatchMethod4(t *testing.T) { Path: "log1", NumberOfEntries: 5, Length: 2000, - Cf: stream.WriteCF, + Cf: consts.WriteCF, Type: backuppb.FileType_Delete, TableId: 1, }, { Path: "log2", NumberOfEntries: 5, Length: 100, - Cf: stream.WriteCF, + Cf: consts.WriteCF, Type: backuppb.FileType_Put, TableId: 1, }, { Path: "log3", NumberOfEntries: 5, Length: 100, - Cf: stream.WriteCF, + Cf: consts.WriteCF, Type: backuppb.FileType_Put, TableId: 2, }, { Path: "log4", NumberOfEntries: 5, Length: 100, - Cf: stream.WriteCF, + Cf: consts.WriteCF, Type: backuppb.FileType_Put, TableId: 1, }, { Path: "log5", NumberOfEntries: 5, Length: 100, - Cf: stream.DefaultCF, + Cf: consts.DefaultCF, Type: backuppb.FileType_Put, TableId: 2, }, @@ -1231,35 +1173,35 @@ func TestApplyKVFilesWithBatchMethod5(t *testing.T) { Path: "log1", NumberOfEntries: 5, Length: 2000, - Cf: stream.WriteCF, + Cf: consts.WriteCF, Type: backuppb.FileType_Delete, TableId: 1, }, { Path: "log2", NumberOfEntries: 5, Length: 100, - Cf: stream.WriteCF, + Cf: consts.WriteCF, Type: backuppb.FileType_Put, TableId: 1, }, { Path: "log3", NumberOfEntries: 5, Length: 100, - Cf: stream.WriteCF, + Cf: consts.WriteCF, Type: backuppb.FileType_Put, TableId: 2, }, { Path: "log4", NumberOfEntries: 5, Length: 100, - Cf: stream.WriteCF, + Cf: consts.WriteCF, Type: backuppb.FileType_Put, TableId: 1, }, { Path: "log5", NumberOfEntries: 5, Length: 100, - Cf: stream.DefaultCF, + Cf: consts.DefaultCF, Type: backuppb.FileType_Put, TableId: 2, }, @@ -1386,16 +1328,16 @@ func TestInitSchemasReplaceForDDL(t *testing.T) { { client := logclient.TEST_NewLogClient(123, 1, 2, 1, domain.NewMockDomain(), fakeSession{}) - cfg := &logclient.BuildTableMappingManagerConfig{CurrentIdMapSaved: false} - _, err := client.BuildTableMappingManager(ctx, cfg) + cfg := &logclient.GetIDMapConfig{LoadSavedIDMap: false} + _, err := client.GetBaseIDMap(ctx, cfg) require.Error(t, err) require.Regexp(t, "failed to get pitr id map from mysql.tidb_pitr_id_map.* [2, 1]", err.Error()) } { client := logclient.TEST_NewLogClient(123, 1, 2, 1, domain.NewMockDomain(), fakeSession{}) - cfg := &logclient.BuildTableMappingManagerConfig{CurrentIdMapSaved: true} - _, err := client.BuildTableMappingManager(ctx, cfg) + cfg := &logclient.GetIDMapConfig{LoadSavedIDMap: true} + _, err := client.GetBaseIDMap(ctx, cfg) require.Error(t, err) require.Regexp(t, "failed to get pitr id map from mysql.tidb_pitr_id_map.* [1, 1]", err.Error()) } @@ -1408,8 +1350,8 @@ func TestInitSchemasReplaceForDDL(t *testing.T) { se, err := g.CreateSession(s.Mock.Storage) require.NoError(t, err) client := logclient.TEST_NewLogClient(123, 1, 2, 1, domain.NewMockDomain(), se) - cfg := &logclient.BuildTableMappingManagerConfig{CurrentIdMapSaved: true} - _, err = client.BuildTableMappingManager(ctx, cfg) + cfg := &logclient.GetIDMapConfig{LoadSavedIDMap: true} + _, err = client.GetBaseIDMap(ctx, cfg) require.Error(t, err) require.Contains(t, err.Error(), "miss upstream table information at `start-ts`(1) but the full backup path is not specified") } @@ -1480,7 +1422,7 @@ func TestPITRIDMap(t *testing.T) { require.NoError(t, err) client := logclient.TEST_NewLogClient(123, 1, 2, 3, nil, se) baseTableMappingManager := &stream.TableMappingManager{ - DbReplaceMap: getDBMap(), + DBReplaceMap: getDBMap(), } err = client.TEST_saveIDMap(ctx, baseTableMappingManager) require.NoError(t, err) @@ -1494,9 +1436,9 @@ func TestPITRIDMap(t *testing.T) { newSchemaReplaces, err = client.TEST_initSchemasMap(ctx, 2) require.NoError(t, err) - require.Equal(t, len(baseTableMappingManager.DbReplaceMap), len(newSchemaReplaces)) + require.Equal(t, len(baseTableMappingManager.DBReplaceMap), len(newSchemaReplaces)) for _, dbMap := range newSchemaReplaces { - baseDbMap := baseTableMappingManager.DbReplaceMap[dbMap.IdMap.UpstreamId] + baseDbMap := baseTableMappingManager.DBReplaceMap[dbMap.IdMap.UpstreamId] require.NotNil(t, baseDbMap) require.Equal(t, baseDbMap.DbID, dbMap.IdMap.DownstreamId) require.Equal(t, baseDbMap.Name, dbMap.Name) @@ -2054,3 +1996,23 @@ func TestPutRawKvWithRetry(t *testing.T) { }) } } + +type mockBatchProcessor struct { + processFunc func( + ctx context.Context, + files []*backuppb.DataFileInfo, + entries []*logclient.KvEntryWithTS, + filterTS uint64, + cf string, + ) ([]*logclient.KvEntryWithTS, error) +} + +func (m *mockBatchProcessor) ProcessBatch( + ctx context.Context, + files []*backuppb.DataFileInfo, + entries []*logclient.KvEntryWithTS, + filterTS uint64, + cf string, +) ([]*logclient.KvEntryWithTS, error) { + return m.processFunc(ctx, files, entries, filterTS, cf) +} diff --git a/br/pkg/restore/log_client/export_test.go b/br/pkg/restore/log_client/export_test.go index e0fd5b06e2e56..19f239f910248 100644 --- a/br/pkg/restore/log_client/export_test.go +++ b/br/pkg/restore/log_client/export_test.go @@ -65,19 +65,19 @@ func (rc *LogClient) TEST_saveIDMap( ctx context.Context, m *stream.TableMappingManager, ) error { - return rc.saveIDMap(ctx, m) + return rc.SaveIdMapWithFailPoints(ctx, m) } func (rc *LogClient) TEST_initSchemasMap( ctx context.Context, restoreTS uint64, ) ([]*backuppb.PitrDBMap, error) { - return rc.initSchemasMap(ctx, restoreTS) + return rc.loadSchemasMap(ctx, restoreTS) } // readStreamMetaByTS is used for streaming task. collect all meta file by TS, it is for test usage. -func (rc *LogFileManager) ReadStreamMeta(ctx context.Context) ([]*MetaName, error) { - metas, err := rc.streamingMeta(ctx) +func (lm *LogFileManager) ReadStreamMeta(ctx context.Context) ([]*MetaName, error) { + metas, err := lm.streamingMeta(ctx) if err != nil { return nil, err } diff --git a/br/pkg/restore/log_client/import.go b/br/pkg/restore/log_client/import.go index a4dbf4ca73839..0b5581c706fa2 100644 --- a/br/pkg/restore/log_client/import.go +++ b/br/pkg/restore/log_client/import.go @@ -35,9 +35,9 @@ import ( importclient "github.com/pingcap/tidb/br/pkg/restore/internal/import_client" "github.com/pingcap/tidb/br/pkg/restore/split" restoreutils "github.com/pingcap/tidb/br/pkg/restore/utils" - "github.com/pingcap/tidb/br/pkg/stream" "github.com/pingcap/tidb/br/pkg/summary" "github.com/pingcap/tidb/br/pkg/utils" + "github.com/pingcap/tidb/br/pkg/utils/consts" "github.com/pingcap/tidb/pkg/kv" pd "github.com/tikv/pd/client" "go.uber.org/multierr" @@ -253,7 +253,7 @@ func (importer *LogFileImporter) downloadAndApplyKVFile( RangeLength: file.RangeLength, IsDelete: file.Type == backuppb.FileType_Delete, StartTs: func() uint64 { - if file.Cf == stream.DefaultCF { + if file.Cf == consts.DefaultCF { return shiftStartTS } return startTS diff --git a/br/pkg/restore/log_client/log_file_manager.go b/br/pkg/restore/log_client/log_file_manager.go index 3faed8335afa9..0aeaee85fedf0 100644 --- a/br/pkg/restore/log_client/log_file_manager.go +++ b/br/pkg/restore/log_client/log_file_manager.go @@ -20,6 +20,8 @@ import ( berrors "github.com/pingcap/tidb/br/pkg/errors" "github.com/pingcap/tidb/br/pkg/storage" "github.com/pingcap/tidb/br/pkg/stream" + "github.com/pingcap/tidb/br/pkg/utils" + "github.com/pingcap/tidb/br/pkg/utils/consts" "github.com/pingcap/tidb/br/pkg/utils/iter" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/util/codec" @@ -83,6 +85,7 @@ type streamMetadataHelper interface { encryptionInfo *encryptionpb.FileEncryptionInfo, ) ([]byte, error) ParseToMetadata(rawMetaData []byte) (*backuppb.Metadata, error) + Close() } type logFilesStatistic struct { @@ -108,8 +111,8 @@ type LogFileManager struct { storage storage.ExternalStorage helper streamMetadataHelper - withMigraionBuilder *WithMigrationsBuilder - withMigrations *WithMigrations + withMigrationBuilder *WithMigrationsBuilder + withMigrations *WithMigrations metadataDownloadBatchSize uint @@ -139,12 +142,12 @@ type DDLMetaGroup struct { // Generally the config cannot be changed during its lifetime. func CreateLogFileManager(ctx context.Context, init LogFileManagerInit) (*LogFileManager, error) { fm := &LogFileManager{ - startTS: init.StartTS, - restoreTS: init.RestoreTS, - storage: init.Storage, - helper: stream.NewMetadataHelper(stream.WithEncryptionManager(init.EncryptionManager)), - withMigraionBuilder: init.MigrationsBuilder, - withMigrations: init.Migrations, + startTS: init.StartTS, + restoreTS: init.RestoreTS, + storage: init.Storage, + helper: stream.NewMetadataHelper(stream.WithEncryptionManager(init.EncryptionManager)), + withMigrationBuilder: init.MigrationsBuilder, + withMigrations: init.Migrations, metadataDownloadBatchSize: init.MetadataDownloadBatchSize, } @@ -155,30 +158,30 @@ func CreateLogFileManager(ctx context.Context, init LogFileManagerInit) (*LogFil return fm, nil } -func (rc *LogFileManager) BuildMigrations(migs []*backuppb.Migration) { - w := rc.withMigraionBuilder.Build(migs) - rc.withMigrations = &w +func (lm *LogFileManager) BuildMigrations(migs []*backuppb.Migration) { + w := lm.withMigrationBuilder.Build(migs) + lm.withMigrations = &w } -func (rc *LogFileManager) ShiftTS() uint64 { - return rc.shiftStartTS +func (lm *LogFileManager) ShiftTS() uint64 { + return lm.shiftStartTS } -func (rc *LogFileManager) loadShiftTS(ctx context.Context) error { +func (lm *LogFileManager) loadShiftTS(ctx context.Context) error { shiftTS := struct { sync.Mutex value uint64 exists bool }{} - err := stream.FastUnmarshalMetaData(ctx, rc.storage, rc.metadataDownloadBatchSize, func(path string, raw []byte) error { - m, err := rc.helper.ParseToMetadata(raw) + err := stream.FastUnmarshalMetaData(ctx, lm.storage, lm.metadataDownloadBatchSize, func(path string, raw []byte) error { + m, err := lm.helper.ParseToMetadata(raw) if err != nil { return err } log.Info("read meta from storage and parse", zap.String("path", path), zap.Uint64("min-ts", m.MinTs), zap.Uint64("max-ts", m.MaxTs), zap.Int32("meta-version", int32(m.MetaVersion))) - ts, ok := stream.UpdateShiftTS(m, rc.startTS, rc.restoreTS) + ts, ok := stream.UpdateShiftTS(m, lm.startTS, lm.restoreTS) shiftTS.Lock() if ok && (!shiftTS.exists || shiftTS.value > ts) { shiftTS.value = ts @@ -192,31 +195,31 @@ func (rc *LogFileManager) loadShiftTS(ctx context.Context) error { return err } if !shiftTS.exists { - rc.shiftStartTS = rc.startTS - rc.withMigraionBuilder.SetShiftStartTS(rc.shiftStartTS) + lm.shiftStartTS = lm.startTS + lm.withMigrationBuilder.SetShiftStartTS(lm.shiftStartTS) return nil } - rc.shiftStartTS = shiftTS.value - rc.withMigraionBuilder.SetShiftStartTS(rc.shiftStartTS) + lm.shiftStartTS = shiftTS.value + lm.withMigrationBuilder.SetShiftStartTS(lm.shiftStartTS) return nil } -func (rc *LogFileManager) streamingMeta(ctx context.Context) (MetaNameIter, error) { - return rc.streamingMetaByTS(ctx, rc.restoreTS) +func (lm *LogFileManager) streamingMeta(ctx context.Context) (MetaNameIter, error) { + return lm.streamingMetaByTS(ctx, lm.restoreTS) } -func (rc *LogFileManager) streamingMetaByTS(ctx context.Context, restoreTS uint64) (MetaNameIter, error) { - it, err := rc.createMetaIterOver(ctx, rc.storage) +func (lm *LogFileManager) streamingMetaByTS(ctx context.Context, restoreTS uint64) (MetaNameIter, error) { + it, err := lm.createMetaIterOver(ctx, lm.storage) if err != nil { return nil, err } filtered := iter.FilterOut(it, func(metaname *MetaName) bool { - return restoreTS < metaname.meta.MinTs || metaname.meta.MaxTs < rc.shiftStartTS + return restoreTS < metaname.meta.MinTs || metaname.meta.MaxTs < lm.shiftStartTS }) return filtered, nil } -func (rc *LogFileManager) createMetaIterOver(ctx context.Context, s storage.ExternalStorage) (MetaNameIter, error) { +func (lm *LogFileManager) createMetaIterOver(ctx context.Context, s storage.ExternalStorage) (MetaNameIter, error) { opt := &storage.WalkOption{SubDir: stream.GetStreamBackupMetaPrefix()} names := []string{} err := s.WalkDir(ctx, opt, func(path string, size int64) error { @@ -235,7 +238,7 @@ func (rc *LogFileManager) createMetaIterOver(ctx context.Context, s storage.Exte if err != nil { return nil, errors.Annotatef(err, "failed during reading file %s", name) } - meta, err := rc.helper.ParseToMetadata(f) + meta, err := lm.helper.ParseToMetadata(f) if err != nil { return nil, errors.Annotatef(err, "failed to parse metadata of file %s", name) } @@ -244,12 +247,12 @@ func (rc *LogFileManager) createMetaIterOver(ctx context.Context, s storage.Exte // TODO: maybe we need to be able to adjust the concurrency to download files, // which currently is the same as the chunk size reader := iter.Transform(namesIter, readMeta, - iter.WithChunkSize(rc.metadataDownloadBatchSize), iter.WithConcurrency(rc.metadataDownloadBatchSize)) + iter.WithChunkSize(lm.metadataDownloadBatchSize), iter.WithConcurrency(lm.metadataDownloadBatchSize)) return reader, nil } -func (rc *LogFileManager) FilterDataFiles(m MetaNameIter) LogIter { - ms := rc.withMigrations.Metas(m) +func (lm *LogFileManager) FilterDataFiles(m MetaNameIter) LogIter { + ms := lm.withMigrations.Metas(m) return iter.FlatMap(ms, func(m *MetaWithMigrations) LogIter { gs := m.Physicals(iter.Enumerate(iter.FromSlice(m.meta.FileGroups))) return iter.FlatMap(gs, func(gim *PhysicalWithMigrations) LogIter { @@ -260,7 +263,7 @@ func (rc *LogFileManager) FilterDataFiles(m MetaNameIter) LogIter { if m.meta.MetaVersion > backuppb.MetaVersion_V1 { di.Item.Path = gim.physical.Item.Path } - return di.Item.IsMeta || rc.ShouldFilterOut(di.Item) + return di.Item.IsMeta || lm.ShouldFilterOutByTs(di.Item) }) return iter.Map(fs, func(di FileIndex) *LogDataFileInfo { return &LogDataFileInfo{ @@ -279,14 +282,14 @@ func (rc *LogFileManager) FilterDataFiles(m MetaNameIter) LogIter { }) } -// ShouldFilterOut checks whether a file should be filtered out via the current client. -func (rc *LogFileManager) ShouldFilterOut(d *backuppb.DataFileInfo) bool { - return d.MinTs > rc.restoreTS || - (d.Cf == stream.WriteCF && d.MaxTs < rc.startTS) || - (d.Cf == stream.DefaultCF && d.MaxTs < rc.shiftStartTS) +// ShouldFilterOutByTs checks whether a file should be filtered out via the current client. +func (lm *LogFileManager) ShouldFilterOutByTs(d *backuppb.DataFileInfo) bool { + return d.MinTs > lm.restoreTS || + (d.Cf == consts.WriteCF && d.MaxTs < lm.startTS) || + (d.Cf == consts.DefaultCF && d.MaxTs < lm.shiftStartTS) } -func (rc *LogFileManager) collectDDLFilesAndPrepareCache( +func (lm *LogFileManager) collectDDLFilesAndPrepareCache( ctx context.Context, files MetaGroupIter, ) ([]Log, error) { @@ -300,23 +303,23 @@ func (rc *LogFileManager) collectDDLFilesAndPrepareCache( dataFileInfos := make([]*backuppb.DataFileInfo, 0) for _, g := range fs.Item { - rc.helper.InitCacheEntry(g.Path, len(g.FileMetas)) + lm.helper.InitCacheEntry(g.Path, len(g.FileMetas)) dataFileInfos = append(dataFileInfos, g.FileMetas...) } return dataFileInfos, nil } -// LoadDDLFilesAndCountDMLFiles loads all DDL files needs to be restored in the restoration. +// LoadDDLFiles loads all DDL files needs to be restored in the restoration. // This function returns all DDL files needing directly because we need sort all of them. -func (rc *LogFileManager) LoadDDLFilesAndCountDMLFiles(ctx context.Context) ([]Log, error) { - m, err := rc.streamingMeta(ctx) +func (lm *LogFileManager) LoadDDLFiles(ctx context.Context) ([]Log, error) { + m, err := lm.streamingMeta(ctx) if err != nil { return nil, err } - mg := rc.FilterMetaFiles(m) + mg := lm.FilterMetaFiles(m) - return rc.collectDDLFilesAndPrepareCache(ctx, mg) + return lm.collectDDLFilesAndPrepareCache(ctx, mg) } type loadDMLFilesConfig struct { @@ -333,17 +336,17 @@ func lDOptWithStatistics(s *logFilesStatistic) loadDMLFilesOption { // LoadDMLFiles loads all DML files needs to be restored in the restoration. // This function returns a stream, because there are usually many DML files need to be restored. -func (rc *LogFileManager) LoadDMLFiles(ctx context.Context) (LogIter, error) { - m, err := rc.streamingMeta(ctx) +func (lm *LogFileManager) LoadDMLFiles(ctx context.Context) (LogIter, error) { + m, err := lm.streamingMeta(ctx) if err != nil { return nil, err } - l := rc.FilterDataFiles(m) + l := lm.FilterDataFiles(m) return l, nil } -func (rc *LogFileManager) FilterMetaFiles(ms MetaNameIter) MetaGroupIter { +func (lm *LogFileManager) FilterMetaFiles(ms MetaNameIter) MetaGroupIter { return iter.FlatMap(ms, func(m *MetaName) MetaGroupIter { return iter.Map(iter.FromSlice(m.meta.FileGroups), func(g *backuppb.DataFileGroup) DDLMetaGroup { metas := iter.FilterOut(iter.FromSlice(g.DataFilesInfo), func(d Log) bool { @@ -351,14 +354,14 @@ func (rc *LogFileManager) FilterMetaFiles(ms MetaNameIter) MetaGroupIter { if m.meta.MetaVersion > backuppb.MetaVersion_V1 { d.Path = g.Path } - if rc.ShouldFilterOut(d) { + if lm.ShouldFilterOutByTs(d) { return true } // count the progress - if rc.Stats != nil { - atomic.AddInt64(&rc.Stats.NumEntries, d.NumberOfEntries) - atomic.AddUint64(&rc.Stats.NumFiles, 1) - atomic.AddUint64(&rc.Stats.Size, d.Length) + if lm.Stats != nil { + atomic.AddInt64(&lm.Stats.NumEntries, d.NumberOfEntries) + atomic.AddUint64(&lm.Stats.NumFiles, 1) + atomic.AddUint64(&lm.Stats.Size, d.Length) } return !d.IsMeta }) @@ -371,15 +374,15 @@ func (rc *LogFileManager) FilterMetaFiles(ms MetaNameIter) MetaGroupIter { }) } -// Fetch compactions that may contain file less than the TS. -func (rc *LogFileManager) GetCompactionIter(ctx context.Context) iter.TryNextor[SSTs] { - return iter.Map(rc.withMigrations.Compactions(ctx, rc.storage), func(c *backuppb.LogFileSubcompaction) SSTs { +// GetCompactionIter fetches compactions that may contain file less than the TS. +func (lm *LogFileManager) GetCompactionIter(ctx context.Context) iter.TryNextor[SSTs] { + return iter.Map(lm.withMigrations.Compactions(ctx, lm.storage), func(c *backuppb.LogFileSubcompaction) SSTs { return &CompactedSSTs{c} }) } -func (rc *LogFileManager) GetIngestedSSTs(ctx context.Context) iter.TryNextor[SSTs] { - return iter.FlatMap(rc.withMigrations.IngestedSSTs(ctx, rc.storage), func(c *backuppb.IngestedSSTs) iter.TryNextor[SSTs] { +func (lm *LogFileManager) GetIngestedSSTs(ctx context.Context) iter.TryNextor[SSTs] { + return iter.FlatMap(lm.withMigrations.IngestedSSTs(ctx, lm.storage), func(c *backuppb.IngestedSSTs) iter.TryNextor[SSTs] { remap := map[int64]int64{} for _, r := range c.RewrittenTables { remap[r.AncestorUpstream] = r.Upstream @@ -397,9 +400,9 @@ func (rc *LogFileManager) GetIngestedSSTs(ctx context.Context) iter.TryNextor[SS }) } -func (rc *LogFileManager) CountExtraSSTTotalKVs(ctx context.Context) (int64, error) { +func (lm *LogFileManager) CountExtraSSTTotalKVs(ctx context.Context) (int64, error) { count := int64(0) - ssts := iter.ConcatAll(rc.GetCompactionIter(ctx), rc.GetIngestedSSTs(ctx)) + ssts := iter.ConcatAll(lm.GetCompactionIter(ctx), lm.GetIngestedSSTs(ctx)) for err, ssts := range iter.AsSeq(ctx, ssts) { if err != nil { return 0, errors.Trace(err) @@ -411,7 +414,7 @@ func (rc *LogFileManager) CountExtraSSTTotalKVs(ctx context.Context) (int64, err return count, nil } -// the kv entry with ts, the ts is decoded from entry. +// KvEntryWithTS is kv entry with ts, the ts is decoded from entry. type KvEntryWithTS struct { E kv.Entry Ts uint64 @@ -427,17 +430,17 @@ func getKeyTS(key []byte) (uint64, error) { return ts, err } -// ReadAllEntries loads content of a log file, with filtering out no needed entries. -func (rc *LogFileManager) ReadAllEntries( +// ReadFilteredEntriesFromFiles loads content of a log file from external storage, and filter out entries based on TS. +func (lm *LogFileManager) ReadFilteredEntriesFromFiles( ctx context.Context, file Log, filterTS uint64, ) ([]*KvEntryWithTS, []*KvEntryWithTS, error) { kvEntries := make([]*KvEntryWithTS, 0) - nextKvEntries := make([]*KvEntryWithTS, 0) + filteredOutKvEntries := make([]*KvEntryWithTS, 0) - buff, err := rc.helper.ReadFile(ctx, file.Path, file.RangeOffset, file.RangeLength, file.CompressionType, - rc.storage, file.FileEncryptionInfo) + buff, err := lm.helper.ReadFile(ctx, file.Path, file.RangeOffset, file.RangeLength, file.CompressionType, + lm.storage, file.FileEncryptionInfo) if err != nil { return nil, nil, errors.Trace(err) } @@ -456,7 +459,7 @@ func (rc *LogFileManager) ReadAllEntries( txnEntry := kv.Entry{Key: iter.Key(), Value: iter.Value()} - if !stream.MaybeDBOrDDLJobHistoryKey(txnEntry.Key) { + if !utils.IsDBOrDDLJobHistoryKey(txnEntry.Key) { // only restore mDB and mDDLHistory continue } @@ -468,11 +471,11 @@ func (rc *LogFileManager) ReadAllEntries( // The commitTs in write CF need be limited on [startTs, restoreTs]. // We can restore more key-value in default CF. - if ts > rc.restoreTS { + if ts > lm.restoreTS { continue - } else if file.Cf == stream.WriteCF && ts < rc.startTS { + } else if file.Cf == consts.WriteCF && ts < lm.startTS { continue - } else if file.Cf == stream.DefaultCF && ts < rc.shiftStartTS { + } else if file.Cf == consts.DefaultCF && ts < lm.shiftStartTS { continue } @@ -488,11 +491,17 @@ func (rc *LogFileManager) ReadAllEntries( if ts < filterTS { kvEntries = append(kvEntries, &KvEntryWithTS{E: txnEntry, Ts: ts}) } else { - nextKvEntries = append(nextKvEntries, &KvEntryWithTS{E: txnEntry, Ts: ts}) + filteredOutKvEntries = append(filteredOutKvEntries, &KvEntryWithTS{E: txnEntry, Ts: ts}) } } - return kvEntries, nextKvEntries, nil + return kvEntries, filteredOutKvEntries, nil +} + +func (lm *LogFileManager) Close() { + if lm.helper != nil { + lm.helper.Close() + } } func Subcompactions(ctx context.Context, prefix string, s storage.ExternalStorage, shiftStartTS, restoredTS uint64) SubCompactionIter { diff --git a/br/pkg/restore/log_client/log_file_manager_test.go b/br/pkg/restore/log_client/log_file_manager_test.go index 79813d6ef78f2..490c26f3ad6dc 100644 --- a/br/pkg/restore/log_client/log_file_manager_test.go +++ b/br/pkg/restore/log_client/log_file_manager_test.go @@ -23,6 +23,7 @@ import ( logclient "github.com/pingcap/tidb/br/pkg/restore/log_client" "github.com/pingcap/tidb/br/pkg/storage" "github.com/pingcap/tidb/br/pkg/stream" + "github.com/pingcap/tidb/br/pkg/utils/consts" "github.com/pingcap/tidb/br/pkg/utils/iter" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/util/codec" @@ -55,7 +56,7 @@ func wr(start, end uint64, minBegin uint64) *backuppb.DataFileInfo { MinTs: start, MaxTs: end, MinBeginTsInDefaultCf: minBegin, - Cf: stream.WriteCF, + Cf: consts.WriteCF, } } @@ -66,7 +67,7 @@ func dr(start, end uint64) *backuppb.DataFileInfo { Path: fmt.Sprintf("write-%06d", id), MinTs: start, MaxTs: end, - Cf: stream.DefaultCF, + Cf: consts.DefaultCF, } } @@ -491,7 +492,7 @@ func testFileManagerWithMeta(t *testing.T, m metaMaker) { ), ).Item } else { - data, err := fm.LoadDDLFilesAndCountDMLFiles(ctx) + data, err := fm.LoadDDLFiles(ctx) req.NoError(err) r = data } @@ -618,8 +619,8 @@ func TestReadAllEntries(t *testing.T) { data, file := generateKvData() fm := logclient.TEST_NewLogFileManager(35, 75, 25, &logclient.FakeStreamMetadataHelper{Data: data}) { - file.Cf = stream.WriteCF - kvEntries, nextKvEntries, err := fm.ReadAllEntries(ctx, file, 50) + file.Cf = consts.WriteCF + kvEntries, nextKvEntries, err := fm.ReadFilteredEntriesFromFiles(ctx, file, 50) require.NoError(t, err) require.Equal(t, []*logclient.KvEntryWithTS{ encodekvEntryWithTS("mDDL", 37), @@ -631,8 +632,8 @@ func TestReadAllEntries(t *testing.T) { }, nextKvEntries) } { - file.Cf = stream.DefaultCF - kvEntries, nextKvEntries, err := fm.ReadAllEntries(ctx, file, 50) + file.Cf = consts.DefaultCF + kvEntries, nextKvEntries, err := fm.ReadFilteredEntriesFromFiles(ctx, file, 50) require.NoError(t, err) require.Equal(t, []*logclient.KvEntryWithTS{ encodekvEntryWithTS("mDDL", 27), diff --git a/br/pkg/restore/snap_client/client.go b/br/pkg/restore/snap_client/client.go index 9d03c080356f5..d8b6db82b4a6d 100644 --- a/br/pkg/restore/snap_client/client.go +++ b/br/pkg/restore/snap_client/client.go @@ -462,8 +462,8 @@ func (rc *SnapClient) InstallPiTRSupport(ctx context.Context, deps PiTRCollDep) return nil } -// Init create db connection and domain for storage. -func (rc *SnapClient) Init(g glue.Glue, store kv.Storage) error { +// InitConnections create db connection and domain for storage. +func (rc *SnapClient) InitConnections(g glue.Glue, store kv.Storage) error { // setDB must happen after set PolicyMode. // we will use policyMode to set session variables. var err error @@ -604,7 +604,7 @@ func (rc *SnapClient) initClients(ctx context.Context, backend *backuppb.Storage return nil } -func (rc *SnapClient) needLoadSchemas(backupMeta *backuppb.BackupMeta) bool { +func needLoadSchemas(backupMeta *backuppb.BackupMeta) bool { return !(backupMeta.IsRawKv || backupMeta.IsTxnKv) } @@ -618,7 +618,7 @@ func (rc *SnapClient) LoadSchemaIfNeededAndInitClient( RawStartKey []byte, RawEndKey []byte, ) error { - if rc.needLoadSchemas(backupMeta) { + if needLoadSchemas(backupMeta) { databases, err := metautil.LoadBackupTables(c, reader, loadStats) if err != nil { return errors.Trace(err) @@ -724,6 +724,15 @@ func (rc *SnapClient) GetDatabases() []*metautil.Database { return dbs } +// GetDatabaseMap returns all databases in a map indexed by db id +func (rc *SnapClient) GetDatabaseMap() map[int64]*metautil.Database { + dbMap := make(map[int64]*metautil.Database) + for _, db := range rc.databases { + dbMap[db.Info.ID] = db + } + return dbMap +} + // HasBackedUpSysDB whether we have backed up system tables // br backs system tables up since 5.1.0 func (rc *SnapClient) HasBackedUpSysDB() bool { diff --git a/br/pkg/restore/snap_client/client_test.go b/br/pkg/restore/snap_client/client_test.go index 2a53bb0b38bd9..b0122e787df7a 100644 --- a/br/pkg/restore/snap_client/client_test.go +++ b/br/pkg/restore/snap_client/client_test.go @@ -49,7 +49,7 @@ func TestCreateTables(t *testing.T) { m := mc g := gluetidb.New() client := snapclient.NewRestoreClient(m.PDClient, m.PDHTTPCli, nil, split.DefaultTestKeepaliveCfg) - err := client.Init(g, m.Storage) + err := client.InitConnections(g, m.Storage) require.NoError(t, err) info, err := m.Domain.GetSnapshotInfoSchema(math.MaxUint64) @@ -120,7 +120,7 @@ func TestNeedCheckTargetClusterFresh(t *testing.T) { g := gluetidb.New() client := snapclient.NewRestoreClient(cluster.PDClient, cluster.PDHTTPCli, nil, split.DefaultTestKeepaliveCfg) - err := client.Init(g, cluster.Storage) + err := client.InitConnections(g, cluster.Storage) require.NoError(t, err) // not set filter and first run with checkpoint @@ -150,7 +150,7 @@ func TestCheckTargetClusterFresh(t *testing.T) { g := gluetidb.New() client := snapclient.NewRestoreClient(cluster.PDClient, cluster.PDHTTPCli, nil, split.DefaultTestKeepaliveCfg) - err := client.Init(g, cluster.Storage) + err := client.InitConnections(g, cluster.Storage) require.NoError(t, err) ctx := context.Background() @@ -167,7 +167,7 @@ func TestCheckTargetClusterFreshWithTable(t *testing.T) { g := gluetidb.New() client := snapclient.NewRestoreClient(cluster.PDClient, cluster.PDHTTPCli, nil, split.DefaultTestKeepaliveCfg) - err := client.Init(g, cluster.Storage) + err := client.InitConnections(g, cluster.Storage) require.NoError(t, err) ctx := context.Background() @@ -202,7 +202,7 @@ func TestInitFullClusterRestore(t *testing.T) { cluster := mc g := gluetidb.New() client := snapclient.NewRestoreClient(cluster.PDClient, cluster.PDHTTPCli, nil, split.DefaultTestKeepaliveCfg) - err := client.Init(g, cluster.Storage) + err := client.InitConnections(g, cluster.Storage) require.NoError(t, err) // explicit filter diff --git a/br/pkg/restore/snap_client/systable_restore.go b/br/pkg/restore/snap_client/systable_restore.go index 524cfa17fdf72..e7bd827710f9e 100644 --- a/br/pkg/restore/snap_client/systable_restore.go +++ b/br/pkg/restore/snap_client/systable_restore.go @@ -181,7 +181,7 @@ func (rc *SnapClient) restoreSystemSchema(ctx context.Context, f filter.Filter, }() if !f.MatchSchema(sysDB) || !rc.withSysTable { - log.Debug("system database filtered out", zap.String("database", sysDB)) + log.Info("system database filtered out", zap.String("database", sysDB)) return nil } originDatabase, ok := rc.databases[temporaryDB.O] diff --git a/br/pkg/restore/snap_client/systable_restore_test.go b/br/pkg/restore/snap_client/systable_restore_test.go index 504b3a45cd6fa..6148522e1de5e 100644 --- a/br/pkg/restore/snap_client/systable_restore_test.go +++ b/br/pkg/restore/snap_client/systable_restore_test.go @@ -36,7 +36,7 @@ func TestCheckSysTableCompatibility(t *testing.T) { cluster := mc g := gluetidb.New() client := snapclient.NewRestoreClient(cluster.PDClient, cluster.PDHTTPCli, nil, split.DefaultTestKeepaliveCfg) - err := client.Init(g, cluster.Storage) + err := client.InitConnections(g, cluster.Storage) require.NoError(t, err) info, err := cluster.Domain.GetSnapshotInfoSchema(math.MaxUint64) diff --git a/br/pkg/restore/tiflashrec/tiflash_recorder.go b/br/pkg/restore/tiflashrec/tiflash_recorder.go index c87f0372f86a6..8cdcfa10182f5 100644 --- a/br/pkg/restore/tiflashrec/tiflash_recorder.go +++ b/br/pkg/restore/tiflashrec/tiflash_recorder.go @@ -130,15 +130,15 @@ func (r *TiFlashRecorder) GenerateResetAlterTableDDLs(info infoschema.InfoSchema func (r *TiFlashRecorder) GenerateAlterTableDDLs(info infoschema.InfoSchema) []string { items := make([]string, 0, len(r.items)) - r.Iterate(func(id int64, replica model.TiFlashReplicaInfo) { - table, ok := info.TableByID(context.Background(), id) + r.Iterate(func(tableId int64, replica model.TiFlashReplicaInfo) { + table, ok := info.TableByID(context.Background(), tableId) if !ok { - log.Warn("Table do not exist, skipping", zap.Int64("id", id)) + log.Warn("Table does not exist, might get filtered out if a custom filter is specified, skipping", zap.Int64("tableId", tableId)) return } schema, ok := infoschema.SchemaByTable(info, table.Meta()) if !ok { - log.Warn("Schema do not exist, skipping", zap.Int64("id", id), zap.Stringer("table", table.Meta().Name)) + log.Warn("Schema do not exist, skipping", zap.Int64("tableId", tableId), zap.Stringer("table", table.Meta().Name)) return } altTableSpec, err := alterTableSpecOf(replica, false) diff --git a/br/pkg/stream/BUILD.bazel b/br/pkg/stream/BUILD.bazel index e3629ca718deb..a18cc95affc00 100644 --- a/br/pkg/stream/BUILD.bazel +++ b/br/pkg/stream/BUILD.bazel @@ -4,14 +4,15 @@ go_library( name = "stream", srcs = [ "decode_kv.go", + "logging_helper.go", "meta_kv.go", "rewrite_meta_rawkv.go", "search.go", "stream_metas.go", "stream_mgr.go", "stream_status.go", + "table_history.go", "table_mapping.go", - "util.go", ], importpath = "github.com/pingcap/tidb/br/pkg/stream", visibility = ["//visibility:public"], @@ -25,6 +26,8 @@ go_library( "//br/pkg/restore/tiflashrec", "//br/pkg/storage", "//br/pkg/streamhelper", + "//br/pkg/utils", + "//br/pkg/utils/consts", "//br/pkg/utils/iter", "//pkg/ddl", "//pkg/kv", @@ -51,6 +54,7 @@ go_library( "@org_golang_x_sync//errgroup", "@org_uber_go_multierr//:multierr", "@org_uber_go_zap//:zap", + "@org_uber_go_zap//zapcore", ], ) @@ -65,14 +69,15 @@ go_test( "stream_metas_test.go", "stream_misc_test.go", "table_mapping_test.go", - "util_test.go", ], embed = [":stream"], flaky = True, - shard_count = 49, + shard_count = 50, deps = [ "//br/pkg/storage", "//br/pkg/streamhelper", + "//br/pkg/utils", + "//br/pkg/utils/consts", "//pkg/ddl", "//pkg/kv", "//pkg/meta", @@ -83,7 +88,6 @@ go_test( "//pkg/types", "//pkg/util/codec", "//pkg/util/intest", - "//pkg/util/table-filter", "@com_github_fsouza_fake_gcs_server//fakestorage", "@com_github_google_uuid//:uuid", "@com_github_pingcap_errors//:errors", @@ -91,7 +95,6 @@ go_test( "@com_github_pingcap_kvproto//pkg/brpb", "@com_github_pingcap_log//:log", "@com_github_stretchr_testify//require", - "@com_github_tikv_client_go_v2//oracle", "@org_golang_x_exp//maps", "@org_uber_go_multierr//:multierr", "@org_uber_go_zap//:zap", diff --git a/br/pkg/stream/logging_helper.go b/br/pkg/stream/logging_helper.go new file mode 100644 index 0000000000000..ab216a9835b2e --- /dev/null +++ b/br/pkg/stream/logging_helper.go @@ -0,0 +1,45 @@ +// Copyright 2024 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 stream + +import ( + "github.com/pingcap/log" + "go.uber.org/zap" + "go.uber.org/zap/zapcore" +) + +func LogDBReplaceMap(title string, dbReplaces map[UpstreamID]*DBReplace) { + for upstreamDbId, dbReplace := range dbReplaces { + log.Info(title, func() []zapcore.Field { + fields := make([]zapcore.Field, 0, (len(dbReplace.TableMap)+1)*3) + fields = append(fields, + zap.String("dbName", dbReplace.Name), + zap.Int64("upstreamId", upstreamDbId), + zap.Int64("downstreamId", dbReplace.DbID)) + for upstreamTableID, tableReplace := range dbReplace.TableMap { + fields = append(fields, + zap.String("table", tableReplace.Name), + zap.Int64("upstreamId", upstreamTableID), + zap.Int64("downstreamId", tableReplace.TableID)) + for upPartId, downPartId := range tableReplace.PartitionMap { + fields = append(fields, + zap.Int64("up partition", upPartId), + zap.Int64("down partition", downPartId)) + } + } + return fields + }()...) + } +} diff --git a/br/pkg/stream/meta_kv_test.go b/br/pkg/stream/meta_kv_test.go index 0ac5b54763022..9f64d69b3f5f6 100644 --- a/br/pkg/stream/meta_kv_test.go +++ b/br/pkg/stream/meta_kv_test.go @@ -6,18 +6,12 @@ import ( "bytes" "testing" + "github.com/pingcap/tidb/br/pkg/utils" "github.com/pingcap/tidb/pkg/meta" - "github.com/pingcap/tidb/pkg/tablecodec" "github.com/pingcap/tidb/pkg/util/codec" "github.com/stretchr/testify/require" ) -func encodeTxnMetaKey(key []byte, field []byte, ts uint64) []byte { - k := tablecodec.EncodeMetaKey(key, field) - txnKey := codec.EncodeBytes(nil, k) - return codec.EncodeUintDesc(txnKey, ts) -} - func TestRawMetaKeyForDB(t *testing.T) { var ( dbID int64 = 1 @@ -25,7 +19,7 @@ func TestRawMetaKeyForDB(t *testing.T) { mDbs = []byte("DBs") ) - txnKey := encodeTxnMetaKey(mDbs, meta.DBkey(dbID), ts) + txnKey := utils.EncodeTxnMetaKey(mDbs, meta.DBkey(dbID), ts) rawMetaKey, err := ParseTxnMetaKeyFrom(txnKey) require.NoError(t, err) @@ -44,7 +38,7 @@ func TestRawMetaKeyForTable(t *testing.T) { tableID int64 = 57 ts uint64 = 400036290571534337 ) - txnKey := encodeTxnMetaKey(meta.DBkey(dbID), meta.TableKey(tableID), ts) + txnKey := utils.EncodeTxnMetaKey(meta.DBkey(dbID), meta.TableKey(tableID), ts) rawMetakey, err := ParseTxnMetaKeyFrom(txnKey) require.NoError(t, err) diff --git a/br/pkg/stream/rewrite_meta_rawkv.go b/br/pkg/stream/rewrite_meta_rawkv.go index cebafbd247fb9..ec2fbb566e984 100644 --- a/br/pkg/stream/rewrite_meta_rawkv.go +++ b/br/pkg/stream/rewrite_meta_rawkv.go @@ -24,49 +24,45 @@ import ( berrors "github.com/pingcap/tidb/br/pkg/errors" "github.com/pingcap/tidb/br/pkg/restore/ingestrec" "github.com/pingcap/tidb/br/pkg/restore/tiflashrec" + "github.com/pingcap/tidb/br/pkg/utils" + "github.com/pingcap/tidb/br/pkg/utils/consts" "github.com/pingcap/tidb/pkg/ddl" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta" "github.com/pingcap/tidb/pkg/meta/model" - filter "github.com/pingcap/tidb/pkg/util/table-filter" "go.uber.org/zap" ) -// Default columnFamily and write columnFamily -const ( - DefaultCF = "default" - WriteCF = "write" -) - type UpstreamID = int64 type DownstreamID = int64 -// TableReplace specifies table information mapping from up-stream cluster to up-stream cluster. +// TableReplace specifies table information mapping from up-stream cluster to down-stream cluster. type TableReplace struct { Name string TableID DownstreamID PartitionMap map[UpstreamID]DownstreamID IndexMap map[UpstreamID]DownstreamID + Filtered bool } -// DBReplace specifies database information mapping from up-stream cluster to up-stream cluster. +// DBReplace specifies database information mapping from up-stream cluster to down-stream cluster. type DBReplace struct { Name string DbID DownstreamID TableMap map[UpstreamID]*TableReplace + Filtered bool } -// SchemasReplace specifies schemas information mapping from up-stream cluster to up-stream cluster. +// SchemasReplace specifies schemas information mapping from up-stream cluster to down-stream cluster. type SchemasReplace struct { - DbMap map[UpstreamID]*DBReplace + DbReplaceMap map[UpstreamID]*DBReplace delRangeRecorder *brDelRangeExecWrapper ingestRecorder *ingestrec.IngestRecorder TiflashRecorder *tiflashrec.TiFlashRecorder - RewriteTS uint64 // used to rewrite commit ts in meta kv. - TableFilter filter.Filter // used to filter schema/table + RewriteTS uint64 // used to rewrite commit ts in meta kv. - AfterTableRewritten func(deleted bool, tableInfo *model.TableInfo) + AfterTableRewrittenFn func(deleted bool, tableInfo *model.TableInfo) } // NewTableReplace creates a TableReplace struct. @@ -76,6 +72,7 @@ func NewTableReplace(name string, newID DownstreamID) *TableReplace { TableID: newID, PartitionMap: make(map[UpstreamID]DownstreamID), IndexMap: make(map[UpstreamID]DownstreamID), + Filtered: false, } } @@ -85,20 +82,26 @@ func NewDBReplace(name string, newID DownstreamID) *DBReplace { Name: name, DbID: newID, TableMap: make(map[UpstreamID]*TableReplace), + Filtered: false, } } // NewSchemasReplace creates a SchemasReplace struct. func NewSchemasReplace( - dbMap map[UpstreamID]*DBReplace, + dbReplaceMap map[UpstreamID]*DBReplace, tiflashRecorder *tiflashrec.TiFlashRecorder, restoreTS uint64, - tableFilter filter.Filter, recordDeleteRange func(*PreDelRangeQuery), ) *SchemasReplace { globalTableIdMap := make(map[UpstreamID]DownstreamID) - for _, dr := range dbMap { + for _, dr := range dbReplaceMap { + if dr.Filtered { + continue + } for tblID, tr := range dr.TableMap { + if tr.Filtered { + continue + } globalTableIdMap[tblID] = tr.TableID for oldpID, newpID := range tr.PartitionMap { globalTableIdMap[oldpID] = newpID @@ -107,12 +110,11 @@ func NewSchemasReplace( } return &SchemasReplace{ - DbMap: dbMap, + DbReplaceMap: dbReplaceMap, delRangeRecorder: newDelRangeExecWrapper(globalTableIdMap, recordDeleteRange), ingestRecorder: ingestrec.New(), TiflashRecorder: tiflashRecorder, RewriteTS: restoreTS, - TableFilter: tableFilter, } } @@ -127,13 +129,16 @@ func (sr *SchemasReplace) rewriteKeyForDB(key []byte, cf string) ([]byte, error) return nil, errors.Trace(err) } - dbMap, exist := sr.DbMap[dbID] + dbMap, exist := sr.DbReplaceMap[dbID] if !exist { return nil, errors.Annotatef(berrors.ErrInvalidArgument, "failed to find db id:%v in maps", dbID) } + if dbMap.Filtered { + return nil, nil + } rawMetaKey.UpdateField(meta.DBkey(dbMap.DbID)) - if cf == WriteCF { + if cf == consts.WriteCF { rawMetaKey.UpdateTS(sr.RewriteTS) } return rawMetaKey.EncodeMetaKey(), nil @@ -145,10 +150,13 @@ func (sr *SchemasReplace) rewriteDBInfo(value []byte) ([]byte, error) { return nil, errors.Trace(err) } - dbMap, exist := sr.DbMap[dbInfo.ID] + dbMap, exist := sr.DbReplaceMap[dbInfo.ID] if !exist { return nil, errors.Annotatef(berrors.ErrInvalidArgument, "failed to find db id:%v in maps", dbInfo.ID) } + if dbMap.Filtered { + return nil, nil + } dbInfo.ID = dbMap.DbID newValue, err := json.Marshal(dbInfo) @@ -175,6 +183,9 @@ func (sr *SchemasReplace) rewriteEntryForDB(e *kv.Entry, cf string) (*kv.Entry, if err != nil { return nil, errors.Trace(err) } + if newKey == nil { + return nil, nil + } return &kv.Entry{Key: newKey, Value: newValue}, nil } @@ -204,19 +215,25 @@ func (sr *SchemasReplace) rewriteKeyForTable( return nil, errors.Trace(err) } - dbReplace, exist := sr.DbMap[dbID] + dbReplace, exist := sr.DbReplaceMap[dbID] if !exist { return nil, errors.Annotatef(berrors.ErrInvalidArgument, "failed to find db id:%v in maps", dbID) } + if dbReplace.Filtered { + return nil, nil + } tableReplace, exist := dbReplace.TableMap[tableID] if !exist { return nil, errors.Annotatef(berrors.ErrInvalidArgument, "failed to find table id:%v in maps", tableID) } + if tableReplace.Filtered { + return nil, nil + } rawMetaKey.UpdateKey(meta.DBkey(dbReplace.DbID)) rawMetaKey.UpdateField(encodeField(tableReplace.TableID)) - if cf == WriteCF { + if cf == consts.WriteCF { rawMetaKey.UpdateTS(sr.RewriteTS) } return rawMetaKey.EncodeMetaKey(), nil @@ -235,15 +252,21 @@ func (sr *SchemasReplace) rewriteTableInfo(value []byte, dbID int64) ([]byte, er } // construct or find the id map. - dbReplace, exist = sr.DbMap[dbID] + dbReplace, exist = sr.DbReplaceMap[dbID] if !exist { return nil, errors.Annotatef(berrors.ErrInvalidArgument, "failed to find db id:%v in maps", dbID) } + if dbReplace.Filtered { + return nil, nil + } tableReplace, exist = dbReplace.TableMap[tableInfo.ID] if !exist { return nil, errors.Annotatef(berrors.ErrInvalidArgument, "failed to find table id:%v in maps", tableInfo.ID) } + if tableReplace.Filtered { + return nil, nil + } // update table ID and partition ID. tableInfo.ID = tableReplace.TableID @@ -263,8 +286,8 @@ func (sr *SchemasReplace) rewriteTableInfo(value []byte, dbID int64) ([]byte, er if tableInfo.TTLInfo != nil { tableInfo.TTLInfo.Enable = false } - if sr.AfterTableRewritten != nil { - sr.AfterTableRewritten(false, &tableInfo) + if sr.AfterTableRewrittenFn != nil { + sr.AfterTableRewrittenFn(false, &tableInfo) } // marshal to json @@ -300,13 +323,17 @@ func (sr *SchemasReplace) rewriteEntryForTable(e *kv.Entry, cf string) (*kv.Entr if err != nil { return nil, errors.Trace(err) } + // got filtered + if newKey == nil { + return nil, nil + } // NOTE: the normal path is in the `SchemaReplace.rewriteTableInfo` // for now, we rewrite key and value separately hence we cannot // get a view of (is_delete, table_id, table_info) at the same time :(. // Maybe we can extract the rewrite part from rewriteTableInfo. - if result.Deleted && sr.AfterTableRewritten != nil { - sr.AfterTableRewritten(true, &model.TableInfo{ID: newTableID}) + if result.Deleted && sr.AfterTableRewrittenFn != nil { + sr.AfterTableRewrittenFn(true, &model.TableInfo{ID: newTableID}) } return &kv.Entry{Key: newKey, Value: result.NewValue}, nil @@ -322,6 +349,9 @@ func (sr *SchemasReplace) rewriteEntryForAutoIncrementIDKey(e *kv.Entry, cf stri if err != nil { return nil, errors.Trace(err) } + if newKey == nil { + return nil, nil + } return &kv.Entry{Key: newKey, Value: e.Value}, nil } @@ -336,6 +366,9 @@ func (sr *SchemasReplace) rewriteEntryForAutoTableIDKey(e *kv.Entry, cf string) if err != nil { return nil, errors.Trace(err) } + if newKey == nil { + return nil, nil + } return &kv.Entry{Key: newKey, Value: e.Value}, nil } @@ -350,6 +383,9 @@ func (sr *SchemasReplace) rewriteEntryForSequenceKey(e *kv.Entry, cf string) (*k if err != nil { return nil, errors.Trace(err) } + if newKey == nil { + return nil, nil + } return &kv.Entry{Key: newKey, Value: e.Value}, nil } @@ -364,6 +400,9 @@ func (sr *SchemasReplace) rewriteEntryForAutoRandomTableIDKey(e *kv.Entry, cf st if err != nil { return nil, errors.Trace(err) } + if newKey == nil { + return nil, nil + } return &kv.Entry{Key: newKey, Value: e.Value}, nil } @@ -374,10 +413,10 @@ type rewriteResult struct { } // rewriteValue rewrite the value if cf is "default", or rewrite the shortValue if cf is "write". -func (sr *SchemasReplace) rewriteValue(value []byte, cf string, rewrite func([]byte) ([]byte, error)) (rewriteResult, error) { +func (sr *SchemasReplace) rewriteValue(value []byte, cf string, rewriteFunc func([]byte) ([]byte, error)) (rewriteResult, error) { switch cf { - case DefaultCF: - newValue, err := rewrite(value) + case consts.DefaultCF: + newValue, err := rewriteFunc(value) if err != nil { return rewriteResult{}, errors.Trace(err) } @@ -385,7 +424,7 @@ func (sr *SchemasReplace) rewriteValue(value []byte, cf string, rewrite func([]b NewValue: newValue, Deleted: false, }, nil - case WriteCF: + case consts.WriteCF: rawWriteCFValue := new(RawWriteCFValue) if err := rawWriteCFValue.ParseFrom(value); err != nil { return rewriteResult{}, errors.Trace(err) @@ -409,7 +448,7 @@ func (sr *SchemasReplace) rewriteValue(value []byte, cf string, rewrite func([]b }, nil } - shortValue, err := rewrite(rawWriteCFValue.GetShortValue()) + shortValue, err := rewriteFunc(rawWriteCFValue.GetShortValue()) if err != nil { log.Info("failed to rewrite short value", zap.ByteString("write-type", []byte{rawWriteCFValue.GetWriteType()}), @@ -428,11 +467,12 @@ func (sr *SchemasReplace) GetIngestRecorder() *ingestrec.IngestRecorder { return sr.ingestRecorder } -// RewriteKvEntry uses to rewrite tableID/dbID in entry.key and entry.value -func (sr *SchemasReplace) RewriteKvEntry(e *kv.Entry, cf string) (*kv.Entry, error) { - // skip mDDLJob - if !IsMetaDBKey(e.Key) { - if cf == DefaultCF && IsMetaDDLJobHistoryKey(e.Key) { // mDDLJobHistory +// RewriteMetaKvEntry uses to rewrite tableID/dbID in entry.key and entry.value +func (sr *SchemasReplace) RewriteMetaKvEntry(e *kv.Entry, cf string) (*kv.Entry, error) { + if !utils.IsMetaDBKey(e.Key) { + // need to special handle ddl job history during actual restore phase. The job history contains index ingestion + // and range deletion that need to be handled separately after restore. + if cf == consts.DefaultCF && utils.IsMetaDDLJobHistoryKey(e.Key) { // mDDLJobHistory job := &model.Job{} if err := job.Decode(e.Value); err != nil { log.Debug("failed to decode the job", @@ -442,7 +482,7 @@ func (sr *SchemasReplace) RewriteKvEntry(e *kv.Entry, cf string) (*kv.Entry, err return nil, nil } - return nil, sr.restoreFromHistory(job) + return nil, sr.processIngestIndexAndDeleteRangeFromJob(job) } return nil, nil } @@ -457,6 +497,7 @@ func (sr *SchemasReplace) RewriteKvEntry(e *kv.Entry, cf string) (*kv.Entry, err } else if !meta.IsDBkey(rawKey.Key) { return nil, nil } + if meta.IsTableKey(rawKey.Field) { return sr.rewriteEntryForTable(e, cf) } else if meta.IsAutoIncrementIDKey(rawKey.Field) { @@ -486,7 +527,10 @@ func (sr *SchemasReplace) tryRecordIngestIndex(job *model.Job) error { return nil } -func (sr *SchemasReplace) restoreFromHistory(job *model.Job) error { +// processIngestIndexAndDeleteRangeFromJob handles two special cases during log backup meta key replay. +// 1. index ingestion is not captured by the log backup, thus we need to restore them manually later +// 2. delete range also needs to be handled to clean up dropped table since it was previously relying on GC to clean it up +func (sr *SchemasReplace) processIngestIndexAndDeleteRangeFromJob(job *model.Job) error { if ddl.JobNeedGC(job) { if err := ddl.AddDelRangeJobInternal(context.TODO(), sr.delRangeRecorder, job); err != nil { return err @@ -543,7 +587,8 @@ func (bdr *brDelRangeExecWrapper) PrepareParamsList(sz int) { func (bdr *brDelRangeExecWrapper) RewriteTableID(tableID int64) (int64, bool) { newTableID, exists := bdr.globalTableIdMap[tableID] if !exists { - log.Warn("failed to find the downstream id when rewrite delete range", zap.Int64("old tableID", tableID)) + log.Warn("failed to find the downstream id when rewrite delete range, "+ + "it might due to table has been filtered out if filters have been specified", zap.Int64("old tableID", tableID)) } return newTableID, exists } diff --git a/br/pkg/stream/rewrite_meta_rawkv_test.go b/br/pkg/stream/rewrite_meta_rawkv_test.go index 6c9b4e846105f..7cfca5625438b 100644 --- a/br/pkg/stream/rewrite_meta_rawkv_test.go +++ b/br/pkg/stream/rewrite_meta_rawkv_test.go @@ -7,14 +7,16 @@ import ( "encoding/json" "testing" + "github.com/pingcap/tidb/br/pkg/utils" + "github.com/pingcap/tidb/br/pkg/utils/consts" "github.com/pingcap/tidb/pkg/ddl" + "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta" "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/tablecodec" "github.com/pingcap/tidb/pkg/types" - filter "github.com/pingcap/tidb/pkg/util/table-filter" "github.com/stretchr/testify/require" ) @@ -29,7 +31,6 @@ func MockEmptySchemasReplace(midr *mockInsertDeleteRange, dbMap map[UpstreamID]* dbMap, nil, 9527, - filter.All(), midr.mockRecordDeleteRange, ) } @@ -59,7 +60,7 @@ func TestRewriteKeyForDB(t *testing.T) { mDbs = []byte("DBs") ) - encodedKey := encodeTxnMetaKey(mDbs, meta.DBkey(dbID), ts) + encodedKey := utils.EncodeTxnMetaKey(mDbs, meta.DBkey(dbID), ts) dbMap := make(map[UpstreamID]*DBReplace) downstreamID := dbID + 100 @@ -69,7 +70,7 @@ func TestRewriteKeyForDB(t *testing.T) { sr := MockEmptySchemasReplace(nil, dbMap) // set restoreKV status and rewrite it. - newKey, err := sr.rewriteKeyForDB(encodedKey, DefaultCF) + newKey, err := sr.rewriteKeyForDB(encodedKey, consts.DefaultCF) require.Nil(t, err) decodedKey, err := ParseTxnMetaKeyFrom(newKey) require.Nil(t, err) @@ -79,7 +80,7 @@ func TestRewriteKeyForDB(t *testing.T) { require.Equal(t, newDBID, downstreamID) // rewrite it again, and get the same result. - newKey, err = sr.rewriteKeyForDB(encodedKey, WriteCF) + newKey, err = sr.rewriteKeyForDB(encodedKey, consts.WriteCF) require.Nil(t, err) decodedKey, err = ParseTxnMetaKeyFrom(newKey) require.Nil(t, err) @@ -110,16 +111,16 @@ func TestRewriteDBInfo(t *testing.T) { require.Nil(t, err) err = json.Unmarshal(newValue, &DBInfo) require.Nil(t, err) - require.Equal(t, DBInfo.ID, sr.DbMap[dbID].DbID) + require.Equal(t, DBInfo.ID, sr.DbReplaceMap[dbID].DbID) // rewrite again, and get the same result. - newId := sr.DbMap[dbID].DbID + newId := sr.DbReplaceMap[dbID].DbID newValue, err = sr.rewriteDBInfo(value) require.Nil(t, err) err = json.Unmarshal(newValue, &DBInfo) require.Nil(t, err) - require.Equal(t, DBInfo.ID, sr.DbMap[dbID].DbID) - require.Equal(t, newId, sr.DbMap[dbID].DbID) + require.Equal(t, DBInfo.ID, sr.DbReplaceMap[dbID].DbID) + require.Equal(t, newId, sr.DbReplaceMap[dbID].DbID) } func TestRewriteKeyForTable(t *testing.T) { @@ -157,7 +158,7 @@ func TestRewriteKeyForTable(t *testing.T) { } for _, ca := range cases { - encodedKey := encodeTxnMetaKey(meta.DBkey(dbID), ca.encodeTableFn(tableID), ts) + encodedKey := utils.EncodeTxnMetaKey(meta.DBkey(dbID), ca.encodeTableFn(tableID), ts) dbMap := make(map[UpstreamID]*DBReplace) downStreamDbID := dbID + 100 @@ -169,7 +170,7 @@ func TestRewriteKeyForTable(t *testing.T) { sr := MockEmptySchemasReplace(nil, dbMap) // set restoreKV status and rewrite it. - newKey, err := sr.rewriteKeyForTable(encodedKey, DefaultCF, ca.decodeTableFn, ca.encodeTableFn) + newKey, err := sr.rewriteKeyForTable(encodedKey, consts.DefaultCF, ca.decodeTableFn, ca.encodeTableFn) require.Nil(t, err) decodedKey, err := ParseTxnMetaKeyFrom(newKey) require.Nil(t, err) @@ -183,7 +184,7 @@ func TestRewriteKeyForTable(t *testing.T) { require.Equal(t, newTblID, downStreamTblID) // rewrite it again, and get the same result. - newKey, err = sr.rewriteKeyForTable(encodedKey, WriteCF, ca.decodeTableFn, ca.encodeTableFn) + newKey, err = sr.rewriteKeyForTable(encodedKey, consts.WriteCF, ca.decodeTableFn, ca.encodeTableFn) require.Nil(t, err) decodedKey, err = ParseTxnMetaKeyFrom(newKey) require.Nil(t, err) @@ -217,7 +218,7 @@ func TestRewriteTableInfo(t *testing.T) { // create schemasReplace. sr := MockEmptySchemasReplace(nil, dbMap) tableCount := 0 - sr.AfterTableRewritten = func(deleted bool, tableInfo *model.TableInfo) { + sr.AfterTableRewrittenFn = func(deleted bool, tableInfo *model.TableInfo) { tableCount++ tableInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ Count: 1, @@ -229,17 +230,17 @@ func TestRewriteTableInfo(t *testing.T) { require.Nil(t, err) err = json.Unmarshal(newValue, &tableInfo) require.Nil(t, err) - require.Equal(t, tableInfo.ID, sr.DbMap[dbId].TableMap[tableID].TableID) + require.Equal(t, tableInfo.ID, sr.DbReplaceMap[dbId].TableMap[tableID].TableID) require.EqualValues(t, tableInfo.TiFlashReplica.Count, 1) // rewrite it again and get the same result. - newID := sr.DbMap[dbId].TableMap[tableID].TableID + newID := sr.DbReplaceMap[dbId].TableMap[tableID].TableID newValue, err = sr.rewriteTableInfo(value, dbId) require.Nil(t, err) err = json.Unmarshal(newValue, &tableInfo) require.Nil(t, err) - require.Equal(t, tableInfo.ID, sr.DbMap[dbId].TableMap[tableID].TableID) - require.Equal(t, newID, sr.DbMap[dbId].TableMap[tableID].TableID) + require.Equal(t, tableInfo.ID, sr.DbReplaceMap[dbId].TableMap[tableID].TableID) + require.Equal(t, newID, sr.DbReplaceMap[dbId].TableMap[tableID].TableID) require.EqualValues(t, tableCount, 2) } @@ -291,7 +292,6 @@ func TestRewriteTableInfoForPartitionTable(t *testing.T) { dbMap, nil, 0, - filter.All(), nil, ) @@ -301,11 +301,11 @@ func TestRewriteTableInfoForPartitionTable(t *testing.T) { 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].TableID) + require.Equal(t, tableInfo.ID, sr.DbReplaceMap[dbId].TableMap[tableID].TableID) require.Equal( t, tableInfo.Partition.Definitions[0].ID, - sr.DbMap[dbId].TableMap[tableID].PartitionMap[pt1ID], + sr.DbReplaceMap[dbId].TableMap[tableID].PartitionMap[pt1ID], ) require.Equal( t, @@ -315,7 +315,7 @@ func TestRewriteTableInfoForPartitionTable(t *testing.T) { require.Equal( t, tableInfo.Partition.Definitions[1].ID, - sr.DbMap[dbId].TableMap[tableID].PartitionMap[pt2ID], + sr.DbReplaceMap[dbId].TableMap[tableID].PartitionMap[pt2ID], ) require.Equal( t, @@ -324,8 +324,8 @@ func TestRewriteTableInfoForPartitionTable(t *testing.T) { ) // rewrite it aggin, and get the same result. - newID1 := sr.DbMap[dbId].TableMap[tableID].PartitionMap[pt1ID] - newID2 := sr.DbMap[dbId].TableMap[tableID].PartitionMap[pt2ID] + newID1 := sr.DbReplaceMap[dbId].TableMap[tableID].PartitionMap[pt1ID] + newID2 := sr.DbReplaceMap[dbId].TableMap[tableID].PartitionMap[pt2ID] newValue, err = sr.rewriteTableInfo(value, dbId) require.Nil(t, err) @@ -335,13 +335,13 @@ func TestRewriteTableInfoForPartitionTable(t *testing.T) { require.Equal( t, tableInfo.Partition.Definitions[0].ID, - sr.DbMap[dbId].TableMap[tableID].PartitionMap[pt1ID], + sr.DbReplaceMap[dbId].TableMap[tableID].PartitionMap[pt1ID], ) require.Equal(t, tableInfo.Partition.Definitions[0].ID, newID1) require.Equal( t, tableInfo.Partition.Definitions[1].ID, - sr.DbMap[dbId].TableMap[tableID].PartitionMap[pt2ID], + sr.DbReplaceMap[dbId].TableMap[tableID].PartitionMap[pt2ID], ) require.Equal(t, tableInfo.Partition.Definitions[1].ID, newID2) } @@ -360,6 +360,7 @@ func TestRewriteTableInfoForExchangePartition(t *testing.T) { tableID2 int64 = 106 tableName2 = "t2" tableInfo model.TableInfo + ts uint64 = 400036290571534337 ) // construct table t1 with the partition pi(pt1, pt2). @@ -401,7 +402,8 @@ func TestRewriteTableInfoForExchangePartition(t *testing.T) { dbMap[dbID2] = NewDBReplace(db2.Name.O, dbID2+100) dbMap[dbID2].TableMap[tableID2] = NewTableReplace(t2.Name.O, tableID2+100) - tc := NewTableMappingManager(dbMap, mockGenGlobalID) + tm := NewTableMappingManager() + tm.MergeBaseDBReplace(dbMap) //exchange partition, t1 partition0 with the t2 t1Copy := t1.Clone() @@ -411,14 +413,19 @@ func TestRewriteTableInfoForExchangePartition(t *testing.T) { value, err := json.Marshal(&t1Copy) require.Nil(t, err) - err = tc.parseTableValueAndUpdateIdMapping(dbID1, value) + // Create an entry for parsing + txnKey := utils.EncodeTxnMetaKey(meta.DBkey(dbID1), meta.TableKey(tableID1), ts) + entry := &kv.Entry{ + Key: txnKey, + Value: value, + } + err = tm.ParseMetaKvAndUpdateIdMapping(entry, consts.DefaultCF) require.Nil(t, err) sr := NewSchemasReplace( - tc.DbReplaceMap, + tm.DBReplaceMap, nil, 0, - filter.All(), nil, ) @@ -434,8 +441,16 @@ func TestRewriteTableInfoForExchangePartition(t *testing.T) { // rewrite no partition table value, err = json.Marshal(&t2Copy) require.Nil(t, err) - err = tc.parseTableValueAndUpdateIdMapping(dbID2, value) + + // Create an entry for parsing the second table + txnKey = utils.EncodeTxnMetaKey(meta.DBkey(dbID2), meta.TableKey(pt1ID), ts) + entry = &kv.Entry{ + Key: txnKey, + Value: value, + } + err = tm.ParseMetaKvAndUpdateIdMapping(entry, consts.DefaultCF) require.Nil(t, err) + value, err = sr.rewriteTableInfo(value, dbID2) require.Nil(t, err) err = json.Unmarshal(value, &tableInfo) @@ -488,7 +503,7 @@ func TestRewriteTableInfoForTTLTable(t *testing.T) { 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].TableID) + require.Equal(t, tableInfo.ID, sr.DbReplaceMap[dbId].TableMap[tableID].TableID) require.NotNil(t, tableInfo.TTLInfo) require.Equal(t, colName, tableInfo.TTLInfo.ColumnName.O) require.Equal(t, "1", tableInfo.TTLInfo.IntervalExprStr) @@ -705,7 +720,7 @@ func TestDeleteRangeForMDDLJob(t *testing.T) { var qargs *PreDelRangeQuery // drop schema - err := schemaReplace.restoreFromHistory(dropSchemaJob) + err := schemaReplace.processIngestIndexAndDeleteRangeFromJob(dropSchemaJob) require.NoError(t, err) qargs = <-midr.queryCh require.Equal(t, len(qargs.ParamsList), len(mDDLJobALLNewTableIDSet)) @@ -715,7 +730,7 @@ func TestDeleteRangeForMDDLJob(t *testing.T) { } // drop table0 - err = schemaReplace.restoreFromHistory(dropTable0Job) + err = schemaReplace.processIngestIndexAndDeleteRangeFromJob(dropTable0Job) require.NoError(t, err) qargs = <-midr.queryCh require.Equal(t, len(qargs.ParamsList), len(mDDLJobALLNewPartitionIDSet)) @@ -728,42 +743,42 @@ func TestDeleteRangeForMDDLJob(t *testing.T) { require.Equal(t, qargs.ParamsList[0].StartKey, encodeTableKey(mDDLJobTable0NewID)) // drop table1 - err = schemaReplace.restoreFromHistory(dropTable1Job) + err = schemaReplace.processIngestIndexAndDeleteRangeFromJob(dropTable1Job) require.NoError(t, err) qargs = <-midr.queryCh require.Equal(t, len(qargs.ParamsList), 1) require.Equal(t, qargs.ParamsList[0].StartKey, encodeTableKey(mDDLJobTable1NewID)) // drop table partition1 - err = schemaReplace.restoreFromHistory(dropTable0Partition1Job) + err = schemaReplace.processIngestIndexAndDeleteRangeFromJob(dropTable0Partition1Job) require.NoError(t, err) qargs = <-midr.queryCh require.Equal(t, len(qargs.ParamsList), 1) require.Equal(t, qargs.ParamsList[0].StartKey, encodeTableKey(mDDLJobPartition1NewID)) // reorganize table partition1 - err = schemaReplace.restoreFromHistory(reorganizeTable0Partition1Job) + err = schemaReplace.processIngestIndexAndDeleteRangeFromJob(reorganizeTable0Partition1Job) require.NoError(t, err) qargs = <-midr.queryCh require.Equal(t, len(qargs.ParamsList), 1) require.Equal(t, encodeTableKey(mDDLJobPartition1NewID), qargs.ParamsList[0].StartKey) // remove table partition1 - err = schemaReplace.restoreFromHistory(removeTable0Partition1Job) + err = schemaReplace.processIngestIndexAndDeleteRangeFromJob(removeTable0Partition1Job) require.NoError(t, err) qargs = <-midr.queryCh require.Equal(t, len(qargs.ParamsList), 1) require.Equal(t, encodeTableKey(mDDLJobPartition1NewID), qargs.ParamsList[0].StartKey) // alter table partition1 - err = schemaReplace.restoreFromHistory(alterTable0Partition1Job) + err = schemaReplace.processIngestIndexAndDeleteRangeFromJob(alterTable0Partition1Job) require.NoError(t, err) qargs = <-midr.queryCh require.Equal(t, len(qargs.ParamsList), 1) require.Equal(t, encodeTableKey(mDDLJobPartition1NewID), qargs.ParamsList[0].StartKey) // roll back add index for table0 - err = schemaReplace.restoreFromHistory(rollBackTable0IndexJob) + err = schemaReplace.processIngestIndexAndDeleteRangeFromJob(rollBackTable0IndexJob) require.NoError(t, err) oldPartitionIDMap := make(map[string]struct{}) for i := 0; i < len(mDDLJobALLNewPartitionIDSet); i++ { @@ -784,7 +799,7 @@ func TestDeleteRangeForMDDLJob(t *testing.T) { } // roll back add index for table1 - err = schemaReplace.restoreFromHistory(rollBackTable1IndexJob) + err = schemaReplace.processIngestIndexAndDeleteRangeFromJob(rollBackTable1IndexJob) require.NoError(t, err) qargs = <-midr.queryCh require.Equal(t, len(qargs.ParamsList), 2) @@ -794,7 +809,7 @@ func TestDeleteRangeForMDDLJob(t *testing.T) { require.Equal(t, encodeTableIndexKey(mDDLJobTable1NewID, int64(tablecodec.TempIndexPrefix|2)), qargs.ParamsList[1].StartKey) // drop index for table0 - err = schemaReplace.restoreFromHistory(dropTable0IndexJob) + err = schemaReplace.processIngestIndexAndDeleteRangeFromJob(dropTable0IndexJob) require.NoError(t, err) oldPartitionIDMap = make(map[string]struct{}) for i := 0; i < len(mDDLJobALLNewPartitionIDSet); i++ { @@ -808,14 +823,14 @@ func TestDeleteRangeForMDDLJob(t *testing.T) { } // drop index for table1 - err = schemaReplace.restoreFromHistory(dropTable1IndexJob) + err = schemaReplace.processIngestIndexAndDeleteRangeFromJob(dropTable1IndexJob) require.NoError(t, err) qargs = <-midr.queryCh require.Equal(t, len(qargs.ParamsList), 1) require.Equal(t, encodeTableIndexKey(mDDLJobTable1NewID, int64(2)), qargs.ParamsList[0].StartKey) // add index for table 0 - err = schemaReplace.restoreFromHistory(addTable0IndexJob) + err = schemaReplace.processIngestIndexAndDeleteRangeFromJob(addTable0IndexJob) require.NoError(t, err) oldPartitionIDMap = make(map[string]struct{}) for i := 0; i < len(mDDLJobALLNewPartitionIDSet); i++ { @@ -829,14 +844,14 @@ func TestDeleteRangeForMDDLJob(t *testing.T) { } // add index for table 1 - err = schemaReplace.restoreFromHistory(addTable1IndexJob) + err = schemaReplace.processIngestIndexAndDeleteRangeFromJob(addTable1IndexJob) require.NoError(t, err) qargs = <-midr.queryCh require.Equal(t, len(qargs.ParamsList), 1) require.Equal(t, encodeTableIndexKey(mDDLJobTable1NewID, tempIndex2), qargs.ParamsList[0].StartKey) // drop column for table0 - err = schemaReplace.restoreFromHistory(dropTable0ColumnJob) + err = schemaReplace.processIngestIndexAndDeleteRangeFromJob(dropTable0ColumnJob) require.NoError(t, err) oldPartitionIDMap = make(map[string]struct{}) for i := 0; i < len(mDDLJobALLNewPartitionIDSet); i++ { @@ -857,7 +872,7 @@ func TestDeleteRangeForMDDLJob(t *testing.T) { } // drop column for table1 - err = schemaReplace.restoreFromHistory(dropTable1ColumnJob) + err = schemaReplace.processIngestIndexAndDeleteRangeFromJob(dropTable1ColumnJob) require.NoError(t, err) qargs = <-midr.queryCh require.Equal(t, len(qargs.ParamsList), len(mDDLJobALLIndexesIDSet)) @@ -867,7 +882,7 @@ func TestDeleteRangeForMDDLJob(t *testing.T) { require.Equal(t, encodeTableIndexKey(mDDLJobTable1NewID, int64(3)), qargs.ParamsList[1].StartKey) // modify column for table0 - err = schemaReplace.restoreFromHistory(modifyTable0ColumnJob) + err = schemaReplace.processIngestIndexAndDeleteRangeFromJob(modifyTable0ColumnJob) require.NoError(t, err) oldPartitionIDMap = make(map[string]struct{}) for i := 0; i < len(mDDLJobALLNewPartitionIDSet); i++ { @@ -888,7 +903,7 @@ func TestDeleteRangeForMDDLJob(t *testing.T) { } // modify column for table1 - err = schemaReplace.restoreFromHistory(modifyTable1ColumnJob) + err = schemaReplace.processIngestIndexAndDeleteRangeFromJob(modifyTable1ColumnJob) require.NoError(t, err) qargs = <-midr.queryCh require.Equal(t, len(qargs.ParamsList), len(mDDLJobALLIndexesIDSet)) @@ -898,7 +913,7 @@ func TestDeleteRangeForMDDLJob(t *testing.T) { require.Equal(t, encodeTableIndexKey(mDDLJobTable1NewID, int64(3)), qargs.ParamsList[1].StartKey) // drop indexes(multi-schema-change) for table0 - err = schemaReplace.restoreFromHistory(multiSchemaChangeJob0) + err = schemaReplace.processIngestIndexAndDeleteRangeFromJob(multiSchemaChangeJob0) require.NoError(t, err) oldPartitionIDMap = make(map[string]struct{}) for l := 0; l < 2; l++ { @@ -914,7 +929,7 @@ func TestDeleteRangeForMDDLJob(t *testing.T) { } // drop indexes(multi-schema-change) for table1 - err = schemaReplace.restoreFromHistory(multiSchemaChangeJob1) + err = schemaReplace.processIngestIndexAndDeleteRangeFromJob(multiSchemaChangeJob1) require.NoError(t, err) qargs = <-midr.queryCh require.Equal(t, len(qargs.ParamsList), 1) @@ -952,7 +967,7 @@ func TestDeleteRangeForMDDLJob2(t *testing.T) { }) var qargs *PreDelRangeQuery // drop schema - err := schemaReplace.restoreFromHistory(dropSchemaJob) + err := schemaReplace.processIngestIndexAndDeleteRangeFromJob(dropSchemaJob) require.NoError(t, err) qargs = <-midr.queryCh require.Equal(t, len(qargs.ParamsList), len(mDDLJobALLNewTableIDSet)) @@ -970,7 +985,7 @@ func TestDeleteRangeForMDDLJob2(t *testing.T) { schemaReplace = MockEmptySchemasReplace(midr, map[int64]*DBReplace{ mDDLJobDBOldID: dbReplace, }) - err = schemaReplace.restoreFromHistory(dropSchemaJob) + err = schemaReplace.processIngestIndexAndDeleteRangeFromJob(dropSchemaJob) require.NoError(t, err) qargs = <-midr.queryCh require.Equal(t, len(qargs.ParamsList), len(mDDLJobALLNewPartitionIDSet)+1) diff --git a/br/pkg/stream/search.go b/br/pkg/stream/search.go index 7cf940a42f135..ea3664739576c 100644 --- a/br/pkg/stream/search.go +++ b/br/pkg/stream/search.go @@ -16,6 +16,7 @@ import ( backuppb "github.com/pingcap/kvproto/pkg/brpb" "github.com/pingcap/log" "github.com/pingcap/tidb/br/pkg/storage" + "github.com/pingcap/tidb/br/pkg/utils/consts" "github.com/pingcap/tidb/pkg/util" "github.com/pingcap/tidb/pkg/util/codec" "go.uber.org/zap" @@ -193,9 +194,9 @@ func (s *StreamBackupSearch) Search(ctx context.Context) ([]*StreamKVInfo, error writeCFEntries := make(map[string]*StreamKVInfo, 64) for entry := range entriesCh { - if entry.CFName == WriteCF { + if entry.CFName == consts.WriteCF { writeCFEntries[entry.EncodedKey] = entry - } else if entry.CFName == DefaultCF { + } else if entry.CFName == consts.DefaultCF { defaultCFEntries[entry.EncodedKey] = entry } } @@ -241,7 +242,7 @@ func (s *StreamBackupSearch) searchFromDataFile( return errors.Annotatef(err, "decode raw key error, file: %s", dataFile.Path) } - if dataFile.Cf == WriteCF { + if dataFile.Cf == consts.WriteCF { rawWriteCFValue := new(RawWriteCFValue) if err := rawWriteCFValue.ParseFrom(v); err != nil { return errors.Annotatef(err, "parse raw write cf value error, file: %s", dataFile.Path) @@ -262,7 +263,7 @@ func (s *StreamBackupSearch) searchFromDataFile( ShortValue: valueStr, } ch <- kvInfo - } else if dataFile.Cf == DefaultCF { + } else if dataFile.Cf == consts.DefaultCF { kvInfo := &StreamKVInfo{ CFName: dataFile.Cf, StartTs: ts, diff --git a/br/pkg/stream/search_test.go b/br/pkg/stream/search_test.go index 224beb5ac7403..2cf3b74b8efe0 100644 --- a/br/pkg/stream/search_test.go +++ b/br/pkg/stream/search_test.go @@ -13,6 +13,7 @@ import ( backuppb "github.com/pingcap/kvproto/pkg/brpb" "github.com/pingcap/tidb/br/pkg/storage" + "github.com/pingcap/tidb/br/pkg/utils/consts" "github.com/pingcap/tidb/pkg/util/codec" "github.com/stretchr/testify/require" ) @@ -121,7 +122,7 @@ func fakeDataFile(t *testing.T, s storage.ExternalStorage) (defaultCFDataFile, w defaultCFCheckSum := sha256.Sum256(defaultCFBuf.Bytes()) defaultCFDataFile = &backuppb.DataFileInfo{ Path: defaultCFFile, - Cf: DefaultCF, + Cf: consts.DefaultCF, Sha256: defaultCFCheckSum[:], } @@ -135,7 +136,7 @@ func fakeDataFile(t *testing.T, s storage.ExternalStorage) (defaultCFDataFile, w writeCFCheckSum := sha256.Sum256(writeCFBuf.Bytes()) writeCFDataFile = &backuppb.DataFileInfo{ Path: writeCFFile, - Cf: WriteCF, + Cf: consts.WriteCF, Sha256: writeCFCheckSum[:], } @@ -178,7 +179,7 @@ func TestMergeCFEntries(t *testing.T) { Key: hex.EncodeToString([]byte(defaultCF.key)), EncodedKey: encodedKey, StartTs: uint64(defaultCF.startTs), - CFName: DefaultCF, + CFName: consts.DefaultCF, Value: defaultCF.val, } } @@ -189,7 +190,7 @@ func TestMergeCFEntries(t *testing.T) { EncodedKey: encodedKey, StartTs: uint64(writeCF.startTs), CommitTs: uint64(writeCF.commitTS), - CFName: WriteCF, + CFName: consts.WriteCF, Value: writeCF.val, } } diff --git a/br/pkg/stream/stream_metas.go b/br/pkg/stream/stream_metas.go index 2b88e7355125f..b48ed97e3a1f3 100644 --- a/br/pkg/stream/stream_metas.go +++ b/br/pkg/stream/stream_metas.go @@ -29,6 +29,7 @@ import ( "github.com/pingcap/tidb/br/pkg/glue" "github.com/pingcap/tidb/br/pkg/logutil" "github.com/pingcap/tidb/br/pkg/storage" + "github.com/pingcap/tidb/br/pkg/utils/consts" "github.com/pingcap/tidb/br/pkg/utils/iter" "github.com/pingcap/tidb/pkg/util" "github.com/pingcap/tidb/pkg/util/mathutil" @@ -302,7 +303,7 @@ func UpdateShiftTS(m *pb.Metadata, startTS uint64, restoreTS uint64) (uint64, bo for _, ds := range m.FileGroups { for _, d := range ds.DataFilesInfo { - if d.Cf == DefaultCF || d.MinBeginTsInDefaultCf == 0 { + if d.Cf == consts.DefaultCF || d.MinBeginTsInDefaultCf == 0 { continue } if d.MinTs > restoreTS || d.MaxTs < startTS { diff --git a/br/pkg/stream/stream_metas_test.go b/br/pkg/stream/stream_metas_test.go index 4d6be57ebd3a3..f11f5cf2ea043 100644 --- a/br/pkg/stream/stream_metas_test.go +++ b/br/pkg/stream/stream_metas_test.go @@ -23,6 +23,7 @@ import ( backuppb "github.com/pingcap/kvproto/pkg/brpb" "github.com/pingcap/log" "github.com/pingcap/tidb/br/pkg/storage" + . "github.com/pingcap/tidb/br/pkg/utils/consts" "github.com/pingcap/tidb/pkg/util/intest" "github.com/stretchr/testify/require" "go.uber.org/multierr" diff --git a/br/pkg/stream/stream_mgr.go b/br/pkg/stream/stream_mgr.go index a9dffb23f017d..1d1add48fbdaa 100644 --- a/br/pkg/stream/stream_mgr.go +++ b/br/pkg/stream/stream_mgr.go @@ -356,6 +356,15 @@ func (*MetadataHelper) Marshal(meta *backuppb.Metadata) ([]byte, error) { return meta.Marshal() } +func (m *MetadataHelper) Close() { + if m.decoder != nil { + m.decoder.Close() + } + if m.encryptionManager != nil { + m.encryptionManager.Close() + } +} + // FastUnmarshalMetaData used a 128 worker pool to speed up // read metadata content from external_storage. func FastUnmarshalMetaData( diff --git a/br/pkg/stream/stream_status.go b/br/pkg/stream/stream_status.go index f22487d2f19ef..2909b8ded8c49 100644 --- a/br/pkg/stream/stream_status.go +++ b/br/pkg/stream/stream_status.go @@ -24,6 +24,7 @@ import ( "github.com/pingcap/tidb/br/pkg/logutil" "github.com/pingcap/tidb/br/pkg/storage" . "github.com/pingcap/tidb/br/pkg/streamhelper" + "github.com/pingcap/tidb/br/pkg/utils" "github.com/tikv/client-go/v2/oracle" pd "github.com/tikv/pd/client" "go.uber.org/zap" @@ -130,9 +131,9 @@ func (p *printByTable) AddTask(task TaskStatus) { table := p.console.CreateTable() table.Add("name", task.Info.Name) table.Add("status", task.colorfulStatusString()) - table.Add("start", fmt.Sprint(FormatDate(oracle.GetTimeFromTS(task.Info.StartTs)))) + table.Add("start", fmt.Sprint(utils.FormatDate(oracle.GetTimeFromTS(task.Info.StartTs)))) if task.Info.EndTs > 0 { - table.Add("end", fmt.Sprint(FormatDate(oracle.GetTimeFromTS(task.Info.EndTs)))) + table.Add("end", fmt.Sprint(utils.FormatDate(oracle.GetTimeFromTS(task.Info.EndTs)))) } s := storage.FormatBackendURL(task.Info.GetStorage()) table.Add("storage", s.String()) @@ -146,7 +147,7 @@ func (p *printByTable) AddTask(task TaskStatus) { if gap > 10*time.Minute { gapColor = color.New(color.FgRed) } - info := fmt.Sprintf("%s; gap=%s", FormatDate(pTime), gapColor.Sprint(gap)) + info := fmt.Sprintf("%s; gap=%s", utils.FormatDate(pTime), gapColor.Sprint(gap)) return info } table.Add("checkpoint[global]", formatTS(task.globalCheckpoint)) diff --git a/br/pkg/stream/table_history.go b/br/pkg/stream/table_history.go new file mode 100644 index 0000000000000..2449c3d3a91e1 --- /dev/null +++ b/br/pkg/stream/table_history.go @@ -0,0 +1,93 @@ +// Copyright 2022-present 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 stream + +const defaultParentTableID = 0 + +// TableLocationInfo stores the table name, db id, and parent table id if is a partition +type TableLocationInfo struct { + DbID int64 + TableName string + ParentTableID int64 // tracking parent table if is a partition +} + +func (t *TableLocationInfo) IsPartition() bool { + return t.ParentTableID != defaultParentTableID +} + +type LogBackupTableHistoryManager struct { + // maps table/partition ID to [original, current] location info + tableNameHistory map[int64][2]TableLocationInfo + dbIdToName map[int64]string +} + +func NewTableHistoryManager() *LogBackupTableHistoryManager { + return &LogBackupTableHistoryManager{ + tableNameHistory: make(map[int64][2]TableLocationInfo), + dbIdToName: make(map[int64]string), + } +} + +// AddTableHistory adds or updates history for a regular table +func (info *LogBackupTableHistoryManager) AddTableHistory(tableId int64, tableName string, dbID int64) { + locationInfo := TableLocationInfo{ + DbID: dbID, + TableName: tableName, + ParentTableID: defaultParentTableID, + } + info.addHistory(tableId, locationInfo) +} + +// AddPartitionHistory adds or updates history for a partition +func (info *LogBackupTableHistoryManager) AddPartitionHistory( + partitionId int64, dbID int64, parentTableID int64, parentTableName string) { + locationInfo := TableLocationInfo{ + DbID: dbID, + TableName: parentTableName, + ParentTableID: parentTableID, + } + info.addHistory(partitionId, locationInfo) +} + +// addHistory is a helper method to maintain the history +func (info *LogBackupTableHistoryManager) addHistory(id int64, locationInfo TableLocationInfo) { + existing, exists := info.tableNameHistory[id] + if !exists { + // first occurrence - store as both original and current + info.tableNameHistory[id] = [2]TableLocationInfo{locationInfo, locationInfo} + } else { + // update current while preserving original + info.tableNameHistory[id] = [2]TableLocationInfo{existing[0], locationInfo} + } +} + +func (info *LogBackupTableHistoryManager) RecordDBIdToName(dbId int64, dbName string) { + info.dbIdToName[dbId] = dbName +} + +// GetTableHistory returns information about all tables that have been renamed. +// Returns a map of table IDs to their original and current locations +func (info *LogBackupTableHistoryManager) GetTableHistory() map[int64][2]TableLocationInfo { + return info.tableNameHistory +} + +func (info *LogBackupTableHistoryManager) GetDBNameByID(dbId int64) (string, bool) { + name, ok := info.dbIdToName[dbId] + return name, ok +} + +func (info *LogBackupTableHistoryManager) GetNewlyCreatedDBHistory() map[int64]string { + return info.dbIdToName +} diff --git a/br/pkg/stream/table_mapping.go b/br/pkg/stream/table_mapping.go index 29773e556c980..365eb55daa862 100644 --- a/br/pkg/stream/table_mapping.go +++ b/br/pkg/stream/table_mapping.go @@ -17,52 +17,73 @@ package stream import ( "context" "encoding/json" + "fmt" + "sort" "github.com/pingcap/errors" backuppb "github.com/pingcap/kvproto/pkg/brpb" + berrors "github.com/pingcap/tidb/br/pkg/errors" + "github.com/pingcap/tidb/br/pkg/utils" + "github.com/pingcap/tidb/br/pkg/utils/consts" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta" "github.com/pingcap/tidb/pkg/meta/model" ) -// TableMappingManager iterates on log backup meta kvs and generate new id for DB, table and partition for +const InitialTempId int64 = 0 + +// TableMappingManager processes each log backup meta kv and generate new id for DB, table and partition for // downstream cluster. It maintains the id mapping and passes down later to the rewrite logic. +// +// The usage in the code base is listed below +// 1. during PiTR, it runs before snapshot restore to collect table id mapping information. For each id to map it +// generates a dummy downstream id first, this is because we can only generate global id after running snapshot restore +// 2. at log restore phase, it merges the db replace map generated from the full backup or previous PiTR task, it will +// replace some dummy id at this step. +// 3. it runs a filter to filter out tables that we don't need +// 4. after all above steps, it uses the genGenGlobalIDs method to generate a batch of ids in one call and replace +// the dummy ids, it builds the final state of the db replace map type TableMappingManager struct { - DbReplaceMap map[UpstreamID]*DBReplace - globalIdMap map[UpstreamID]DownstreamID - genGlobalIdFn func(ctx context.Context) (int64, error) + DBReplaceMap map[UpstreamID]*DBReplace + + // used during scanning log to identify already seen id mapping. For example after exchange partition, the + // exchanged-in table already had an id mapping can be identified in the partition so don't allocate a new id. + globalIdMap map[UpstreamID]DownstreamID + + // a counter for temporary IDs, need to get real global id + // once full restore completes + tempIDCounter DownstreamID } -func NewTableMappingManager( - dbReplaceMap map[UpstreamID]*DBReplace, - genGlobalIdFn func(ctx context.Context) (int64, error)) *TableMappingManager { - if dbReplaceMap == nil { - dbReplaceMap = make(map[UpstreamID]*DBReplace) +func NewTableMappingManager() *TableMappingManager { + return &TableMappingManager{ + DBReplaceMap: make(map[UpstreamID]*DBReplace), + globalIdMap: make(map[UpstreamID]DownstreamID), + tempIDCounter: InitialTempId, } +} - globalTableIdMap := make(map[UpstreamID]DownstreamID) - for _, dr := range dbReplaceMap { - for tblID, tr := range dr.TableMap { - globalTableIdMap[tblID] = tr.TableID - for oldpID, newpID := range tr.PartitionMap { - globalTableIdMap[oldpID] = newpID - } - } +func (tm *TableMappingManager) FromDBReplaceMap(dbReplaceMap map[UpstreamID]*DBReplace) error { + if !tm.IsEmpty() { + return errors.Annotate(berrors.ErrRestoreInvalidRewrite, + "expect table mapping manager empty when need to load ID map") } - return &TableMappingManager{ - DbReplaceMap: dbReplaceMap, - globalIdMap: globalTableIdMap, - genGlobalIdFn: genGlobalIdFn, + if dbReplaceMap == nil { + dbReplaceMap = make(map[UpstreamID]*DBReplace) } + + // doesn't even need to build globalIdMap since loading DBReplaceMap from saved checkpoint + tm.DBReplaceMap = dbReplaceMap + return nil } // ParseMetaKvAndUpdateIdMapping collect table information // the keys and values that are selected to parse here follows the implementation in rewrite_meta_rawkv. Maybe // parsing a subset of these keys/values would suffice, but to make it safe we decide to parse exactly same as // in rewrite_meta_rawkv. -func (tc *TableMappingManager) ParseMetaKvAndUpdateIdMapping(e *kv.Entry, cf string) error { - if !IsMetaDBKey(e.Key) { +func (tm *TableMappingManager) ParseMetaKvAndUpdateIdMapping(e *kv.Entry, cf string) error { + if !utils.IsMetaDBKey(e.Key) { return nil } @@ -73,18 +94,18 @@ func (tc *TableMappingManager) ParseMetaKvAndUpdateIdMapping(e *kv.Entry, cf str if meta.IsDBkey(rawKey.Field) { // parse db key - err := tc.parseDBKeyAndUpdateIdMapping(rawKey.Field) + err := tm.parseDBKeyAndUpdateIdMapping(rawKey.Field) if err != nil { return errors.Trace(err) } // parse value and update if exists - value, err := extractValue(e, cf) + value, err := ExtractValue(e, cf) if err != nil { return errors.Trace(err) } if value != nil { - return tc.parseDBValueAndUpdateIdMapping(value) + return tm.parseDBValueAndUpdateIdMapping(value) } } else if !meta.IsDBkey(rawKey.Key) { return nil @@ -97,40 +118,40 @@ func (tc *TableMappingManager) ParseMetaKvAndUpdateIdMapping(e *kv.Entry, cf str } // parse table key and update - err = tc.parseTableIdAndUpdateIdMapping(rawKey.Key, rawKey.Field, meta.ParseTableKey) + err = tm.parseTableIdAndUpdateIdMapping(rawKey.Key, rawKey.Field, meta.ParseTableKey) if err != nil { return errors.Trace(err) } // parse value and update if exists - value, err := extractValue(e, cf) + value, err := ExtractValue(e, cf) if err != nil { return errors.Trace(err) } if value != nil { - return tc.parseTableValueAndUpdateIdMapping(dbID, value) + return tm.parseTableValueAndUpdateIdMapping(dbID, value) } } else if meta.IsAutoIncrementIDKey(rawKey.Field) { // parse auto increment key and update - err = tc.parseTableIdAndUpdateIdMapping(rawKey.Key, rawKey.Field, meta.ParseAutoIncrementIDKey) + err = tm.parseTableIdAndUpdateIdMapping(rawKey.Key, rawKey.Field, meta.ParseAutoIncrementIDKey) if err != nil { return errors.Trace(err) } } else if meta.IsAutoTableIDKey(rawKey.Field) { // parse auto table key and update - err = tc.parseTableIdAndUpdateIdMapping(rawKey.Key, rawKey.Field, meta.ParseAutoTableIDKey) + err = tm.parseTableIdAndUpdateIdMapping(rawKey.Key, rawKey.Field, meta.ParseAutoTableIDKey) if err != nil { return errors.Trace(err) } } else if meta.IsSequenceKey(rawKey.Field) { // parse sequence key and update - err = tc.parseTableIdAndUpdateIdMapping(rawKey.Key, rawKey.Field, meta.ParseSequenceKey) + err = tm.parseTableIdAndUpdateIdMapping(rawKey.Key, rawKey.Field, meta.ParseSequenceKey) if err != nil { return errors.Trace(err) } } else if meta.IsAutoRandomTableIDKey(rawKey.Field) { // parse sequence key and update - err = tc.parseTableIdAndUpdateIdMapping(rawKey.Key, rawKey.Field, meta.ParseAutoRandomTableIDKey) + err = tm.parseTableIdAndUpdateIdMapping(rawKey.Key, rawKey.Field, meta.ParseAutoRandomTableIDKey) if err != nil { return errors.Trace(err) } @@ -139,23 +160,23 @@ func (tc *TableMappingManager) ParseMetaKvAndUpdateIdMapping(e *kv.Entry, cf str return nil } -func (tc *TableMappingManager) parseDBKeyAndUpdateIdMapping(field []byte) error { +func (tm *TableMappingManager) parseDBKeyAndUpdateIdMapping(field []byte) error { dbID, err := meta.ParseDBKey(field) if err != nil { return errors.Trace(err) } - _, err = tc.getOrCreateDBReplace(dbID) + _, err = tm.getOrCreateDBReplace(dbID) return errors.Trace(err) } -func (tc *TableMappingManager) parseDBValueAndUpdateIdMapping(value []byte) error { +func (tm *TableMappingManager) parseDBValueAndUpdateIdMapping(value []byte) error { dbInfo := new(model.DBInfo) if err := json.Unmarshal(value, dbInfo); err != nil { return errors.Trace(err) } - dbReplace, err := tc.getOrCreateDBReplace(dbInfo.ID) + dbReplace, err := tm.getOrCreateDBReplace(dbInfo.ID) if err != nil { return errors.Trace(err) } @@ -164,32 +185,25 @@ func (tc *TableMappingManager) parseDBValueAndUpdateIdMapping(value []byte) erro } // getOrCreateDBReplace gets an existing DBReplace or creates a new one if not found -func (tc *TableMappingManager) getOrCreateDBReplace(dbID int64) (*DBReplace, error) { - dbReplace, exist := tc.DbReplaceMap[dbID] +func (tm *TableMappingManager) getOrCreateDBReplace(dbID int64) (*DBReplace, error) { + dbReplace, exist := tm.DBReplaceMap[dbID] if !exist { - newID, err := tc.genGlobalIdFn(context.Background()) - if err != nil { - return nil, errors.Trace(err) - } - tc.globalIdMap[dbID] = newID + newID := tm.generateTempID() + tm.globalIdMap[dbID] = newID dbReplace = NewDBReplace("", newID) - tc.DbReplaceMap[dbID] = dbReplace + tm.DBReplaceMap[dbID] = dbReplace } return dbReplace, nil } // getOrCreateTableReplace gets an existing TableReplace or creates a new one if not found -func (tc *TableMappingManager) getOrCreateTableReplace(dbReplace *DBReplace, tableID int64) (*TableReplace, error) { +func (tm *TableMappingManager) getOrCreateTableReplace(dbReplace *DBReplace, tableID int64) (*TableReplace, error) { tableReplace, exist := dbReplace.TableMap[tableID] if !exist { - newID, exist := tc.globalIdMap[tableID] + newID, exist := tm.globalIdMap[tableID] if !exist { - var err error - newID, err = tc.genGlobalIdFn(context.Background()) - if err != nil { - return nil, errors.Trace(err) - } - tc.globalIdMap[tableID] = newID + newID = tm.generateTempID() + tm.globalIdMap[tableID] = newID } tableReplace = NewTableReplace("", newID) dbReplace.TableMap[tableID] = tableReplace @@ -197,7 +211,7 @@ func (tc *TableMappingManager) getOrCreateTableReplace(dbReplace *DBReplace, tab return tableReplace, nil } -func (tc *TableMappingManager) parseTableIdAndUpdateIdMapping( +func (tm *TableMappingManager) parseTableIdAndUpdateIdMapping( key []byte, field []byte, parseField func([]byte) (tableID int64, err error)) error { @@ -211,63 +225,233 @@ func (tc *TableMappingManager) parseTableIdAndUpdateIdMapping( return errors.Trace(err) } - dbReplace, err := tc.getOrCreateDBReplace(dbID) + dbReplace, err := tm.getOrCreateDBReplace(dbID) if err != nil { return errors.Trace(err) } - _, err = tc.getOrCreateTableReplace(dbReplace, tableID) + _, err = tm.getOrCreateTableReplace(dbReplace, tableID) if err != nil { return errors.Trace(err) } return nil } -func (tc *TableMappingManager) parseTableValueAndUpdateIdMapping(dbID int64, value []byte) error { +func (tm *TableMappingManager) parseTableValueAndUpdateIdMapping(dbID int64, value []byte) error { var tableInfo model.TableInfo if err := json.Unmarshal(value, &tableInfo); err != nil { return errors.Trace(err) } - dbReplace, err := tc.getOrCreateDBReplace(dbID) + dbReplace, err := tm.getOrCreateDBReplace(dbID) if err != nil { return errors.Trace(err) } - tableReplace, err := tc.getOrCreateTableReplace(dbReplace, tableInfo.ID) + tableReplace, err := tm.getOrCreateTableReplace(dbReplace, tableInfo.ID) if err != nil { return errors.Trace(err) } tableReplace.Name = tableInfo.Name.O // update table ID and partition ID. - tableInfo.ID = tableReplace.TableID partitions := tableInfo.GetPartitionInfo() if partitions != nil { - for i, partition := range partitions.Definitions { - newID, exist := tableReplace.PartitionMap[partition.ID] + for _, partition := range partitions.Definitions { + _, exist := tableReplace.PartitionMap[partition.ID] if !exist { - newID, exist = tc.globalIdMap[partition.ID] + newID, exist := tm.globalIdMap[partition.ID] if !exist { - newID, err = tc.genGlobalIdFn(context.Background()) - if err != nil { - return errors.Trace(err) - } - tc.globalIdMap[partition.ID] = newID + newID = tm.generateTempID() + tm.globalIdMap[partition.ID] = newID } tableReplace.PartitionMap[partition.ID] = newID } - partitions.Definitions[i].ID = newID } } return nil } +func (tm *TableMappingManager) MergeBaseDBReplace(baseMap map[UpstreamID]*DBReplace) { + // first pass: update all global IDs + for upstreamID, baseDBReplace := range baseMap { + tm.globalIdMap[upstreamID] = baseDBReplace.DbID + + for tableUpID, baseTableReplace := range baseDBReplace.TableMap { + tm.globalIdMap[tableUpID] = baseTableReplace.TableID + + for partUpID, basePartDownID := range baseTableReplace.PartitionMap { + tm.globalIdMap[partUpID] = basePartDownID + } + } + } + + // second pass: update the DBReplaceMap structure + // first update all existing entries using the global ID map + for upDBID, existingDBReplace := range tm.DBReplaceMap { + if newID, exists := tm.globalIdMap[upDBID]; exists { + existingDBReplace.DbID = newID + } + + for upTableID, existingTableReplace := range existingDBReplace.TableMap { + if newID, exists := tm.globalIdMap[upTableID]; exists { + existingTableReplace.TableID = newID + } + + for partUpID := range existingTableReplace.PartitionMap { + if newID, exists := tm.globalIdMap[partUpID]; exists { + existingTableReplace.PartitionMap[partUpID] = newID + } + } + } + } + + // then add any new entries from the base map + for upstreamID, baseDBReplace := range baseMap { + if _, exists := tm.DBReplaceMap[upstreamID]; !exists { + tm.DBReplaceMap[upstreamID] = baseDBReplace + } else { + existingDBReplace := tm.DBReplaceMap[upstreamID] + for tableUpID, baseTableReplace := range baseDBReplace.TableMap { + if _, exists := existingDBReplace.TableMap[tableUpID]; !exists { + existingDBReplace.TableMap[tableUpID] = baseTableReplace + } else { + // merge partition mappings for existing tables + existingTableReplace := existingDBReplace.TableMap[tableUpID] + for partUpID, partDownID := range baseTableReplace.PartitionMap { + if _, exists := existingTableReplace.PartitionMap[partUpID]; !exists { + existingTableReplace.PartitionMap[partUpID] = partDownID + } + } + } + } + } + } +} + +func (tm *TableMappingManager) IsEmpty() bool { + return len(tm.DBReplaceMap) == 0 +} + +func (tm *TableMappingManager) ReplaceTemporaryIDs( + ctx context.Context, genGenGlobalIDs func(ctx context.Context, n int) ([]int64, error)) error { + if tm.tempIDCounter == InitialTempId { + // no temporary IDs were allocated + return nil + } + + // find actually used temporary IDs + usedTempIDs := make(map[DownstreamID]UpstreamID) + + // helper function to check and add temporary ID + addTempIDIfNeeded := func(downID DownstreamID, upID UpstreamID) error { + if downID < 0 { + if prevUpID, exists := usedTempIDs[downID]; exists { + // ok if point to the same upstream + if prevUpID == upID { + return nil + } + return errors.Annotate(berrors.ErrRestoreInvalidRewrite, + fmt.Sprintf("found duplicate temporary ID %d, existing upstream ID: %d, new upstream ID: %d", + downID, prevUpID, upID)) + } + usedTempIDs[downID] = upID + } + return nil + } + + // check DBReplaceMap for used temporary IDs + // any value less than 0 is temporary ID + for upDBId, dr := range tm.DBReplaceMap { + if err := addTempIDIfNeeded(dr.DbID, upDBId); err != nil { + return err + } + for upTableID, tr := range dr.TableMap { + if err := addTempIDIfNeeded(tr.TableID, upTableID); err != nil { + return err + } + for upPartID, partID := range tr.PartitionMap { + if err := addTempIDIfNeeded(partID, upPartID); err != nil { + return err + } + } + } + } + + tempIDs := make([]DownstreamID, 0, len(usedTempIDs)) + // convert to sorted slice + for id := range usedTempIDs { + tempIDs = append(tempIDs, id) + } + + // sort to -1, -2, -4, -8 ... etc + sort.Slice(tempIDs, func(i, j int) bool { + return tempIDs[i] > tempIDs[j] + }) + + // early return if no temp id used + if len(tempIDs) == 0 { + tm.tempIDCounter = InitialTempId + return nil + } + + // generate real global IDs only for actually used temporary IDs + newIDs, err := genGenGlobalIDs(ctx, len(tempIDs)) + if err != nil { + return errors.Trace(err) + } + + // create mapping from temp IDs to new IDs + idMapping := make(map[DownstreamID]DownstreamID, len(tempIDs)) + for i, tempID := range tempIDs { + idMapping[tempID] = newIDs[i] + } + + // replace temp id in DBReplaceMap + for _, dr := range tm.DBReplaceMap { + if newID, exists := idMapping[dr.DbID]; exists { + dr.DbID = newID + } + + for _, tr := range dr.TableMap { + if newID, exists := idMapping[tr.TableID]; exists { + tr.TableID = newID + } + + for oldPID, tempPID := range tr.PartitionMap { + if newID, exists := idMapping[tempPID]; exists { + tr.PartitionMap[oldPID] = newID + } + } + } + } + + tm.tempIDCounter = InitialTempId + return nil +} + +func (tm *TableMappingManager) FilterDBReplaceMap(tracker *utils.PiTRIdTracker) { + // iterate through existing DBReplaceMap + for dbID, dbReplace := range tm.DBReplaceMap { + if !tracker.ContainsDB(dbID) { + dbReplace.Filtered = true + continue + } + + // filter tables in this database + for tableID, tableReplace := range dbReplace.TableMap { + if !tracker.ContainsPhysicalId(dbID, tableID) { + tableReplace.Filtered = true + } + } + } +} + // ToProto produces schemas id maps from up-stream to down-stream. -func (tc *TableMappingManager) ToProto() []*backuppb.PitrDBMap { - dbMaps := make([]*backuppb.PitrDBMap, 0, len(tc.DbReplaceMap)) +func (tm *TableMappingManager) ToProto() []*backuppb.PitrDBMap { + dbMaps := make([]*backuppb.PitrDBMap, 0, len(tm.DBReplaceMap)) - for dbID, dr := range tc.DbReplaceMap { + for dbID, dr := range tm.DBReplaceMap { dbm := backuppb.PitrDBMap{ Name: dr.Name, IdMap: &backuppb.IDMap{ @@ -298,7 +482,6 @@ func (tc *TableMappingManager) ToProto() []*backuppb.PitrDBMap { } dbMaps = append(dbMaps, &dbm) } - return dbMaps } @@ -317,15 +500,14 @@ func FromDBMapProto(dbMaps []*backuppb.PitrDBMap) map[UpstreamID]*DBReplace { } } } - return dbReplaces } -func extractValue(e *kv.Entry, cf string) ([]byte, error) { +func ExtractValue(e *kv.Entry, cf string) ([]byte, error) { switch cf { - case DefaultCF: + case consts.DefaultCF: return e.Value, nil - case WriteCF: + case consts.WriteCF: rawWriteCFValue := new(RawWriteCFValue) if err := rawWriteCFValue.ParseFrom(e.Value); err != nil { return nil, errors.Trace(err) @@ -338,3 +520,8 @@ func extractValue(e *kv.Entry, cf string) ([]byte, error) { return nil, errors.Errorf("unsupported column family: %s", cf) } } + +func (tm *TableMappingManager) generateTempID() DownstreamID { + tm.tempIDCounter-- + return tm.tempIDCounter +} diff --git a/br/pkg/stream/table_mapping_test.go b/br/pkg/stream/table_mapping_test.go index 358c3c09ce872..d09040ffed5e0 100644 --- a/br/pkg/stream/table_mapping_test.go +++ b/br/pkg/stream/table_mapping_test.go @@ -17,8 +17,11 @@ package stream import ( "context" "encoding/json" + "errors" "testing" + "github.com/pingcap/tidb/br/pkg/utils" + "github.com/pingcap/tidb/br/pkg/utils/consts" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta" "github.com/pingcap/tidb/pkg/meta/model" @@ -26,15 +29,7 @@ import ( "github.com/stretchr/testify/require" ) -var increaseID int64 = 100 - -func mockGenGlobalID(_ctx context.Context) (int64, error) { - increaseID++ - return increaseID, nil -} - func TestToProto(t *testing.T) { - increaseID = 100 var ( dbName, tblName string = "db1", "t1" oldDBID UpstreamID = 100 @@ -55,9 +50,11 @@ func TestToProto(t *testing.T) { drs[oldDBID] = dr // create schemas replace and test ToProto(). - tc := NewTableMappingManager(drs, mockGenGlobalID) + tm := NewTableMappingManager() + err := tm.FromDBReplaceMap(drs) + require.NoError(t, err) - dbMap := tc.ToProto() + dbMap := tm.ToProto() require.Equal(t, len(dbMap), 1) require.Equal(t, dbMap[0].Name, dbName) require.Equal(t, dbMap[0].IdMap.UpstreamId, oldDBID) @@ -87,6 +84,944 @@ func TestToProto(t *testing.T) { require.Equal(t, drs2, drs) } +func TestMergeBaseDBReplace(t *testing.T) { + tests := []struct { + name string + existing map[UpstreamID]*DBReplace + base map[UpstreamID]*DBReplace + expected map[UpstreamID]*DBReplace + }{ + { + name: "merge into empty existing map", + existing: map[UpstreamID]*DBReplace{}, + base: map[UpstreamID]*DBReplace{ + 1: { + Name: "db1", + DbID: 1000, + TableMap: map[UpstreamID]*TableReplace{ + 10: {TableID: 1010, Name: "table1"}, + }, + }, + }, + expected: map[UpstreamID]*DBReplace{ + 1: { + Name: "db1", + DbID: 1000, + TableMap: map[UpstreamID]*TableReplace{ + 10: {TableID: 1010, Name: "table1"}, + }, + }, + }, + }, + { + name: "merge empty base map", + existing: map[UpstreamID]*DBReplace{ + 1: { + Name: "db1", + DbID: -1, + TableMap: map[UpstreamID]*TableReplace{ + 10: {TableID: -10, Name: "table1"}, + }, + }, + }, + base: map[UpstreamID]*DBReplace{}, + expected: map[UpstreamID]*DBReplace{ + 1: { + Name: "db1", + DbID: -1, + TableMap: map[UpstreamID]*TableReplace{ + 10: {TableID: -10, Name: "table1"}, + }, + }, + }, + }, + { + name: "merge new database with partitions", + existing: map[UpstreamID]*DBReplace{ + 1: { + Name: "db1", + DbID: -1, + TableMap: map[UpstreamID]*TableReplace{ + 10: { + TableID: -10, + Name: "table1", + PartitionMap: map[UpstreamID]DownstreamID{ + 100: -100, + }, + }, + }, + }, + }, + base: map[UpstreamID]*DBReplace{ + 2: { + Name: "db2", + DbID: 2000, + TableMap: map[UpstreamID]*TableReplace{ + 20: { + TableID: 2020, + Name: "table2", + PartitionMap: map[UpstreamID]DownstreamID{ + 200: 2200, + }, + }, + }, + }, + }, + expected: map[UpstreamID]*DBReplace{ + 1: { + Name: "db1", + DbID: -1, + TableMap: map[UpstreamID]*TableReplace{ + 10: { + TableID: -10, + Name: "table1", + PartitionMap: map[UpstreamID]DownstreamID{ + 100: -100, + }, + }, + }, + }, + 2: { + Name: "db2", + DbID: 2000, + TableMap: map[UpstreamID]*TableReplace{ + 20: { + TableID: 2020, + Name: "table2", + PartitionMap: map[UpstreamID]DownstreamID{ + 200: 2200, + }, + }, + }, + }, + }, + }, + { + name: "merge existing database with multiple tables", + existing: map[UpstreamID]*DBReplace{ + 1: { + Name: "db1", + DbID: -1, + TableMap: map[UpstreamID]*TableReplace{ + 10: {TableID: -10, Name: "table1"}, + 11: {TableID: -11, Name: "table2"}, + }, + }, + }, + base: map[UpstreamID]*DBReplace{ + 1: { + Name: "db1", + DbID: 1000, + TableMap: map[UpstreamID]*TableReplace{ + 10: {TableID: 1010, Name: "table1"}, + 11: {TableID: 1011, Name: "table2"}, + }, + }, + }, + expected: map[UpstreamID]*DBReplace{ + 1: { + Name: "db1", + DbID: 1000, + TableMap: map[UpstreamID]*TableReplace{ + 10: {TableID: 1010, Name: "table1"}, + 11: {TableID: 1011, Name: "table2"}, + }, + }, + }, + }, + { + name: "merge with complex partition updates", + existing: map[UpstreamID]*DBReplace{ + 1: { + Name: "db1", + DbID: -1, + TableMap: map[UpstreamID]*TableReplace{ + 10: { + TableID: -10, + Name: "table1", + PartitionMap: map[UpstreamID]DownstreamID{ + 100: -100, + 101: -101, + }, + }, + }, + }, + }, + base: map[UpstreamID]*DBReplace{ + 1: { + Name: "db1", + DbID: 1000, + TableMap: map[UpstreamID]*TableReplace{ + 10: { + TableID: 1010, + Name: "table1", + PartitionMap: map[UpstreamID]DownstreamID{ + 100: 1100, + 101: 1101, + 102: 1102, // new partition + }, + }, + }, + }, + }, + expected: map[UpstreamID]*DBReplace{ + 1: { + Name: "db1", + DbID: 1000, + TableMap: map[UpstreamID]*TableReplace{ + 10: { + TableID: 1010, + Name: "table1", + PartitionMap: map[UpstreamID]DownstreamID{ + 100: 1100, + 101: 1101, + 102: 1102, + }, + }, + }, + }, + }, + }, + { + name: "merge multiple databases with mixed states", + existing: map[UpstreamID]*DBReplace{ + 1: { + Name: "db1", + DbID: -1, + TableMap: map[UpstreamID]*TableReplace{ + 10: {TableID: -10, Name: "table1"}, + }, + }, + 2: { + Name: "db2", + DbID: 2000, + TableMap: map[UpstreamID]*TableReplace{ + 20: {TableID: 2020, Name: "table2"}, + }, + }, + }, + base: map[UpstreamID]*DBReplace{ + 1: { + Name: "db1", + DbID: 1000, + TableMap: map[UpstreamID]*TableReplace{ + 10: {TableID: 1010, Name: "table1"}, + }, + }, + 3: { + Name: "db3", + DbID: 3000, + TableMap: map[UpstreamID]*TableReplace{ + 30: {TableID: 3030, Name: "table3"}, + }, + }, + }, + expected: map[UpstreamID]*DBReplace{ + 1: { + Name: "db1", + DbID: 1000, + TableMap: map[UpstreamID]*TableReplace{ + 10: {TableID: 1010, Name: "table1"}, + }, + }, + 2: { + Name: "db2", + DbID: 2000, + TableMap: map[UpstreamID]*TableReplace{ + 20: {TableID: 2020, Name: "table2"}, + }, + }, + 3: { + Name: "db3", + DbID: 3000, + TableMap: map[UpstreamID]*TableReplace{ + 30: {TableID: 3030, Name: "table3"}, + }, + }, + }, + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + tm := NewTableMappingManager() + tm.DBReplaceMap = tt.existing + tm.MergeBaseDBReplace(tt.base) + require.Equal(t, tt.expected, tm.DBReplaceMap) + + // Additional verification for deep equality of nested structures + for dbID, dbReplace := range tt.expected { + require.Contains(t, tm.DBReplaceMap, dbID) + require.Equal(t, dbReplace.Name, tm.DBReplaceMap[dbID].Name) + require.Equal(t, dbReplace.DbID, tm.DBReplaceMap[dbID].DbID) + + for tblID, tblReplace := range dbReplace.TableMap { + require.Contains(t, tm.DBReplaceMap[dbID].TableMap, tblID) + require.Equal(t, tblReplace.Name, tm.DBReplaceMap[dbID].TableMap[tblID].Name) + require.Equal(t, tblReplace.TableID, tm.DBReplaceMap[dbID].TableMap[tblID].TableID) + require.Equal(t, tblReplace.PartitionMap, tm.DBReplaceMap[dbID].TableMap[tblID].PartitionMap) + } + } + }) + } +} + +func TestFilterDBReplaceMap(t *testing.T) { + tests := []struct { + name string + initial map[UpstreamID]*DBReplace + filter *utils.PiTRIdTracker + expected map[UpstreamID]*DBReplace + }{ + { + name: "empty filter keeps nothing", + initial: map[UpstreamID]*DBReplace{ + 1: { + Name: "db1", + DbID: 1000, + TableMap: map[UpstreamID]*TableReplace{ + 10: {TableID: 1010, Name: "table1"}, + }, + }, + }, + filter: &utils.PiTRIdTracker{ + DBIdToPhysicalId: map[int64]map[int64]struct{}{}, + }, + expected: map[UpstreamID]*DBReplace{}, + }, + { + name: "filter specific database", + initial: map[UpstreamID]*DBReplace{ + 1: { + Name: "db1", + DbID: 1000, + TableMap: map[UpstreamID]*TableReplace{ + 10: {TableID: 1010, Name: "table1"}, + }, + }, + 2: { + Name: "db2", + DbID: 2000, + TableMap: map[UpstreamID]*TableReplace{ + 20: {TableID: 2020, Name: "table2"}, + }, + }, + }, + filter: &utils.PiTRIdTracker{ + DBIdToPhysicalId: map[int64]map[int64]struct{}{ + 1: {10: struct{}{}}, + }, + }, + expected: map[UpstreamID]*DBReplace{ + 1: { + Name: "db1", + DbID: 1000, + TableMap: map[UpstreamID]*TableReplace{ + 10: {TableID: 1010, Name: "table1"}, + }, + }, + }, + }, + { + name: "filter specific tables within database", + initial: map[UpstreamID]*DBReplace{ + 1: { + Name: "db1", + DbID: 1000, + TableMap: map[UpstreamID]*TableReplace{ + 10: {TableID: 1010, Name: "table1"}, + 11: {TableID: 1011, Name: "table2"}, + 12: {TableID: 1012, Name: "table3"}, + }, + }, + }, + filter: &utils.PiTRIdTracker{ + DBIdToPhysicalId: map[int64]map[int64]struct{}{ + 1: { + 10: struct{}{}, + 12: struct{}{}, + }, + }, + }, + expected: map[UpstreamID]*DBReplace{ + 1: { + Name: "db1", + DbID: 1000, + TableMap: map[UpstreamID]*TableReplace{ + 10: {TableID: 1010, Name: "table1"}, + 12: {TableID: 1012, Name: "table3"}, + }, + }, + }, + }, + { + name: "filter tables with partitions", + initial: map[UpstreamID]*DBReplace{ + 1: { + Name: "db1", + DbID: 1000, + TableMap: map[UpstreamID]*TableReplace{ + 10: { + TableID: 1010, + Name: "table1", + PartitionMap: map[UpstreamID]DownstreamID{ + 100: 1100, + 101: 1101, + }, + }, + 11: { + TableID: 1011, + Name: "table2", + PartitionMap: map[UpstreamID]DownstreamID{ + 102: 1102, + 103: 1103, + }, + }, + }, + }, + }, + filter: &utils.PiTRIdTracker{ + DBIdToPhysicalId: map[int64]map[int64]struct{}{ + 1: {10: struct{}{}}, + }, + }, + expected: map[UpstreamID]*DBReplace{ + 1: { + Name: "db1", + DbID: 1000, + TableMap: map[UpstreamID]*TableReplace{ + 10: { + TableID: 1010, + Name: "table1", + PartitionMap: map[UpstreamID]DownstreamID{ + 100: 1100, + 101: 1101, + }, + }, + }, + }, + }, + }, + { + name: "filter with multiple databases and tables", + initial: map[UpstreamID]*DBReplace{ + 1: { + Name: "db1", + DbID: 1000, + TableMap: map[UpstreamID]*TableReplace{ + 10: {TableID: 1010, Name: "table1"}, + 11: {TableID: 1011, Name: "table2"}, + }, + }, + 2: { + Name: "db2", + DbID: 2000, + TableMap: map[UpstreamID]*TableReplace{ + 20: {TableID: 2020, Name: "table3"}, + 21: {TableID: 2021, Name: "table4"}, + }, + }, + 3: { + Name: "db3", + DbID: 3000, + TableMap: map[UpstreamID]*TableReplace{ + 30: {TableID: 3030, Name: "table5"}, + }, + }, + }, + filter: &utils.PiTRIdTracker{ + DBIdToPhysicalId: map[int64]map[int64]struct{}{ + 1: {10: struct{}{}}, + 2: { + 20: struct{}{}, + 21: struct{}{}, + }, + }, + }, + expected: map[UpstreamID]*DBReplace{ + 1: { + Name: "db1", + DbID: 1000, + TableMap: map[UpstreamID]*TableReplace{ + 10: {TableID: 1010, Name: "table1"}, + }, + }, + 2: { + Name: "db2", + DbID: 2000, + TableMap: map[UpstreamID]*TableReplace{ + 20: {TableID: 2020, Name: "table3"}, + 21: {TableID: 2021, Name: "table4"}, + }, + }, + }, + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + tm := NewTableMappingManager() + tm.DBReplaceMap = tt.initial + + tm.FilterDBReplaceMap(tt.filter) + + // verify DBReplaceMap is as expected + require.Equal(t, tt.expected, tm.DBReplaceMap) + }) + } +} + +func TestReplaceTemporaryIDs(t *testing.T) { + tests := []struct { + name string + initial map[UpstreamID]*DBReplace + tempCounter DownstreamID + genGlobalIDs func(context.Context, int) ([]int64, error) + expected map[UpstreamID]*DBReplace + expectedErr error + }{ + { + name: "no temporary IDs", + initial: map[UpstreamID]*DBReplace{ + 1: { + Name: "db1", + DbID: 1000, + TableMap: map[UpstreamID]*TableReplace{ + 10: {TableID: 1010, Name: "table1"}, + }, + }, + }, + tempCounter: InitialTempId, + genGlobalIDs: func(ctx context.Context, n int) ([]int64, error) { + return nil, nil + }, + expected: map[UpstreamID]*DBReplace{ + 1: { + Name: "db1", + DbID: 1000, + TableMap: map[UpstreamID]*TableReplace{ + 10: {TableID: 1010, Name: "table1"}, + }, + }, + }, + expectedErr: nil, + }, + { + name: "replace all temporary IDs", + initial: map[UpstreamID]*DBReplace{ + 1: { + Name: "db1", + DbID: -1, + TableMap: map[UpstreamID]*TableReplace{ + 10: { + TableID: -2, + Name: "table1", + PartitionMap: map[UpstreamID]DownstreamID{ + 100: -3, + }, + }, + }, + }, + }, + tempCounter: -3, + genGlobalIDs: func(ctx context.Context, n int) ([]int64, error) { + return []int64{1000, 1010, 1020}, nil + }, + expected: map[UpstreamID]*DBReplace{ + 1: { + Name: "db1", + DbID: 1000, + TableMap: map[UpstreamID]*TableReplace{ + 10: { + TableID: 1010, + Name: "table1", + PartitionMap: map[UpstreamID]DownstreamID{ + 100: 1020, + }, + }, + }, + }, + }, + expectedErr: nil, + }, + { + name: "mixed temporary and global IDs", + initial: map[UpstreamID]*DBReplace{ + 1: { + Name: "db1", + DbID: 1000, + TableMap: map[UpstreamID]*TableReplace{ + 10: { + TableID: -1, + Name: "table1", + PartitionMap: map[UpstreamID]DownstreamID{ + 100: 1100, + 101: -2, + }, + }, + }, + }, + 2: { + Name: "db2", + DbID: -3, + TableMap: map[UpstreamID]*TableReplace{ + 20: {TableID: 2000, Name: "table2"}, + }, + }, + }, + tempCounter: -3, + genGlobalIDs: func(ctx context.Context, n int) ([]int64, error) { + return []int64{2010, 2020, 2030}, nil + }, + expected: map[UpstreamID]*DBReplace{ + 1: { + Name: "db1", + DbID: 1000, + TableMap: map[UpstreamID]*TableReplace{ + 10: { + TableID: 2010, + Name: "table1", + PartitionMap: map[UpstreamID]DownstreamID{ + 100: 1100, + 101: 2020, + }, + }, + }, + }, + 2: { + Name: "db2", + DbID: 2030, + TableMap: map[UpstreamID]*TableReplace{ + 20: {TableID: 2000, Name: "table2"}, + }, + }, + }, + expectedErr: nil, + }, + { + name: "error generating global IDs", + initial: map[UpstreamID]*DBReplace{ + 1: { + Name: "db1", + DbID: -1, + TableMap: map[UpstreamID]*TableReplace{ + 10: {TableID: -2, Name: "table1"}, + }, + }, + }, + tempCounter: -2, + genGlobalIDs: func(ctx context.Context, n int) ([]int64, error) { + return nil, errors.New("failed to generate global IDs") + }, + expected: map[UpstreamID]*DBReplace{ + 1: { + Name: "db1", + DbID: -1, + TableMap: map[UpstreamID]*TableReplace{ + 10: {TableID: -2, Name: "table1"}, + }, + }, + }, + expectedErr: errors.New("failed to generate global IDs"), + }, + { + name: "complex structure with multiple temporary IDs", + initial: map[UpstreamID]*DBReplace{ + 1: { + Name: "db1", + DbID: -1, + TableMap: map[UpstreamID]*TableReplace{ + 10: { + TableID: -2, + Name: "table1", + PartitionMap: map[UpstreamID]DownstreamID{ + 100: -3, + 101: -4, + }, + }, + 11: { + TableID: -5, + Name: "table2", + PartitionMap: map[UpstreamID]DownstreamID{ + 102: -6, + }, + }, + }, + }, + 2: { + Name: "db2", + DbID: -7, + TableMap: map[UpstreamID]*TableReplace{ + 20: { + TableID: -8, + Name: "table3", + }, + }, + }, + }, + tempCounter: -8, + genGlobalIDs: func(ctx context.Context, n int) ([]int64, error) { + ids := make([]int64, n) + for i := 0; i < n; i++ { + ids[i] = int64(1000 + i*10) + } + return ids, nil + }, + expected: map[UpstreamID]*DBReplace{ + 1: { + Name: "db1", + DbID: 1000, + TableMap: map[UpstreamID]*TableReplace{ + 10: { + TableID: 1010, + Name: "table1", + PartitionMap: map[UpstreamID]DownstreamID{ + 100: 1020, + 101: 1030, + }, + }, + 11: { + TableID: 1040, + Name: "table2", + PartitionMap: map[UpstreamID]DownstreamID{ + 102: 1050, + }, + }, + }, + }, + 2: { + Name: "db2", + DbID: 1060, + TableMap: map[UpstreamID]*TableReplace{ + 20: { + TableID: 1070, + Name: "table3", + }, + }, + }, + }, + expectedErr: nil, + }, + { + name: "non-consecutive temporary IDs", + initial: map[UpstreamID]*DBReplace{ + 1: { + Name: "db1", + DbID: -5, + TableMap: map[UpstreamID]*TableReplace{ + 10: { + TableID: -2, + Name: "table1", + PartitionMap: map[UpstreamID]DownstreamID{ + 100: -8, + 101: -1, + }, + }, + 11: { + TableID: -15, + Name: "table2", + PartitionMap: map[UpstreamID]DownstreamID{ + 102: -3, + }, + }, + }, + }, + }, + tempCounter: -15, + genGlobalIDs: func(ctx context.Context, n int) ([]int64, error) { + ids := make([]int64, n) + for i := 0; i < n; i++ { + ids[i] = int64(2000 + i*10) + } + return ids, nil + }, + expected: map[UpstreamID]*DBReplace{ + 1: { + Name: "db1", + DbID: 2030, + TableMap: map[UpstreamID]*TableReplace{ + 10: { + TableID: 2010, + Name: "table1", + PartitionMap: map[UpstreamID]DownstreamID{ + 100: 2040, + 101: 2000, + }, + }, + 11: { + TableID: 2050, + Name: "table2", + PartitionMap: map[UpstreamID]DownstreamID{ + 102: 2020, + }, + }, + }, + }, + }, + expectedErr: nil, + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + tm := NewTableMappingManager() + tm.DBReplaceMap = tt.initial + tm.tempIDCounter = tt.tempCounter + + err := tm.ReplaceTemporaryIDs(context.Background(), tt.genGlobalIDs) + + if tt.expectedErr != nil { + require.Error(t, err) + require.Contains(t, err.Error(), tt.expectedErr.Error()) + return + } + + require.NoError(t, err) + require.Equal(t, tt.expected, tm.DBReplaceMap) + require.Equal(t, InitialTempId, tm.tempIDCounter) + }) + } +} + +func TestParseMetaKvAndUpdateIdMapping(t *testing.T) { + var ( + dbID int64 = 40 + dbName = "test_db" + tableID int64 = 100 + tableName = "test_table" + pt1ID int64 = 101 + pt2ID int64 = 102 + pt1Name = "pt1" + pt2Name = "pt2" + ts uint64 = 400036290571534337 + ) + + tc := NewTableMappingManager() + + // Test DB key + dbKey := meta.DBkey(dbID) + dbInfo := &model.DBInfo{ + ID: dbID, + Name: ast.NewCIStr(dbName), + } + dbValue, err := json.Marshal(dbInfo) + require.NoError(t, err) + + // Encode DB key in a transaction + txnDBKey := utils.EncodeTxnMetaKey([]byte("DBs"), dbKey, ts) + entry := &kv.Entry{ + Key: txnDBKey, + Value: dbValue, + } + + // Test parsing DB key and value + err = tc.ParseMetaKvAndUpdateIdMapping(entry, consts.DefaultCF) + require.NoError(t, err) + require.Contains(t, tc.DBReplaceMap, dbID) + require.Equal(t, dbName, tc.DBReplaceMap[dbID].Name) + + // Test table key + pi := model.PartitionInfo{ + Enable: true, + Definitions: make([]model.PartitionDefinition, 0), + } + pi.Definitions = append(pi.Definitions, + model.PartitionDefinition{ + ID: pt1ID, + Name: ast.NewCIStr(pt1Name), + }, + model.PartitionDefinition{ + ID: pt2ID, + Name: ast.NewCIStr(pt2Name), + }, + ) + + tableInfo := &model.TableInfo{ + ID: tableID, + Name: ast.NewCIStr(tableName), + Partition: &pi, + } + tableValue, err := json.Marshal(tableInfo) + require.NoError(t, err) + + // Encode table key in a transaction + txnTableKey := utils.EncodeTxnMetaKey(meta.DBkey(dbID), meta.TableKey(tableID), ts) + tableEntry := &kv.Entry{ + Key: txnTableKey, + Value: tableValue, + } + + // Test parsing table key and value + err = tc.ParseMetaKvAndUpdateIdMapping(tableEntry, consts.DefaultCF) + require.NoError(t, err) + require.Contains(t, tc.DBReplaceMap[dbID].TableMap, tableID) + require.Equal(t, tableName, tc.DBReplaceMap[dbID].TableMap[tableID].Name) + + // Verify partition IDs are mapped + require.Contains(t, tc.DBReplaceMap[dbID].TableMap[tableID].PartitionMap, pt1ID) + require.Contains(t, tc.DBReplaceMap[dbID].TableMap[tableID].PartitionMap, pt2ID) + + // Test non-meta key + nonMetaEntry := &kv.Entry{ + Key: []byte("not_a_meta_key"), + Value: []byte("some_value"), + } + err = tc.ParseMetaKvAndUpdateIdMapping(nonMetaEntry, consts.DefaultCF) + require.NoError(t, err) + + // Test auto increment key with different IDs + autoIncrDBID := int64(50) + autoIncrTableID := int64(200) + autoIncrKey := utils.EncodeTxnMetaKey(meta.DBkey(autoIncrDBID), meta.AutoIncrementIDKey(autoIncrTableID), ts) + autoIncrEntry := &kv.Entry{ + Key: autoIncrKey, + Value: []byte("1"), + } + err = tc.ParseMetaKvAndUpdateIdMapping(autoIncrEntry, consts.DefaultCF) + require.NoError(t, err) + require.Contains(t, tc.DBReplaceMap, autoIncrDBID) + require.Contains(t, tc.DBReplaceMap[autoIncrDBID].TableMap, autoIncrTableID) + + // Test auto table ID key with different IDs + autoTableDBID := int64(60) + autoTableTableID := int64(300) + autoTableKey := utils.EncodeTxnMetaKey(meta.DBkey(autoTableDBID), meta.AutoTableIDKey(autoTableTableID), ts) + autoTableEntry := &kv.Entry{ + Key: autoTableKey, + Value: []byte("1"), + } + err = tc.ParseMetaKvAndUpdateIdMapping(autoTableEntry, consts.DefaultCF) + require.NoError(t, err) + require.Contains(t, tc.DBReplaceMap, autoTableDBID) + require.Contains(t, tc.DBReplaceMap[autoTableDBID].TableMap, autoTableTableID) + + // Test sequence key with different IDs + seqDBID := int64(70) + seqTableID := int64(400) + seqKey := utils.EncodeTxnMetaKey(meta.DBkey(seqDBID), meta.SequenceKey(seqTableID), ts) + seqEntry := &kv.Entry{ + Key: seqKey, + Value: []byte("1"), + } + err = tc.ParseMetaKvAndUpdateIdMapping(seqEntry, consts.DefaultCF) + require.NoError(t, err) + require.Contains(t, tc.DBReplaceMap, seqDBID) + require.Contains(t, tc.DBReplaceMap[seqDBID].TableMap, seqTableID) + + // Test auto random table ID key with different IDs + autoRandomDBID := int64(80) + autoRandomTableID := int64(500) + autoRandomKey := utils.EncodeTxnMetaKey(meta.DBkey(autoRandomDBID), meta.AutoRandomTableIDKey(autoRandomTableID), ts) + autoRandomEntry := &kv.Entry{ + Key: autoRandomKey, + Value: []byte("1"), + } + err = tc.ParseMetaKvAndUpdateIdMapping(autoRandomEntry, consts.DefaultCF) + require.NoError(t, err) + require.Contains(t, tc.DBReplaceMap, autoRandomDBID) + require.Contains(t, tc.DBReplaceMap[autoRandomDBID].TableMap, autoRandomTableID) +} + func TestParseMetaKvAndUpdateIdMapping(t *testing.T) { increaseID = 100 var ( diff --git a/br/pkg/stream/util.go b/br/pkg/stream/util.go deleted file mode 100644 index 10215a68df61d..0000000000000 --- a/br/pkg/stream/util.go +++ /dev/null @@ -1,26 +0,0 @@ -// Copyright 2022 PingCAP, Inc. Licensed under Apache-2.0. - -package stream - -import ( - "strings" - "time" -) - -const DATE_FORMAT = "2006-01-02 15:04:05.999999999 -0700" - -func FormatDate(ts time.Time) string { - return ts.Format(DATE_FORMAT) -} - -func IsMetaDBKey(key []byte) bool { - return strings.HasPrefix(string(key), "mDB") -} - -func IsMetaDDLJobHistoryKey(key []byte) bool { - return strings.HasPrefix(string(key), "mDDLJobH") -} - -func MaybeDBOrDDLJobHistoryKey(key []byte) bool { - return strings.HasPrefix(string(key), "mD") -} diff --git a/br/pkg/stream/util_test.go b/br/pkg/stream/util_test.go deleted file mode 100644 index 6dda62a04ad60..0000000000000 --- a/br/pkg/stream/util_test.go +++ /dev/null @@ -1,48 +0,0 @@ -// Copyright 2022 PingCAP, Inc. Licensed under Apache-2.0. - -package stream - -import ( - "testing" - "time" - - "github.com/stretchr/testify/require" - "github.com/tikv/client-go/v2/oracle" -) - -func TestDateFormat(t *testing.T) { - cases := []struct { - ts uint64 - target string - }{ - { - 434604259287760897, - "2022-07-15 19:14:39.534 +0800", - }, - { - 434605479096221697, - "2022-07-15 20:32:12.734 +0800", - }, - { - 434605478903808000, - "2022-07-15 20:32:12 +0800", - }, - } - - timeZone, _ := time.LoadLocation("Asia/Shanghai") - for _, ca := range cases { - date := FormatDate(oracle.GetTimeFromTS(ca.ts).In(timeZone)) - require.Equal(t, ca.target, date) - } -} - -func TestPrefix(t *testing.T) { - require.True(t, IsMetaDBKey([]byte("mDBs"))) - require.False(t, IsMetaDBKey([]byte("mDDL"))) - require.True(t, IsMetaDDLJobHistoryKey([]byte("mDDLJobHistory"))) - require.False(t, IsMetaDDLJobHistoryKey([]byte("mDDL"))) - require.True(t, MaybeDBOrDDLJobHistoryKey([]byte("mDL"))) - require.True(t, MaybeDBOrDDLJobHistoryKey([]byte("mDB:"))) - require.True(t, MaybeDBOrDDLJobHistoryKey([]byte("mDDLHistory"))) - require.False(t, MaybeDBOrDDLJobHistoryKey([]byte("DDL"))) -} diff --git a/br/pkg/task/BUILD.bazel b/br/pkg/task/BUILD.bazel index 625baff340ed4..0b92cb252f2f6 100644 --- a/br/pkg/task/BUILD.bazel +++ b/br/pkg/task/BUILD.bazel @@ -118,7 +118,7 @@ go_test( ], embed = [":task"], flaky = True, - shard_count = 39, + shard_count = 40, deps = [ "//br/pkg/backup", "//br/pkg/config", diff --git a/br/pkg/task/backup.go b/br/pkg/task/backup.go index 1243cd334bf0b..cffc747104647 100644 --- a/br/pkg/task/backup.go +++ b/br/pkg/task/backup.go @@ -585,8 +585,8 @@ func RunBackup(c context.Context, g glue.Glue, cmdName string, cfg *BackupConfig }) } - // nothing to backup - if len(ranges) == 0 { + // check on ranges and schemas and if nothing to back up do early return + if len(ranges) == 0 && (schemas == nil || schemas.Len() == 0) { pdAddress := strings.Join(cfg.PD, ",") log.Warn("Nothing to backup, maybe connected to cluster for restoring", zap.String("PD address", pdAddress)) diff --git a/br/pkg/task/common.go b/br/pkg/task/common.go index 1f613ba9f934c..0e2048a6a47b9 100644 --- a/br/pkg/task/common.go +++ b/br/pkg/task/common.go @@ -252,9 +252,12 @@ type Config struct { // should be removed after TiDB upgrades the BR dependency. Filter filter.MySQLReplicationRules - FilterStr []string `json:"filter-strings" toml:"filter-strings"` - TableFilter filter.Filter `json:"-" toml:"-"` - SwitchModeInterval time.Duration `json:"switch-mode-interval" toml:"switch-mode-interval"` + FilterStr []string `json:"filter-strings" toml:"filter-strings"` + TableFilter filter.Filter `json:"-" toml:"-"` + // PiTRTableTracker generated from TableFilter during snapshot restore, it has all the db id and table id that needs + // to be restored + PiTRTableTracker *utils.PiTRIdTracker `json:"-" toml:"-"` + SwitchModeInterval time.Duration `json:"switch-mode-interval" toml:"switch-mode-interval"` // Schemas is a database name set, to check whether the restore database has been backup Schemas map[string]struct{} // Tables is a table name set, to check whether the restore table has been backup @@ -401,7 +404,7 @@ func DefineTableFlags(command *cobra.Command) { _ = command.MarkFlagRequired(flagTable) } -// DefineFilterFlags defines the --filter and --case-sensitive flags for `full` subcommand. +// DefineFilterFlags defines the --filter and --case-sensitive flags. func DefineFilterFlags(command *cobra.Command, defaultFilter []string, setHidden bool) { flags := command.Flags() flags.StringArrayP(flagFilter, "f", defaultFilter, "select tables to process") @@ -1004,3 +1007,9 @@ func progressFileWriterRoutine(ctx context.Context, progress glue.Progress, tota } } } + +func WriteStringToConsole(g glue.Glue, msg string) error { + b := []byte(msg) + _, err := glue.GetConsole(g).Out().Write(b) + return err +} diff --git a/br/pkg/task/config_test.go b/br/pkg/task/config_test.go index fe632f88d1c7c..4edaca86f47ec 100644 --- a/br/pkg/task/config_test.go +++ b/br/pkg/task/config_test.go @@ -194,7 +194,7 @@ func TestCheckRestoreDBAndTable(t *testing.T) { for _, db := range ca.backupDBs { backupDBs = append(backupDBs, db) } - err := CheckRestoreDBAndTable(backupDBs, cfg) + err := VerifyDBAndTableInBackup(backupDBs, cfg) require.NoError(t, err) } } diff --git a/br/pkg/task/restore.go b/br/pkg/task/restore.go index 2d7e2d806c4b0..3c2a8172fdfb5 100644 --- a/br/pkg/task/restore.go +++ b/br/pkg/task/restore.go @@ -31,6 +31,7 @@ import ( "github.com/pingcap/tidb/br/pkg/restore" snapclient "github.com/pingcap/tidb/br/pkg/restore/snap_client" "github.com/pingcap/tidb/br/pkg/restore/tiflashrec" + "github.com/pingcap/tidb/br/pkg/stream" "github.com/pingcap/tidb/br/pkg/summary" "github.com/pingcap/tidb/br/pkg/utils" "github.com/pingcap/tidb/br/pkg/version" @@ -537,7 +538,6 @@ func (cfg *RestoreConfig) adjustRestoreConfigForStreamRestore() { // another goroutine is used to iterate the backup file cfg.PitrConcurrency += 1 log.Info("set restore kv files concurrency", zap.Int("concurrency", int(cfg.PitrConcurrency))) - cfg.Config.Concurrency = cfg.PitrConcurrency if cfg.ConcurrencyPerStore.Value > 0 { log.Info("set restore compacted sst files concurrency per store", zap.Int("concurrency", int(cfg.ConcurrencyPerStore.Value))) @@ -601,8 +601,8 @@ func CheckNewCollationEnable( return enabled, nil } -// CheckRestoreDBAndTable is used to check whether the restore dbs or tables have been backup -func CheckRestoreDBAndTable(schemas []*metautil.Database, cfg *RestoreConfig) error { +// VerifyDBAndTableInBackup is used to check whether the restore dbs or tables have been backup +func VerifyDBAndTableInBackup(schemas []*metautil.Database, cfg *RestoreConfig) error { if len(cfg.Schemas) == 0 && len(cfg.Tables) == 0 { return nil } @@ -622,6 +622,8 @@ func CheckRestoreDBAndTable(schemas []*metautil.Database, cfg *RestoreConfig) er tablesMap[utils.EncloseDBAndTable(dbName, table.Info.Name.L)] = struct{}{} } } + + // check on if explicit schema/table filter matches restoreSchemas := cfg.Schemas restoreTables := cfg.Tables for schema := range restoreSchemas { @@ -720,7 +722,10 @@ func RunRestore(c context.Context, g glue.Glue, cmdName string, cfg *RestoreConf if err := version.CheckClusterVersion(c, mgr.GetPDClient(), version.CheckVersionForBR); err != nil { return errors.Trace(err) } - restoreError = runSnapshotRestore(c, mgr, g, cmdName, cfg, nil) + snapshotRestoreConfig := SnapshotRestoreConfig{ + RestoreConfig: cfg, + } + restoreError = runSnapshotRestore(c, mgr, g, cmdName, &snapshotRestoreConfig) } if restoreError != nil { return errors.Trace(restoreError) @@ -757,7 +762,13 @@ func RunRestore(c context.Context, g glue.Glue, cmdName string, cfg *RestoreConf return nil } -func runSnapshotRestore(c context.Context, mgr *conn.Mgr, g glue.Glue, cmdName string, cfg *RestoreConfig, checkInfo *PiTRTaskInfo) error { +type SnapshotRestoreConfig struct { + *RestoreConfig + piTRTaskInfo *PiTRTaskInfo + logTableHistoryManager *stream.LogBackupTableHistoryManager +} + +func runSnapshotRestore(c context.Context, mgr *conn.Mgr, g glue.Glue, cmdName string, cfg *SnapshotRestoreConfig) error { cfg.Adjust() defer summary.Summary(cmdName) ctx, cancel := context.WithCancel(c) @@ -770,9 +781,31 @@ func runSnapshotRestore(c context.Context, mgr *conn.Mgr, g glue.Glue, cmdName s ctx = opentracing.ContextWithSpan(ctx, span1) } - codec := mgr.GetStorage().GetCodec() + // reads out information from backup meta file and do requirement checking if needed + u, s, backupMeta, err := ReadBackupMeta(ctx, metautil.MetaFile, &cfg.Config) + if err != nil { + return errors.Trace(err) + } + if cfg.CheckRequirements { + log.Info("Checking incompatible TiCDC changefeeds before restoring.", + logutil.ShortError(err), zap.Uint64("restore-ts", backupMeta.EndVersion)) + if err := checkIncompatibleChangefeed(ctx, backupMeta.EndVersion, mgr.GetDomain().GetEtcdClient()); err != nil { + return errors.Trace(err) + } - // need retrieve these configs from tikv if not set in command. + backupVersion := version.NormalizeBackupVersion(backupMeta.ClusterVersion) + if backupVersion != nil { + if versionErr := version.CheckClusterVersion(ctx, mgr.GetPDClient(), version.CheckVersionForBackup(backupVersion)); versionErr != nil { + return errors.Trace(versionErr) + } + } + } + if _, err = CheckNewCollationEnable(backupMeta.GetNewCollationsEnabled(), g, mgr.GetStorage(), cfg.CheckRequirements); err != nil { + return errors.Trace(err) + } + + // build restore client + // need to retrieve these configs from tikv if not set in command. kvConfigs := &pconfig.KVConfig{ ImportGoroutines: cfg.ConcurrencyPerStore, MergeRegionSize: cfg.MergeSmallRegionSizeBytes, @@ -791,43 +824,19 @@ func runSnapshotRestore(c context.Context, mgr *conn.Mgr, g glue.Glue, cmdName s cfg.ConcurrencyPerStore = kvConfigs.ImportGoroutines // using tikv config to set the concurrency-per-store for client. client.SetConcurrencyPerStore(cfg.ConcurrencyPerStore.Value) - err := configureRestoreClient(ctx, client, cfg) - if err != nil { - return errors.Trace(err) - } - // Init DB connection sessions - err = client.Init(g, mgr.GetStorage()) - defer client.Close() + err = configureRestoreClient(ctx, client, cfg.RestoreConfig) if err != nil { return errors.Trace(err) } - - u, s, backupMeta, err := ReadBackupMeta(ctx, metautil.MetaFile, &cfg.Config) + // InitConnections DB connection sessions + err = client.InitConnections(g, mgr.GetStorage()) if err != nil { return errors.Trace(err) } + defer client.Close() - if cfg.CheckRequirements { - err := checkIncompatibleChangefeed(ctx, backupMeta.EndVersion, mgr.GetDomain().GetEtcdClient()) - log.Info("Checking incompatible TiCDC changefeeds before restoring.", - logutil.ShortError(err), zap.Uint64("restore-ts", backupMeta.EndVersion)) - if err != nil { - return errors.Trace(err) - } - } - - backupVersion := version.NormalizeBackupVersion(backupMeta.ClusterVersion) - if cfg.CheckRequirements && backupVersion != nil { - if versionErr := version.CheckClusterVersion(ctx, mgr.GetPDClient(), version.CheckVersionForBackup(backupVersion)); versionErr != nil { - return errors.Trace(versionErr) - } - } - if _, err = CheckNewCollationEnable(backupMeta.GetNewCollationsEnabled(), g, mgr.GetStorage(), cfg.CheckRequirements); err != nil { - return errors.Trace(err) - } - - reader := metautil.NewMetaReader(backupMeta, s, &cfg.CipherInfo) - if err = client.LoadSchemaIfNeededAndInitClient(c, backupMeta, u, reader, cfg.LoadStats, nil, nil); err != nil { + metaReader := metautil.NewMetaReader(backupMeta, s, &cfg.CipherInfo) + if err = client.LoadSchemaIfNeededAndInitClient(ctx, backupMeta, u, metaReader, cfg.LoadStats, nil, nil); err != nil { return errors.Trace(err) } @@ -846,15 +855,44 @@ func runSnapshotRestore(c context.Context, mgr *conn.Mgr, g glue.Glue, cmdName s existsCheckpointMetadata := checkpoint.ExistsSstRestoreCheckpoint(ctx, mgr.GetDomain(), checkpoint.SnapshotRestoreCheckpointDatabaseName) checkpointFirstRun = !existsCheckpointMetadata } - if err = CheckRestoreDBAndTable(client.GetDatabases(), cfg); err != nil { + if err = VerifyDBAndTableInBackup(client.GetDatabases(), cfg.RestoreConfig); err != nil { return err } - files, tables, dbs := filterRestoreFiles(client, cfg) - if len(dbs) == 0 && len(tables) != 0 { - return errors.Annotate(berrors.ErrRestoreInvalidBackup, "contain tables but no databases") + + // filters out db/table/files using filter + fileMap, tableMap, dbMap, err := filterRestoreFiles(client, cfg.RestoreConfig) + if err != nil { + return errors.Trace(err) } + log.Info("found items to restore after filtering", + zap.Int("files", len(fileMap)), + zap.Int("tables", len(tableMap)), + zap.Int("db", len(dbMap))) + + // only run when this full restore is part of the PiTR + if cfg.logTableHistoryManager != nil { + // adjust tables to restore in the snapshot restore phase since it will later be renamed during + // log restore and will fall into or out of the filter range. + err := AdjustTablesToRestoreAndCreateTableTracker(cfg.logTableHistoryManager, cfg.RestoreConfig, + client.GetDatabaseMap(), fileMap, tableMap, dbMap) + if err != nil { + return errors.Trace(err) + } + + log.Info("adjusted items to restore", + zap.Int("files", len(fileMap)), + zap.Int("tables", len(tableMap)), + zap.Int("db", len(dbMap))) + + // need to update to include all eligible table id from snapshot restore + UpdatePiTRTableTracker(cfg.RestoreConfig, tableMap) + } + files := utils.FlattenValues(fileMap) + tables := utils.Values(tableMap) + dbs := utils.Values(dbMap) if cfg.CheckRequirements && checkpointFirstRun { + // after figuring out what files to restore, check if disk has enough space if err := checkDiskSpace(ctx, mgr, files, tables); err != nil { return errors.Trace(err) } @@ -870,28 +908,29 @@ func runSnapshotRestore(c context.Context, mgr *conn.Mgr, g glue.Glue, cmdName s } // for full + log restore. should check the cluster is empty. - if client.IsFull() && checkInfo != nil && checkInfo.FullRestoreCheckErr != nil { - return checkInfo.FullRestoreCheckErr + if client.IsFull() && cfg.piTRTaskInfo != nil && cfg.piTRTaskInfo.FullRestoreCheckErr != nil { + return cfg.piTRTaskInfo.FullRestoreCheckErr } importModeSwitcher := restore.NewImportModeSwitcher(mgr.GetPDClient(), cfg.Config.SwitchModeInterval, mgr.GetTLSConfig()) - restoreSchedulers, schedulersConfig, err := restore.RestorePreWork(ctx, mgr, importModeSwitcher, cfg.Online, true) + restoreSchedulersFunc, schedulersConfig, err := restore.RestorePreWork(ctx, mgr, importModeSwitcher, cfg.Online, true) if err != nil { return errors.Trace(err) } - schedulersRemovable := false + // need to know whether restore has been completed so can restore schedulers + canRestoreSchedulers := false defer func() { // don't reset pd scheduler if checkpoint mode is used and restored is not finished - if cfg.UseCheckpoint && !schedulersRemovable { - log.Info("skip removing pd schehduler for next retry") + if cfg.UseCheckpoint && !canRestoreSchedulers { + log.Info("skip removing pd scheduler for next retry") return } - log.Info("start to remove the pd scheduler") + log.Info("start to restore pd scheduler") // run the post-work to avoid being stuck in the import // mode or emptied schedulers. - restore.RestorePostWork(ctx, importModeSwitcher, restoreSchedulers, cfg.Online) - log.Info("finish removing pd scheduler") + restore.RestorePostWork(ctx, importModeSwitcher, restoreSchedulersFunc, cfg.Online) + log.Info("finish restoring pd scheduler") }() if isFullRestore(cmdName) { @@ -909,7 +948,7 @@ func runSnapshotRestore(c context.Context, mgr *conn.Mgr, g glue.Glue, cmdName s } } else if client.IsFull() && checkpointFirstRun && cfg.CheckRequirements { if err := checkTableExistence(ctx, mgr, tables, g); err != nil { - schedulersRemovable = true + canRestoreSchedulers = true return errors.Trace(err) } } @@ -921,8 +960,7 @@ func runSnapshotRestore(c context.Context, mgr *conn.Mgr, g glue.Glue, cmdName s } // preallocate the table id, because any ddl job or database creation(include checkpoint) also allocates the global ID - err = client.AllocTableIDs(ctx, tables) - if err != nil { + if err = client.AllocTableIDs(ctx, tables); err != nil { return errors.Trace(err) } @@ -934,7 +972,7 @@ func runSnapshotRestore(c context.Context, mgr *conn.Mgr, g glue.Glue, cmdName s return errors.Trace(err) } if restoreSchedulersConfigFromCheckpoint != nil { - restoreSchedulers = mgr.MakeUndoFunctionByConfig(*restoreSchedulersConfigFromCheckpoint) + restoreSchedulersFunc = mgr.MakeUndoFunctionByConfig(*restoreSchedulersConfigFromCheckpoint) } checkpointSetWithTableID = sets @@ -942,7 +980,7 @@ func runSnapshotRestore(c context.Context, mgr *conn.Mgr, g glue.Glue, cmdName s // need to flush the whole checkpoint data so that br can quickly jump to // the log kv restore step when the next retry. log.Info("wait for flush checkpoint...") - client.WaitForFinishCheckpoint(ctx, len(cfg.FullBackupStorage) > 0 || !schedulersRemovable) + client.WaitForFinishCheckpoint(ctx, len(cfg.FullBackupStorage) > 0 || !canRestoreSchedulers) }() } @@ -1025,8 +1063,7 @@ func runSnapshotRestore(c context.Context, mgr *conn.Mgr, g glue.Glue, cmdName s } // execute DDL first - err = client.ExecDDLs(ctx, ddlJobs) - if err != nil { + if err = client.ExecDDLs(ctx, ddlJobs); err != nil { return errors.Trace(err) } @@ -1034,8 +1071,6 @@ func runSnapshotRestore(c context.Context, mgr *conn.Mgr, g glue.Glue, cmdName s if len(dbs) == 0 && len(tables) == 0 { log.Info("nothing to restore, all databases and tables are filtered out") // even nothing to restore, we show a success message since there is no failure. - summary.SetSuccessStatus(true) - return nil } if err = client.CreateDatabases(ctx, dbs); err != nil { @@ -1061,6 +1096,7 @@ func runSnapshotRestore(c context.Context, mgr *conn.Mgr, g glue.Glue, cmdName s return errors.Trace(err) } + codec := mgr.GetStorage().GetCodec() if len(files) == 0 { log.Info("no files, empty databases and tables are restored") summary.SetSuccessStatus(true) @@ -1149,6 +1185,8 @@ func runSnapshotRestore(c context.Context, mgr *conn.Mgr, g glue.Glue, cmdName s // The cost of rename user table / replace into system table wouldn't be so high. // So leave it out of the pipeline for easier implementation. + log.Info("restoring system schemas", zap.Bool("withSys", cfg.WithSysTable), + zap.Strings("filter", cfg.FilterStr)) err = client.RestoreSystemSchemas(ctx, cfg.TableFilter) if err != nil { return errors.Trace(err) @@ -1159,7 +1197,7 @@ func runSnapshotRestore(c context.Context, mgr *conn.Mgr, g glue.Glue, cmdName s return err } - schedulersRemovable = true + canRestoreSchedulers = true // Set task summary to success status. summary.SetSuccessStatus(true) @@ -1321,35 +1359,201 @@ func checkTableExistence(ctx context.Context, mgr *conn.Mgr, tables []*metautil. return nil } -// filterRestoreFiles filters tables that can't be processed after applying cfg.TableFilter.MatchTable. -// if the db has no table that can be processed, the db will be filtered too. +// filterRestoreFiles filters out dbs and tables. func filterRestoreFiles( client *snapclient.SnapClient, cfg *RestoreConfig, -) (files []*backuppb.File, tables []*metautil.Table, dbs []*metautil.Database) { +) (fileMap map[int64][]*backuppb.File, tableMap map[int64]*metautil.Table, dbMap map[int64]*metautil.Database, err error) { + fileMap = make(map[int64][]*backuppb.File) + tableMap = make(map[int64]*metautil.Table) + dbMap = make(map[int64]*metautil.Database) + for _, db := range client.GetDatabases() { dbName := db.Info.Name.O - if name, ok := utils.GetSysDBName(db.Info.Name); utils.IsSysDB(name) && ok { - dbName = name - } if checkpoint.IsCheckpointDB(db.Info.Name) { continue } - if !cfg.TableFilter.MatchSchema(dbName) { + if !utils.MatchSchema(cfg.TableFilter, dbName, cfg.WithSysTable) { continue } - dbs = append(dbs, db) + dbMap[db.Info.ID] = db for _, table := range db.Tables { - if table.Info == nil || !cfg.TableFilter.MatchTable(dbName, table.Info.Name.O) { + if table.Info == nil || !utils.MatchTable(cfg.TableFilter, dbName, table.Info.Name.O, cfg.WithSysTable) { + continue + } + + // Add table to tableMap using table ID as key + tableMap[table.Info.ID] = table + + // Add files to fileMap using table ID as key + fileMap[table.Info.ID] = table.Files + } + } + + // sanity check + if len(dbMap) == 0 && len(tableMap) != 0 { + err = errors.Annotate(berrors.ErrRestoreInvalidBackup, "contains tables but no databases") + } + return +} + +// getDBNameFromIDInBackup gets database name from either snapshot or log backup history +func getDBNameFromIDInBackup( + dbID int64, + snapshotDBMap map[int64]*metautil.Database, + logBackupTableHistory *stream.LogBackupTableHistoryManager, +) (dbName string, exists bool) { + // check in snapshot + if snapDb, exists := snapshotDBMap[dbID]; exists { + return snapDb.Info.Name.O, true + } + // check during log backup + if name, exists := logBackupTableHistory.GetDBNameByID(dbID); exists { + return name, true + } + log.Warn("did not find db id in full/log backup, "+ + "likely different filters are specified for full/log backup and restore, ignoring this db", + zap.Any("dbId", dbID)) + return "", false +} + +// getTableIDToTrack determines which ID should be tracked in the PiTR table tracker +// For regular tables, it returns the physical id itself +// For partitioned tables, it returns the parent table ID +func getTableIDToTrack(uniqueID int64, end *stream.TableLocationInfo) int64 { + if end.IsPartition() { + return end.ParentTableID // return parent table ID for partitions + } + return uniqueID // return uniqueID for regular tables +} + +func AdjustTablesToRestoreAndCreateTableTracker( + logBackupTableHistory *stream.LogBackupTableHistoryManager, + cfg *RestoreConfig, + snapshotDBMap map[int64]*metautil.Database, + fileMap map[int64][]*backuppb.File, + tableMap map[int64]*metautil.Table, + dbMap map[int64]*metautil.Database, +) (err error) { + // build tracker for pitr restore to use later + piTRTableTracker := utils.NewPiTRTableTracker() + + // put all the newly created db that matches the filter during log backup into the pitr table tracker + newlyCreatedDBs := logBackupTableHistory.GetNewlyCreatedDBHistory() + for dbId, dbName := range newlyCreatedDBs { + if utils.MatchSchema(cfg.TableFilter, dbName, cfg.WithSysTable) { + piTRTableTracker.AddDB(dbId) + } + } + + // get all the tables seen during the log backup + tableHistory := logBackupTableHistory.GetTableHistory() + + // physical id can be table id or partition id + for physicalID, dbIDAndTableName := range tableHistory { + start := dbIDAndTableName[0] + end := dbIDAndTableName[1] + log.Info("#### table history", zap.Any("start", start.TableName), + zap.Any("start parent id", start.ParentTableID), zap.Any("end", end.TableName), + zap.Any("end parent id", end.ParentTableID)) + // get end state DB name + endDBName, exists := getDBNameFromIDInBackup(end.DbID, snapshotDBMap, logBackupTableHistory) + if !exists { + continue + } + + startTableID := getTableIDToTrack(physicalID, &start) + endTableID := getTableIDToTrack(physicalID, &end) + // handle end state in filter range cases + // 1. original == current, didn't have renaming + // 2. original has been renamed/exchanged and current is in the filter range + // we need to restore original table + if utils.MatchTable(cfg.TableFilter, endDBName, end.TableName, cfg.WithSysTable) { + // put this db/table id into pitr tracker as it matches with user's filter + // have to update tracker here since table might be empty or not in snapshot so nothing will be returned . + // but we still need to capture this table id to restore during log restore. + piTRTableTracker.AddPhysicalId(end.DbID, endTableID) + log.Info("####### putting into tracker", zap.Any("end table", endTableID), + zap.Any("start table", startTableID)) + + // if start table id is already tracked, continue + if _, exists = tableMap[startTableID]; exists { continue } - files = append(files, table.Files...) - tables = append(tables, table) + + // check if snapshot contains the start db/table + startDB, exists := snapshotDBMap[start.DbID] + if !exists { + // original db created during log backup, or full backup has a filter that filters out this db, + // either way snapshot doesn't have information about this db so doesn't need to restore at snapshot + continue + } + log.Info("####### adding extra", zap.Any("end table", endTableID), + zap.Any("start table", startTableID)) + + // need to restore the matching table in snapshot restore phase + // note: assuming below op is less likely to happen, if it happens a lot we can build a map + // to optimize + for _, startTable := range startDB.Tables { + if startTable.Info == nil { + continue + } + if startTable.Info.ID == startTableID { + // tracks this start table + piTRTableTracker.AddPhysicalId(start.DbID, startTableID) + if startTable.Info.Partition != nil { + for _, partInfo := range startTable.Info.Partition.Definitions { + piTRTableTracker.AddPhysicalId(start.DbID, partInfo.ID) + } + } + // clean up if needed after restore finishes + piTRTableTracker.AddTableToCleanup(start.DbID, startTableID) + fileMap[startTable.Info.ID] = startTable.Files + dbMap[start.DbID] = startDB + tableMap[startTable.Info.ID] = startTable + // only one table id will match + break + } + } + } else { + startDbName, exists := getDBNameFromIDInBackup(start.DbID, snapshotDBMap, logBackupTableHistory) + if !exists { + continue + } + // check if start state is in filter range + if utils.MatchTable(cfg.TableFilter, startDbName, start.TableName, cfg.WithSysTable) { + // if start state is in the tracker meaning it was added on purpose in previous step, don't remove it + // if start is partition, don't remove the table as rest of the table is still needed + if piTRTableTracker.ContainsPhysicalId(start.DbID, startTableID) || start.IsPartition() { + // clean up later after restore + piTRTableTracker.AddTableToCleanup(end.DbID, endTableID) + continue + } + + // if is a table, remove it now since will be moved out later + // for correctness, don't remove db even if all tables are filtered out + delete(fileMap, startTableID) + delete(tableMap, startTableID) + } } } + // store the tracker into config + cfg.PiTRTableTracker = piTRTableTracker return } +func UpdatePiTRTableTracker(cfg *RestoreConfig, tableMap map[int64]*metautil.Table) { + for _, table := range tableMap { + cfg.PiTRTableTracker.AddPhysicalId(table.DB.ID, table.Info.ID) + if table.Info.Partition != nil { + for _, partInfo := range table.Info.Partition.Definitions { + cfg.PiTRTableTracker.AddPhysicalId(table.DB.ID, partInfo.ID) + } + } + } + log.Info("pitr table tracker", zap.String("map", cfg.PiTRTableTracker.String())) +} + // tweakLocalConfForRestore tweaks some of configs of TiDB to make the restore progress go well. // return a function that could restore the config to origin. func tweakLocalConfForRestore() func() { @@ -1582,3 +1786,21 @@ func checkIsInActions(action model.ActionType, actions map[model.ActionType]stru _, ok := actions[action] return ok } + +func afterTableRestoredCh(ctx context.Context, createdTables []*snapclient.CreatedTable) <-chan *snapclient.CreatedTable { + outCh := make(chan *snapclient.CreatedTable) + + go func() { + defer close(outCh) + + for _, createdTable := range createdTables { + select { + case <-ctx.Done(): + return + default: + outCh <- createdTable + } + } + }() + return outCh +} diff --git a/br/pkg/task/restore_raw.go b/br/pkg/task/restore_raw.go index 1680e60472f7b..f38040d569b70 100644 --- a/br/pkg/task/restore_raw.go +++ b/br/pkg/task/restore_raw.go @@ -101,7 +101,7 @@ func RunRestoreRaw(c context.Context, g glue.Glue, cmdName string, cfg *RestoreR client.SetRateLimit(cfg.RateLimit) client.SetCrypter(&cfg.CipherInfo) client.SetConcurrencyPerStore(cfg.ConcurrencyPerStore.Value) - err = client.Init(g, mgr.GetStorage()) + err = client.InitConnections(g, mgr.GetStorage()) defer client.Close() if err != nil { return errors.Trace(err) diff --git a/br/pkg/task/restore_test.go b/br/pkg/task/restore_test.go index 27200e6bb486f..2c0bfbea9d85d 100644 --- a/br/pkg/task/restore_test.go +++ b/br/pkg/task/restore_test.go @@ -21,14 +21,16 @@ import ( "github.com/pingcap/tidb/br/pkg/mock" "github.com/pingcap/tidb/br/pkg/restore/split" "github.com/pingcap/tidb/br/pkg/restore/tiflashrec" + "github.com/pingcap/tidb/br/pkg/stream" "github.com/pingcap/tidb/br/pkg/task" - utiltest "github.com/pingcap/tidb/br/pkg/utiltest" + "github.com/pingcap/tidb/br/pkg/utiltest" "github.com/pingcap/tidb/pkg/ddl" "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/types" + filter "github.com/pingcap/tidb/pkg/util/table-filter" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" "github.com/tikv/client-go/v2/oracle" @@ -523,3 +525,249 @@ func TestCheckTikvSpace(t *testing.T) { store := pdhttp.StoreInfo{Store: pdhttp.MetaStore{ID: 1}, Status: pdhttp.StoreStatus{Available: "500PB"}} require.NoError(t, task.CheckStoreSpace(400*pb, &store)) } + +type testCase struct { + name string + filterPattern []string + logBackupHistory []struct { + tableID int64 + tableName string + dbID int64 + } + dbIDToName map[int64]string + expectedTableIDs map[int64][]int64 // dbID -> []tableID + expectedDBs []int64 + expectedTables []int64 +} + +func TestAdjustTablesToRestoreAndCreateTableTracker(t *testing.T) { + // Setup common test database and table maps + dbInfo1 := model.DBInfo{ + ID: 1, + Name: ast.NewCIStr("test_db_1"), + } + dbInfo2 := model.DBInfo{ + ID: 2, + Name: ast.NewCIStr("test_db_2"), + } + snapshotDBMap := map[int64]*metautil.Database{ + 1: { + Info: &dbInfo1, + Tables: []*metautil.Table{ + { + DB: &dbInfo1, + Info: &model.TableInfo{ + ID: 11, + Name: ast.NewCIStr("test_table_11"), + }, + }, + { + DB: &dbInfo1, + Info: &model.TableInfo{ + ID: 12, + Name: ast.NewCIStr("test_table_12"), + }, + }, + }, + }, + 2: { + Info: &dbInfo2, + Tables: []*metautil.Table{ + { + DB: &dbInfo2, + Info: &model.TableInfo{ + ID: 21, + Name: ast.NewCIStr("test_table_21"), + }, + }, + }, + }, + } + + tests := []testCase{ + { + name: "Basic table tracking", + filterPattern: []string{"test_db*.*"}, + logBackupHistory: []struct { + tableID int64 + tableName string + dbID int64 + }{ + {11, "test_table_11", 1}, + {12, "test_table_12", 1}, + {21, "test_table_21", 2}, + }, + expectedTableIDs: map[int64][]int64{ + 1: {11, 12}, + 2: {21}, + }, + expectedDBs: []int64{1, 2}, + expectedTables: []int64{11, 12, 21}, + }, + { + name: "Table not in filter", + filterPattern: []string{"other_db.other_table"}, + logBackupHistory: []struct { + tableID int64 + tableName string + dbID int64 + }{ + {11, "test_table_11", 1}, + {12, "test_table_12", 1}, + {21, "test_table_21", 2}, + }, + expectedTableIDs: map[int64][]int64{}, + expectedDBs: []int64{}, + expectedTables: []int64{}, + }, + { + name: "New table created during log backup", + filterPattern: []string{"test_db*.*"}, + logBackupHistory: []struct { + tableID int64 + tableName string + dbID int64 + }{ + {11, "test_table_11", 1}, + {12, "test_table_12", 1}, + {21, "test_table_21", 2}, + {13, "new_table", 1}, + }, + expectedTableIDs: map[int64][]int64{ + 1: {11, 12, 13}, + 2: {21}, + }, + expectedDBs: []int64{1, 2}, + expectedTables: []int64{11, 12, 21}, // 13 not in full backup + }, + { + name: "Table renamed into filter during log backup", + filterPattern: []string{"test_db_2.*"}, + logBackupHistory: []struct { + tableID int64 + tableName string + dbID int64 + }{ + {11, "test_table_11", 1}, // drop + {11, "renamed_table", 2}, // create + {12, "test_table_12", 1}, + {21, "test_table_21", 2}, + }, + expectedTableIDs: map[int64][]int64{ + 2: {11, 21}, + }, + expectedDBs: []int64{1, 2}, // need original db for restore + expectedTables: []int64{11, 21}, + }, + { + name: "Table renamed out of filter during log backup", + filterPattern: []string{"test_db_1.*"}, + logBackupHistory: []struct { + tableID int64 + tableName string + dbID int64 + }{ + {11, "test_table_11", 1}, + {11, "renamed_table", 2}, + {12, "test_table_12", 1}, + {21, "test_table_21", 2}, + }, + expectedTableIDs: map[int64][]int64{ + 1: {12}, + }, + expectedDBs: []int64{1}, + expectedTables: []int64{12}, + }, + { + name: "Log backup table not in snapshot", + filterPattern: []string{"test_db_1.*"}, + logBackupHistory: []struct { + tableID int64 + tableName string + dbID int64 + }{ + {11, "test_table", 1}, + }, + // using empty snapshotDBMap for this test + expectedTableIDs: map[int64][]int64{}, + expectedDBs: []int64{}, + expectedTables: []int64{}, + }, + { + name: "DB created during log backup", + filterPattern: []string{"test_db_1.*"}, + logBackupHistory: []struct { + tableID int64 + tableName string + dbID int64 + }{ + {11, "test_table", 1}, + }, + dbIDToName: map[int64]string{ + 1: "test_db_1", + }, + // using empty snapshotDBMap for this test + expectedTableIDs: map[int64][]int64{ + 1: {11}, + }, + expectedDBs: []int64{}, // not in full backup + expectedTables: []int64{}, // not in full backup + }, + } + + for _, tc := range tests { + t.Run(tc.name, func(t *testing.T) { + // use local snapshotDBMap for special cases that need empty map + localSnapshotDBMap := snapshotDBMap + if tc.name == "Log backup table not in snapshot" || tc.name == "DB created during log backup" { + localSnapshotDBMap = map[int64]*metautil.Database{} + } + + fileMap := map[int64][]*backuppb.File{} + tableMap := map[int64]*metautil.Table{} + dbMap := map[int64]*metautil.Database{} + logBackupTableHistory := stream.NewTableHistoryManager() + + for _, h := range tc.logBackupHistory { + logBackupTableHistory.AddTableHistory(h.tableID, h.tableName, h.dbID) + } + + for dbID, dbName := range tc.dbIDToName { + logBackupTableHistory.RecordDBIdToName(dbID, dbName) + } + + testFilter, err := filter.Parse(tc.filterPattern) + require.NoError(t, err) + cfg := &task.RestoreConfig{ + Config: task.Config{ + TableFilter: testFilter, + }, + } + + // Run the function + err = task.AdjustTablesToRestoreAndCreateTableTracker( + logBackupTableHistory, + cfg, + localSnapshotDBMap, + fileMap, + tableMap, + dbMap, + ) + require.NoError(t, err) + + for dbID, tableIDs := range tc.expectedTableIDs { + for _, tableID := range tableIDs { + require.True(t, cfg.PiTRTableTracker.ContainsPhysicalId(dbID, tableID)) + } + } + + for _, dbID := range tc.expectedDBs { + require.NotNil(t, dbMap[dbID]) + } + + for _, tableID := range tc.expectedTables { + require.NotNil(t, tableMap[tableID]) + } + }) + } +} diff --git a/br/pkg/task/restore_txn.go b/br/pkg/task/restore_txn.go index 16d8e099f659d..720d40dc6bf69 100644 --- a/br/pkg/task/restore_txn.go +++ b/br/pkg/task/restore_txn.go @@ -44,7 +44,7 @@ func RunRestoreTxn(c context.Context, g glue.Glue, cmdName string, cfg *Config) client.SetRateLimit(cfg.RateLimit) client.SetCrypter(&cfg.CipherInfo) client.SetConcurrencyPerStore(uint(cfg.Concurrency)) - err = client.Init(g, mgr.GetStorage()) + err = client.InitConnections(g, mgr.GetStorage()) defer client.Close() if err != nil { return errors.Trace(err) diff --git a/br/pkg/task/stream.go b/br/pkg/task/stream.go index e83cb92c7d4d4..54c7afb063fb1 100644 --- a/br/pkg/task/stream.go +++ b/br/pkg/task/stream.go @@ -57,6 +57,7 @@ import ( "github.com/pingcap/tidb/br/pkg/utils/iter" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta/model" + "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/util/cdcutil" "github.com/spf13/pflag" "github.com/tikv/client-go/v2/oracle" @@ -482,11 +483,11 @@ func (s *streamMgr) checkStreamStartEnable(ctx context.Context) error { return nil } -type RestoreFunc func(string) error +type RestoreGCFunc func(string) error -// KeepGcDisabled keeps GC disabled and return a function that used to gc enabled. +// DisableGC disables and returns a function that can enable gc back. // gc.ratio-threshold = "-1.0", which represents disable gc in TiKV. -func KeepGcDisabled(g glue.Glue, store kv.Storage) (RestoreFunc, string, error) { +func DisableGC(g glue.Glue, store kv.Storage) (RestoreGCFunc, string, error) { se, err := g.CreateSession(store) if err != nil { return nil, "", errors.Trace(err) @@ -597,7 +598,7 @@ func RunStreamStart( // locked means this is a stream task restart. Or create a new stream task. if locked { - logInfo, err := getLogRange(ctx, &cfg.Config) + logInfo, err := getLogInfo(ctx, &cfg.Config) if err != nil { return errors.Trace(err) } @@ -711,13 +712,13 @@ func RunStreamMetadata( ctx = opentracing.ContextWithSpan(ctx, span1) } - logInfo, err := getLogRange(ctx, &cfg.Config) + logInfo, err := getLogInfo(ctx, &cfg.Config) if err != nil { return errors.Trace(err) } - logMinDate := stream.FormatDate(oracle.GetTimeFromTS(logInfo.logMinTS)) - logMaxDate := stream.FormatDate(oracle.GetTimeFromTS(logInfo.logMaxTS)) + logMinDate := utils.FormatDate(oracle.GetTimeFromTS(logInfo.logMinTS)) + logMaxDate := utils.FormatDate(oracle.GetTimeFromTS(logInfo.logMaxTS)) summary.Log(cmdName, zap.Uint64("log-min-ts", logInfo.logMinTS), zap.String("log-min-date", logMinDate), zap.Uint64("log-max-ts", logInfo.logMaxTS), @@ -1260,7 +1261,8 @@ func checkIncompatibleChangefeed(ctx context.Context, backupTS uint64, etcdCLI * return nil } -// RunStreamRestore restores stream log. +// RunStreamRestore is the entry point to do PiTR restore. It can optionally start a full/snapshot restore followed +// by the log restore. func RunStreamRestore( c context.Context, mgr *conn.Mgr, @@ -1279,7 +1281,7 @@ func RunStreamRestore( if err != nil { return errors.Trace(err) } - logInfo, err := getLogRangeWithStorage(ctx, s) + logInfo, err := getLogInfoFromStorage(ctx, s) if err != nil { return errors.Trace(err) } @@ -1317,7 +1319,7 @@ func RunStreamRestore( return errors.Trace(err) } - checkInfo, err := checkPiTRTaskInfo(ctx, mgr, g, cfg) + taskInfo, err := generatePiTRTaskInfo(ctx, mgr, g, cfg) if err != nil { return errors.Trace(err) } @@ -1326,42 +1328,92 @@ func RunStreamRestore( failpoint.Return(errors.New("failpoint: failed before full restore")) }) - recorder := tiflashrec.New() - cfg.tiflashRecorder = recorder + // restore log. + cfg.adjustRestoreConfigForStreamRestore() + cfg.tiflashRecorder = tiflashrec.New() + logClient, err := createLogClient(ctx, g, cfg, mgr) + if err != nil { + return errors.Trace(err) + } + defer logClient.Close(ctx) + + ddlFiles, err := logClient.LoadDDLFiles(ctx) + if err != nil { + return errors.Trace(err) + } + // TODO: pitr filtered restore doesn't support restore system table yet + if cfg.ExplicitFilter { + if cfg.TableFilter.MatchSchema(mysql.SystemDB) { + return errors.Annotatef(berrors.ErrInvalidArgument, + "PiTR doesn't support custom filter to include system db, consider to exclude system db") + } + } + metaInfoProcessor := logclient.NewMetaKVInfoProcessor(logClient) + // only doesn't need to build if id map has been saved during log restore + idMapSaved := isCurrentIdMapSaved(taskInfo.CheckpointInfo) + if !idMapSaved { + // we restore additional tables at full snapshot phase when it is renamed into the filter range + // later in log backup. + // we also ignore the tables that currently in filter range but later renamed out of the filter. + log.Info("reading meta kv files to collect table renaming and id mapping information") + err = metaInfoProcessor.ReadMetaKVFilesAndBuildInfo(ctx, ddlFiles) + if err != nil { + return errors.Trace(err) + } + dbReplace := metaInfoProcessor.GetTableMappingManager().DBReplaceMap + stream.LogDBReplaceMap("scanned log meta kv before snapshot restore", dbReplace) + } + // restore full snapshot. - if checkInfo.NeedFullRestore { + if taskInfo.NeedFullRestore { logStorage := cfg.Config.Storage cfg.Config.Storage = cfg.FullBackupStorage + snapshotRestoreConfig := SnapshotRestoreConfig{ + RestoreConfig: cfg, + piTRTaskInfo: taskInfo, + logTableHistoryManager: metaInfoProcessor.GetTableHistoryManager(), + } // TiFlash replica is restored to down-stream on 'pitr' currently. - if err = runSnapshotRestore(ctx, mgr, g, FullRestoreCmd, cfg, checkInfo); err != nil { + if err = runSnapshotRestore(ctx, mgr, g, FullRestoreCmd, &snapshotRestoreConfig); err != nil { return errors.Trace(err) } cfg.Config.Storage = logStorage } else if len(cfg.FullBackupStorage) > 0 { - skipMsg := []byte(fmt.Sprintf("%s command is skipped due to checkpoint mode for restore\n", FullRestoreCmd)) - if _, err := glue.GetConsole(g).Out().Write(skipMsg); err != nil { + if err = WriteStringToConsole(g, fmt.Sprintf("%s is skipped due to checkpoint mode for restore\n", FullRestoreCmd)); err != nil { return errors.Trace(err) } - if checkInfo.CheckpointInfo != nil && checkInfo.CheckpointInfo.Metadata != nil && checkInfo.CheckpointInfo.Metadata.TiFlashItems != nil { + if taskInfo.hasTiFlashItemsInCheckpoint() { log.Info("load tiflash records of snapshot restore from checkpoint") - cfg.tiflashRecorder.Load(checkInfo.CheckpointInfo.Metadata.TiFlashItems) + cfg.tiflashRecorder.Load(taskInfo.CheckpointInfo.Metadata.TiFlashItems) } } - // restore log. - cfg.adjustRestoreConfigForStreamRestore() - if err := restoreStream(ctx, mgr, g, cfg, checkInfo.CheckpointInfo); err != nil { + logRestoreConfig := &LogRestoreConfig{ + RestoreConfig: cfg, + checkpointTaskInfo: taskInfo.CheckpointInfo, + tableMappingManager: metaInfoProcessor.GetTableMappingManager(), + logClient: logClient, + ddlFiles: ddlFiles, + } + if err := restoreStream(ctx, mgr, g, logRestoreConfig); err != nil { return errors.Trace(err) } return nil } -// RunStreamRestore start restore job +type LogRestoreConfig struct { + *RestoreConfig + checkpointTaskInfo *checkpoint.CheckpointTaskInfoForLogRestore + tableMappingManager *stream.TableMappingManager + logClient *logclient.LogClient + ddlFiles []logclient.Log +} + +// restoreStream starts the log restore func restoreStream( c context.Context, mgr *conn.Mgr, g glue.Glue, - cfg *RestoreConfig, - taskInfo *checkpoint.CheckpointTaskInfoForLogRestore, + cfg *LogRestoreConfig, ) (err error) { var ( totalKVCount uint64 @@ -1383,9 +1435,9 @@ func restoreStream( zap.Uint64("source-start-point", cfg.StartTS), zap.Uint64("source-end-point", cfg.RestoreTS), zap.Uint64("target-end-point", currentTS), - zap.String("source-start", stream.FormatDate(oracle.GetTimeFromTS(cfg.StartTS))), - zap.String("source-end", stream.FormatDate(oracle.GetTimeFromTS(cfg.RestoreTS))), - zap.String("target-end", stream.FormatDate(oracle.GetTimeFromTS(currentTS))), + zap.String("source-start", utils.FormatDate(oracle.GetTimeFromTS(cfg.StartTS))), + zap.String("source-end", utils.FormatDate(oracle.GetTimeFromTS(cfg.RestoreTS))), + zap.String("target-end", utils.FormatDate(oracle.GetTimeFromTS(currentTS))), zap.Uint64("total-kv-count", totalKVCount), zap.Uint64("skipped-kv-count-by-checkpoint", checkpointTotalKVCount), zap.String("total-size", units.HumanSize(float64(totalSize))), @@ -1411,39 +1463,39 @@ func restoreStream( ctx = opentracing.ContextWithSpan(ctx, span1) } - client, err := createRestoreClient(ctx, g, cfg, mgr) + client := cfg.logClient + migs, err := client.GetMigrations(ctx) if err != nil { - return errors.Annotate(err, "failed to create restore client") + return errors.Trace(err) } - defer client.Close(ctx) + client.BuildMigrations(migs.Migs) + defer cleanUpWithRetErr(&err, migs.ReadLock.Unlock) defer client.RestoreSSTStatisticFields(&extraFields) - if taskInfo != nil && taskInfo.Metadata != nil { - // reuse the task's rewrite ts - log.Info("reuse the task's rewrite ts", zap.Uint64("rewrite-ts", taskInfo.Metadata.RewriteTS)) - currentTS = taskInfo.Metadata.RewriteTS - } else { - currentTS, err = restore.GetTSWithRetry(ctx, mgr.GetPDClient()) - if err != nil { - return errors.Trace(err) - } + ddlFiles := cfg.ddlFiles + tableMappingManager := cfg.tableMappingManager + + currentTS, err = getCurrentTSFromCheckpointOrPD(ctx, mgr, cfg) + if err != nil { + return errors.Trace(err) } + if err := client.SetCurrentTS(currentTS); err != nil { return errors.Trace(err) } importModeSwitcher := restore.NewImportModeSwitcher(mgr.GetPDClient(), cfg.Config.SwitchModeInterval, mgr.GetTLSConfig()) - restoreSchedulers, _, err := restore.RestorePreWork(ctx, mgr, importModeSwitcher, cfg.Online, false) + restoreSchedulersFunc, _, err := restore.RestorePreWork(ctx, mgr, importModeSwitcher, cfg.Online, false) if err != nil { return errors.Trace(err) } // Always run the post-work even on error, so we don't stuck in the import // mode or emptied schedulers - defer restore.RestorePostWork(ctx, importModeSwitcher, restoreSchedulers, cfg.Online) + defer restore.RestorePostWork(ctx, importModeSwitcher, restoreSchedulersFunc, cfg.Online) // It need disable GC in TiKV when PiTR. // because the process of PITR is concurrent and kv events isn't sorted by tso. - restoreGc, oldRatio, err := KeepGcDisabled(g, mgr.GetStorage()) + restoreGCFunc, oldGCRatio, err := DisableGC(g, mgr.GetStorage()) if err != nil { return errors.Trace(err) } @@ -1455,90 +1507,42 @@ func restoreStream( return } - // If the oldRatio is negative, which is not normal status. + // If the oldGcRatio is negative, which is not normal status. // It should set default value "1.1" after PiTR finished. - if strings.HasPrefix(oldRatio, "-") { - log.Warn("the original gc-ratio is negative, reset by default value 1.1", zap.String("old-gc-ratio", oldRatio)) - oldRatio = utils.DefaultGcRatioVal + if strings.HasPrefix(oldGCRatio, "-") { + log.Warn("the original gc-ratio is negative, reset by default value 1.1", zap.String("old-gc-ratio", oldGCRatio)) + oldGCRatio = utils.DefaultGcRatioVal } - log.Info("start to restore gc", zap.String("ratio", oldRatio)) - if err := restoreGc(oldRatio); err != nil { - log.Error("failed to set gc enabled", zap.Error(err)) + log.Info("start to restore gc", zap.String("ratio", oldGCRatio)) + if err := restoreGCFunc(oldGCRatio); err != nil { + log.Error("failed to restore gc", zap.Error(err)) } log.Info("finish restoring gc") }() var sstCheckpointSets map[string]struct{} if cfg.UseCheckpoint { - oldRatioFromCheckpoint, err := client.InitCheckpointMetadataForLogRestore(ctx, cfg.StartTS, cfg.RestoreTS, oldRatio, cfg.tiflashRecorder) + gcRatioFromCheckpoint, err := client.LoadOrCreateCheckpointMetadataForLogRestore(ctx, cfg.StartTS, cfg.RestoreTS, oldGCRatio, cfg.tiflashRecorder) if err != nil { return errors.Trace(err) } - oldRatio = oldRatioFromCheckpoint + oldGCRatio = gcRatioFromCheckpoint sstCheckpointSets, err = client.InitCheckpointMetadataForCompactedSstRestore(ctx) if err != nil { return errors.Trace(err) } } - encryptionManager, err := encryption.NewManager(&cfg.LogBackupCipherInfo, &cfg.MasterKeyConfig) - if err != nil { - return errors.Annotate(err, "failed to create encryption manager for log restore") - } - defer encryptionManager.Close() - err = client.InstallLogFileManager(ctx, cfg.StartTS, cfg.RestoreTS, cfg.MetadataDownloadBatchSize, encryptionManager) - if err != nil { - return err - } - migs, err := client.GetMigrations(ctx) - if err != nil { - return errors.Trace(err) - } - client.BuildMigrations(migs.Migs) - defer cleanUpWithRetErr(&err, migs.ReadLock.Unlock) - // get full backup meta storage to generate rewrite rules. - fullBackupStorage, err := parseFullBackupTablesStorage(cfg) - if err != nil { + // build and save id map + if err := buildAndSaveIDMapIfNeeded(ctx, client, cfg, tableMappingManager); err != nil { return errors.Trace(err) } - // load the id maps only when the checkpoint mode is used and not the first execution - currentIdMapSaved := false - if taskInfo != nil && taskInfo.Progress == checkpoint.InLogRestoreAndIdMapPersist { - currentIdMapSaved = true - } - - ddlFiles, err := client.LoadDDLFilesAndCountDMLFiles(ctx) - if err != nil { - return err - } - - // get the schemas ID replace information. - // since targeted full backup storage, need to use the full backup cipher - tableMappingManager, err := client.BuildTableMappingManager(ctx, &logclient.BuildTableMappingManagerConfig{ - CurrentIdMapSaved: currentIdMapSaved, - TableFilter: cfg.TableFilter, - FullBackupStorage: fullBackupStorage, - CipherInfo: &cfg.Config.CipherInfo, - Files: ddlFiles, - }) + // build schema replace + schemasReplace, err := buildSchemaReplace(client, cfg, tableMappingManager) if err != nil { return errors.Trace(err) } - schemasReplace := stream.NewSchemasReplace(tableMappingManager.DbReplaceMap, cfg.tiflashRecorder, - client.CurrentTS(), cfg.TableFilter, client.RecordDeleteRange) - schemasReplace.AfterTableRewritten = func(deleted bool, tableInfo *model.TableInfo) { - // When the table replica changed to 0, the tiflash replica might be set to `nil`. - // We should remove the table if we meet. - if deleted || tableInfo.TiFlashReplica == nil { - cfg.tiflashRecorder.DelTable(tableInfo.ID) - return - } - cfg.tiflashRecorder.AddTable(tableInfo.ID, *tableInfo.TiFlashReplica) - // Remove the replica firstly. Let's restore them at the end. - tableInfo.TiFlashReplica = nil - } - updateStats := func(kvCount uint64, size uint64) { mu.Lock() defer mu.Unlock() @@ -1546,14 +1550,14 @@ func restoreStream( totalSize += size } + var rp *logclient.RestoreMetaKVProcessor if err = glue.WithProgress(ctx, g, "Restore Meta Files", int64(len(ddlFiles)), !cfg.LogProgress, func(p glue.Progress) error { - client.RunGCRowsLoader(ctx) - return client.RestoreAndRewriteMetaKVFiles(ctx, ddlFiles, schemasReplace, updateStats, p.Inc) + rp = logclient.NewRestoreMetaKVProcessor(client, schemasReplace, updateStats, p.Inc) + return rp.RestoreAndRewriteMetaKVFiles(ctx, ddlFiles) }); err != nil { return errors.Annotate(err, "failed to restore meta files") } - - rewriteRules := initRewriteRules(schemasReplace) + rewriteRules := buildRewriteRules(schemasReplace) ingestRecorder := schemasReplace.GetIngestRecorder() if err := rangeFilterFromIngestRecorder(ingestRecorder, rewriteRules); err != nil { @@ -1650,10 +1654,18 @@ func restoreStream( return errors.Annotate(err, "failed to clean up") } + // we could restore some extra tables if exchange partition happens + // clean up tables that's not in the filter range + if err = cleanUpFilteredOutTables(ctx, client, cfg.RestoreConfig, schemasReplace); err != nil { + return errors.Annotate(err, "failed to drop filtered tables") + } + + // to delete range(table, schema) that's dropped during log backup if err = client.InsertGCRows(ctx); err != nil { return errors.Annotate(err, "failed to insert rows into gc_delete_range") } + // index ingestion is not captured by regular log backup, so we need to manually ingest again if err = client.RepairIngestIndex(ctx, ingestRecorder, g); err != nil { return errors.Annotate(err, "failed to repair ingest index") } @@ -1689,11 +1701,11 @@ func restoreStream( return nil } -func createRestoreClient(ctx context.Context, g glue.Glue, cfg *RestoreConfig, mgr *conn.Mgr) (*logclient.LogClient, error) { +func createLogClient(ctx context.Context, g glue.Glue, cfg *RestoreConfig, mgr *conn.Mgr) (*logclient.LogClient, error) { var err error keepaliveCfg := GetKeepalive(&cfg.Config) keepaliveCfg.PermitWithoutStream = true - client := logclient.NewRestoreClient(mgr.GetPDClient(), mgr.GetPDHTTPClient(), mgr.GetTLSConfig(), keepaliveCfg) + client := logclient.NewLogClient(mgr.GetPDClient(), mgr.GetPDHTTPClient(), mgr.GetTLSConfig(), keepaliveCfg) err = client.Init(ctx, g, mgr.GetStorage()) if err != nil { @@ -1725,7 +1737,8 @@ func createRestoreClient(ctx context.Context, g glue.Glue, cfg *RestoreConfig, m } return nil, nil } - err = client.InitClients(ctx, u, createCheckpointSessionFn, uint(cfg.Concurrency), cfg.ConcurrencyPerStore.Value) + + err = client.InitClients(ctx, u, createCheckpointSessionFn, uint(cfg.PitrConcurrency), cfg.ConcurrencyPerStore.Value) if err != nil { return nil, errors.Trace(err) } @@ -1735,6 +1748,14 @@ func createRestoreClient(ctx context.Context, g glue.Glue, cfg *RestoreConfig, m return nil, errors.Trace(err) } + encryptionManager, err := encryption.NewManager(&cfg.LogBackupCipherInfo, &cfg.MasterKeyConfig) + if err != nil { + return nil, errors.Annotate(err, "failed to create encryption manager for log restore") + } + if err = client.InstallLogFileManager(ctx, cfg.StartTS, cfg.RestoreTS, cfg.MetadataDownloadBatchSize, encryptionManager); err != nil { + return nil, errors.Trace(err) + } + return client, nil } @@ -1790,8 +1811,8 @@ type backupLogInfo struct { clusterID uint64 } -// getLogRange gets the log-min-ts and log-max-ts of starting log backup. -func getLogRange( +// getLogInfo gets the log-min-ts and log-max-ts of starting log backup. +func getLogInfo( ctx context.Context, cfg *Config, ) (backupLogInfo, error) { @@ -1799,10 +1820,10 @@ func getLogRange( if err != nil { return backupLogInfo{}, errors.Trace(err) } - return getLogRangeWithStorage(ctx, s) + return getLogInfoFromStorage(ctx, s) } -func getLogRangeWithStorage( +func getLogInfoFromStorage( ctx context.Context, s storage.ExternalStorage, ) (backupLogInfo, error) { @@ -1909,26 +1930,27 @@ func parseFullBackupTablesStorage( }, nil } -func initRewriteRules(schemasReplace *stream.SchemasReplace) map[int64]*restoreutils.RewriteRules { +func buildRewriteRules(schemasReplace *stream.SchemasReplace) map[int64]*restoreutils.RewriteRules { rules := make(map[int64]*restoreutils.RewriteRules) - filter := schemasReplace.TableFilter - for _, dbReplace := range schemasReplace.DbMap { - if utils.IsSysDB(dbReplace.Name) || !filter.MatchSchema(dbReplace.Name) { + for _, dbReplace := range schemasReplace.DbReplaceMap { + if dbReplace.Filtered { continue } - for oldTableID, tableReplace := range dbReplace.TableMap { - if !filter.MatchTable(dbReplace.Name, tableReplace.Name) { + if tableReplace.Filtered { continue } - if _, exist := rules[oldTableID]; !exist { log.Info("add rewrite rule", zap.String("tableName", dbReplace.Name+"."+tableReplace.Name), zap.Int64("oldID", oldTableID), zap.Int64("newID", tableReplace.TableID)) rules[oldTableID] = restoreutils.GetRewriteRuleOfTable( oldTableID, tableReplace.TableID, tableReplace.IndexMap, false) + } else { + log.Info("skip adding table rewrite rule, already exists", + zap.Int64("oldID", oldTableID), + zap.Int64("newID", tableReplace.TableID)) } for oldID, newID := range tableReplace.PartitionMap { @@ -1937,6 +1959,10 @@ func initRewriteRules(schemasReplace *stream.SchemasReplace) map[int64]*restoreu zap.String("tableName", dbReplace.Name+"."+tableReplace.Name), zap.Int64("oldID", oldID), zap.Int64("newID", newID)) rules[oldID] = restoreutils.GetRewriteRuleOfTable(oldID, newID, tableReplace.IndexMap, false) + } else { + log.Info("skip adding partition rewrite rule, already exists", + zap.Int64("oldID", oldID), + zap.Int64("newID", newID)) } } } @@ -1945,7 +1971,7 @@ func initRewriteRules(schemasReplace *stream.SchemasReplace) map[int64]*restoreu } // ShiftTS gets a smaller shiftTS than startTS. -// It has a safe duration between shiftTS and startTS for trasaction. +// It has a safe duration between shiftTS and startTS for transaction. func ShiftTS(startTS uint64) uint64 { physical := oracle.ExtractPhysical(startTS) logical := oracle.ExtractLogical(startTS) @@ -1971,7 +1997,11 @@ type PiTRTaskInfo struct { FullRestoreCheckErr error } -func checkPiTRTaskInfo( +func (p *PiTRTaskInfo) hasTiFlashItemsInCheckpoint() bool { + return p.CheckpointInfo != nil && p.CheckpointInfo.Metadata != nil && p.CheckpointInfo.Metadata.TiFlashItems != nil +} + +func generatePiTRTaskInfo( ctx context.Context, mgr *conn.Mgr, g glue.Glue, @@ -1994,7 +2024,7 @@ func checkPiTRTaskInfo( if err != nil { return checkInfo, errors.Trace(err) } - // the log restore checkpoint metadata is persist, so the PITR is in the log restore stage. + // the log restore checkpoint metadata is persisted, so the PITR is in the log restore stage. if curTaskInfo.Metadata != nil { // TODO: check whether user has manually modified the cluster(ddl). If so, regard the behavior // as restore from scratch. (update `curTaskInfo.RewriteTs` to 0 as an uninitial value) @@ -2065,3 +2095,146 @@ func waitUntilSchemaReload(ctx context.Context, client *logclient.LogClient) err log.Info("reloading schema finished", zap.Duration("timeTaken", time.Since(reloadStart))) return nil } + +func isCurrentIdMapSaved(checkpointTaskInfo *checkpoint.CheckpointTaskInfoForLogRestore) bool { + return checkpointTaskInfo != nil && checkpointTaskInfo.Progress == checkpoint.InLogRestoreAndIdMapPersisted +} + +func buildSchemaReplace( + client *logclient.LogClient, + cfg *LogRestoreConfig, + tableMappingManager *stream.TableMappingManager) (*stream.SchemasReplace, error) { + schemasReplace := stream.NewSchemasReplace(tableMappingManager.DBReplaceMap, cfg.tiflashRecorder, + client.CurrentTS(), client.RecordDeleteRange) + schemasReplace.AfterTableRewrittenFn = func(deleted bool, tableInfo *model.TableInfo) { + // When the table replica changed to 0, the tiflash replica might be set to `nil`. + // We should remove the table if we meet. + if deleted || tableInfo.TiFlashReplica == nil { + cfg.tiflashRecorder.DelTable(tableInfo.ID) + return + } + cfg.tiflashRecorder.AddTable(tableInfo.ID, *tableInfo.TiFlashReplica) + // Remove the replica first and restore them at the end. + tableInfo.TiFlashReplica = nil + } + return schemasReplace, nil +} + +func buildAndSaveIDMapIfNeeded(ctx context.Context, client *logclient.LogClient, cfg *LogRestoreConfig, + tableMappingManager *stream.TableMappingManager) error { + // get full backup meta storage if needed. + fullBackupStorageConfig, err := parseFullBackupTablesStorage(cfg.RestoreConfig) + if err != nil { + return errors.Trace(err) + } + + // get the schemas ID replace information. + saved := isCurrentIdMapSaved(cfg.checkpointTaskInfo) + dbReplaces, err := client.GetBaseIDMap(ctx, &logclient.GetIDMapConfig{ + LoadSavedIDMap: saved, + PiTRTableTracker: cfg.PiTRTableTracker, + FullBackupStorageConfig: fullBackupStorageConfig, + CipherInfo: &cfg.Config.CipherInfo, + }) + if err != nil { + return errors.Trace(err) + } + + if saved { + err := tableMappingManager.FromDBReplaceMap(dbReplaces) + if err != nil { + return errors.Trace(err) + } + } else { + tableMappingManager.MergeBaseDBReplace(dbReplaces) + if cfg.PiTRTableTracker != nil { + tableMappingManager.FilterDBReplaceMap(cfg.PiTRTableTracker) + } else { + log.Warn("pitr table tracker is nil, base map is not from full backup") + } + err = tableMappingManager.ReplaceTemporaryIDs(ctx, client.GenGlobalIDs) + if err != nil { + return errors.Trace(err) + } + if err = client.SaveIdMapWithFailPoints(ctx, tableMappingManager); err != nil { + return errors.Trace(err) + } + } + return nil +} + +func getCurrentTSFromCheckpointOrPD(ctx context.Context, mgr *conn.Mgr, cfg *LogRestoreConfig) (uint64, error) { + if cfg.checkpointTaskInfo != nil && cfg.checkpointTaskInfo.Metadata != nil { + // reuse the checkpoint task's rewrite ts + rewriteTS := cfg.checkpointTaskInfo.Metadata.RewriteTS + log.Info("reuse the task's rewrite ts", zap.Uint64("rewrite-ts", rewriteTS)) + return rewriteTS, nil + } + currentTS, err := restore.GetTSWithRetry(ctx, mgr.GetPDClient()) + if err != nil { + return 0, errors.Trace(err) + } + return currentTS, nil +} + +func cleanUpFilteredOutTables(ctx context.Context, client *logclient.LogClient, cfg *RestoreConfig, + schemaReplace *stream.SchemasReplace) error { + if cfg.PiTRTableTracker == nil || cfg.PiTRTableTracker.TableToCleanup == nil { + return nil + } + tablesToCleanup := cfg.PiTRTableTracker.TableToCleanup + tableFilter := cfg.TableFilter + infoSchema := client.GetDomain().InfoSchema() + log.Info("start to drop filtered out tables post restore") + for upDbID, tables := range tablesToCleanup { + dbReplace, exist := schemaReplace.DbReplaceMap[upDbID] + if !exist { + return errors.Errorf("database ID mapping not found in schema replace for upstream DB ID %d", upDbID) + } + downDbID := dbReplace.DbID + + dbInfo, exist := infoSchema.SchemaByID(downDbID) + if !exist { + log.Info("database not found during cleanup, skipping", + zap.Int64("upstream_db_id", upDbID), + zap.Int64("downstream_db_id", downDbID)) + continue + } + + for upTableID := range tables { + // get downstream table ID from schema replace + tableReplace, exist := dbReplace.TableMap[upTableID] + if !exist { + return errors.Errorf("table ID mapping not found in schema replace for upstream table ID %d in DB %d", + upTableID, upDbID) + } + downTableID := tableReplace.TableID + tbl, exist := infoSchema.TableByID(ctx, downTableID) + if !exist { + log.Info("table not found during cleanup, skipping", + zap.Int64("upstream_db_id", upDbID), + zap.Int64("downstream_db_id", downDbID), + zap.Int64("upstream_table_id", upTableID), + zap.Int64("downstream_table_id", downTableID)) + continue + } + + // check if table matches filter + if !tableFilter.MatchTable(dbInfo.Name.O, tbl.Meta().Name.O) { + log.Info("dropping filtered out table", + zap.String("db", dbInfo.Name.O), + zap.String("table", tbl.Meta().Name.O), + zap.Int64("upstream_db_id", upDbID), + zap.Int64("downstream_db_id", downDbID), + zap.Int64("upstream_table_id", upTableID), + zap.Int64("downstream_table_id", downTableID)) + + err := client.DropTable(ctx, dbInfo.Name.O, tbl.Meta().Name.O) + if err != nil { + return errors.Trace(err) + } + } + } + } + return nil +} diff --git a/br/pkg/task/stream_test.go b/br/pkg/task/stream_test.go index f3393a22a03d1..50da91ec28094 100644 --- a/br/pkg/task/stream_test.go +++ b/br/pkg/task/stream_test.go @@ -192,7 +192,7 @@ func TestGetLogRangeWithFullBackupDir(t *testing.T) { cfg := Config{ Storage: testDir, } - _, err = getLogRange(context.TODO(), &cfg) + _, err = getLogInfo(context.TODO(), &cfg) require.ErrorIs(t, err, berrors.ErrStorageUnknown) require.ErrorContains(t, err, "the storage has been used for full backup") } @@ -215,7 +215,7 @@ func TestGetLogRangeWithLogBackupDir(t *testing.T) { cfg := Config{ Storage: testDir, } - logInfo, err := getLogRange(context.TODO(), &cfg) + logInfo, err := getLogInfo(context.TODO(), &cfg) require.Nil(t, err) require.Equal(t, logInfo.logMinTS, startLogBackupTS) } diff --git a/br/pkg/utils/BUILD.bazel b/br/pkg/utils/BUILD.bazel index 85d5d3c36c7a1..aa8c0c836ed30 100644 --- a/br/pkg/utils/BUILD.bazel +++ b/br/pkg/utils/BUILD.bazel @@ -9,6 +9,7 @@ go_library( "dyn_pprof_unix.go", "encryption.go", "error_handling.go", + "filter.go", "json.go", "key.go", "misc.go", @@ -36,10 +37,13 @@ go_library( "//pkg/parser/terror", "//pkg/parser/types", "//pkg/sessionctx", + "//pkg/tablecodec", "//pkg/util", + "//pkg/util/codec", "//pkg/util/encrypt", "//pkg/util/logutil", "//pkg/util/sqlexec", + "//pkg/util/table-filter", "@com_github_cheggaaa_pb_v3//:pb", "@com_github_docker_go_units//:go-units", "@com_github_google_uuid//:uuid", @@ -75,6 +79,7 @@ go_test( "backoff_test.go", "db_test.go", "error_handling_test.go", + "filter_test.go", "json_test.go", "key_test.go", "main_test.go", @@ -86,7 +91,7 @@ go_test( ], embed = [":utils"], flaky = True, - shard_count = 34, + shard_count = 37, deps = [ "//br/pkg/errors", "//pkg/kv", @@ -104,6 +109,7 @@ go_test( "@com_github_pingcap_kvproto//pkg/brpb", "@com_github_pingcap_kvproto//pkg/errorpb", "@com_github_stretchr_testify//require", + "@com_github_tikv_client_go_v2//oracle", "@com_github_tikv_client_go_v2//tikv", "@com_github_tikv_pd_client//:client", "@io_etcd_go_etcd_tests_v3//integration", diff --git a/br/pkg/utils/consts/BUILD.bazel b/br/pkg/utils/consts/BUILD.bazel new file mode 100644 index 0000000000000..1c9766fe93e1a --- /dev/null +++ b/br/pkg/utils/consts/BUILD.bazel @@ -0,0 +1,8 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library") + +go_library( + name = "consts", + srcs = ["consts.go"], + importpath = "github.com/pingcap/tidb/br/pkg/utils/consts", + visibility = ["//visibility:public"], +) diff --git a/br/pkg/utils/consts/consts.go b/br/pkg/utils/consts/consts.go new file mode 100644 index 0000000000000..93390bcbacc38 --- /dev/null +++ b/br/pkg/utils/consts/consts.go @@ -0,0 +1,21 @@ +// Copyright 2024 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 consts + +// Default columnFamily and write columnFamily +const ( + DefaultCF = "default" + WriteCF = "write" +) diff --git a/br/pkg/utils/filter.go b/br/pkg/utils/filter.go new file mode 100644 index 0000000000000..9daee9482e2b2 --- /dev/null +++ b/br/pkg/utils/filter.go @@ -0,0 +1,150 @@ +// Copyright 2024 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 utils + +import ( + "fmt" + "sort" + "strings" + + filter "github.com/pingcap/tidb/pkg/util/table-filter" +) + +// PiTRIdTracker tracks all the DB and physical ids that need to restore in a PiTR +type PiTRIdTracker struct { + DBIdToPhysicalId map[int64]map[int64]struct{} + TableToCleanup map[int64]map[int64]struct{} +} + +func NewPiTRTableTracker() *PiTRIdTracker { + return &PiTRIdTracker{ + DBIdToPhysicalId: make(map[int64]map[int64]struct{}), + TableToCleanup: make(map[int64]map[int64]struct{}), + } +} + +// AddPhysicalId adds a physical ID to the filter for the given database ID +func (t *PiTRIdTracker) AddPhysicalId(dbID, physicalId int64) { + if t.DBIdToPhysicalId == nil { + t.DBIdToPhysicalId = make(map[int64]map[int64]struct{}) + } + + if _, ok := t.DBIdToPhysicalId[dbID]; !ok { + t.DBIdToPhysicalId[dbID] = make(map[int64]struct{}) + } + + t.DBIdToPhysicalId[dbID][physicalId] = struct{}{} +} + +// AddTableToCleanup tracks all the table id need to be cleaned up after restore +func (t *PiTRIdTracker) AddTableToCleanup(dbID, tableId int64) { + if t.TableToCleanup == nil { + t.TableToCleanup = make(map[int64]map[int64]struct{}) + } + + if _, ok := t.TableToCleanup[dbID]; !ok { + t.TableToCleanup[dbID] = make(map[int64]struct{}) + } + + t.TableToCleanup[dbID][tableId] = struct{}{} +} + +// AddDB adds the database id +func (t *PiTRIdTracker) AddDB(dbID int64) { + if t.DBIdToPhysicalId == nil { + t.DBIdToPhysicalId = make(map[int64]map[int64]struct{}) + } + + if _, ok := t.DBIdToPhysicalId[dbID]; !ok { + t.DBIdToPhysicalId[dbID] = make(map[int64]struct{}) + } +} + +// Remove removes a table ID from the filter for the given database ID. +// Returns true if the table was found and removed, false otherwise. +func (t *PiTRIdTracker) Remove(dbID, physicalId int64) bool { + if tables, ok := t.DBIdToPhysicalId[dbID]; ok { + if _, exists := tables[physicalId]; exists { + delete(tables, physicalId) + return true + } + } + return false +} + +// ContainsPhysicalId checks if the given database ID and table ID combination exists in the filter +func (t *PiTRIdTracker) ContainsPhysicalId(dbID, tableID int64) bool { + if tables, ok := t.DBIdToPhysicalId[dbID]; ok { + _, exists := tables[tableID] + return exists + } + return false +} + +// ContainsDB checks if the given database ID exists in the filter +func (t *PiTRIdTracker) ContainsDB(dbID int64) bool { + _, ok := t.DBIdToPhysicalId[dbID] + return ok +} + +// String returns a string representation of the PiTRIdTracker for debugging +func (t *PiTRIdTracker) String() string { + if t == nil || t.DBIdToPhysicalId == nil { + return "PiTRIdTracker{nil}" + } + + var result strings.Builder + result.WriteString("PiTRIdTracker{\n") + for dbID, tables := range t.DBIdToPhysicalId { + result.WriteString(fmt.Sprintf(" DB[%d]: {", dbID)) + tableIDs := make([]int64, 0, len(tables)) + for tableID := range tables { + tableIDs = append(tableIDs, tableID) + } + // Sort for consistent output + sort.Slice(tableIDs, func(i, j int) bool { return tableIDs[i] < tableIDs[j] }) + for i, tableID := range tableIDs { + if i > 0 { + result.WriteString(", ") + } + result.WriteString(fmt.Sprintf("%d", tableID)) + } + result.WriteString("}\n") + } + result.WriteString("}") + return result.String() +} + +func MatchSchema(filter filter.Filter, schema string, withSys bool) bool { + if name, ok := StripTempTableNamePrefixIfNeeded(schema); IsSysDB(name) && ok { + // early return if system tables are disabled + if !withSys { + return false + } + schema = name + } + return filter.MatchSchema(schema) +} + +func MatchTable(filter filter.Filter, schema, table string, withSys bool) bool { + if name, ok := StripTempTableNamePrefixIfNeeded(schema); IsSysDB(name) && ok { + // early return if system tables are disabled + if !withSys { + return false + } + schema = name + } + return filter.MatchTable(schema, table) +} diff --git a/br/pkg/utils/filter_test.go b/br/pkg/utils/filter_test.go new file mode 100644 index 0000000000000..4998afcb5f653 --- /dev/null +++ b/br/pkg/utils/filter_test.go @@ -0,0 +1,53 @@ +package utils + +import ( + "testing" + + "github.com/stretchr/testify/require" +) + +func TestPiTRTableTracker(t *testing.T) { + t.Run("test new tracker", func(t *testing.T) { + tracker := NewPiTRTableTracker() + require.NotNil(t, tracker) + require.NotNil(t, tracker.DBIdToPhysicalId) + require.Empty(t, tracker.DBIdToPhysicalId) + }) + + t.Run("test update and contains table", func(t *testing.T) { + tracker := NewPiTRTableTracker() + + tracker.AddDB(1) + tracker.AddPhysicalId(1, 100) + tracker.AddDB(2) + require.True(t, tracker.ContainsDB(1)) + require.True(t, tracker.ContainsDB(2)) + require.True(t, tracker.ContainsPhysicalId(1, 100)) + require.False(t, tracker.ContainsPhysicalId(1, 101)) + require.False(t, tracker.ContainsPhysicalId(2, 100)) + + tracker.AddPhysicalId(1, 101) + tracker.AddPhysicalId(2, 200) + require.True(t, tracker.ContainsPhysicalId(1, 100)) + require.True(t, tracker.ContainsPhysicalId(1, 101)) + require.True(t, tracker.ContainsPhysicalId(2, 200)) + + tracker.AddPhysicalId(3, 300) + require.True(t, tracker.ContainsDB(3)) + require.True(t, tracker.ContainsPhysicalId(3, 300)) + }) + + t.Run("test remove table", func(t *testing.T) { + tracker := NewPiTRTableTracker() + + tracker.AddPhysicalId(1, 100) + tracker.AddPhysicalId(1, 101) + + require.True(t, tracker.Remove(1, 100)) + require.False(t, tracker.ContainsPhysicalId(1, 100)) + require.True(t, tracker.ContainsPhysicalId(1, 101)) + + require.False(t, tracker.Remove(1, 102)) + require.False(t, tracker.Remove(2, 100)) + }) +} diff --git a/br/pkg/utils/key.go b/br/pkg/utils/key.go index d8371a023fca7..4eceaf9ea2162 100644 --- a/br/pkg/utils/key.go +++ b/br/pkg/utils/key.go @@ -8,12 +8,15 @@ import ( "fmt" "io" "strings" + "time" "github.com/pingcap/errors" "github.com/pingcap/log" berrors "github.com/pingcap/tidb/br/pkg/errors" "github.com/pingcap/tidb/br/pkg/logutil" "github.com/pingcap/tidb/pkg/kv" + "github.com/pingcap/tidb/pkg/tablecodec" + "github.com/pingcap/tidb/pkg/util/codec" "go.uber.org/zap" ) @@ -190,3 +193,27 @@ func IntersectAll(s1 []kv.KeyRange, s2 []kv.KeyRange) []kv.KeyRange { } return rs } + +const DateFormat = "2006-01-02 15:04:05.999999999 -0700" + +func FormatDate(ts time.Time) string { + return ts.Format(DateFormat) +} + +func IsMetaDBKey(key []byte) bool { + return strings.HasPrefix(string(key), "mDB") +} + +func IsMetaDDLJobHistoryKey(key []byte) bool { + return strings.HasPrefix(string(key), "mDDLJobH") +} + +func IsDBOrDDLJobHistoryKey(key []byte) bool { + return strings.HasPrefix(string(key), "mD") +} + +func EncodeTxnMetaKey(key []byte, field []byte, ts uint64) []byte { + k := tablecodec.EncodeMetaKey(key, field) + txnKey := codec.EncodeBytes(nil, k) + return codec.EncodeUintDesc(txnKey, ts) +} diff --git a/br/pkg/utils/key_test.go b/br/pkg/utils/key_test.go index 28e24055443de..86dec2c546609 100644 --- a/br/pkg/utils/key_test.go +++ b/br/pkg/utils/key_test.go @@ -7,9 +7,11 @@ import ( "fmt" "slices" "testing" + "time" "github.com/pingcap/tidb/pkg/kv" "github.com/stretchr/testify/require" + "github.com/tikv/client-go/v2/oracle" ) func TestParseKey(t *testing.T) { @@ -177,3 +179,40 @@ func TestClampKeyRanges(t *testing.T) { }) } } + +func TestDateFormat(t *testing.T) { + cases := []struct { + ts uint64 + target string + }{ + { + 434604259287760897, + "2022-07-15 19:14:39.534 +0800", + }, + { + 434605479096221697, + "2022-07-15 20:32:12.734 +0800", + }, + { + 434605478903808000, + "2022-07-15 20:32:12 +0800", + }, + } + + timeZone, _ := time.LoadLocation("Asia/Shanghai") + for _, ca := range cases { + date := FormatDate(oracle.GetTimeFromTS(ca.ts).In(timeZone)) + require.Equal(t, ca.target, date) + } +} + +func TestPrefix(t *testing.T) { + require.True(t, IsMetaDBKey([]byte("mDBs"))) + require.False(t, IsMetaDBKey([]byte("mDDL"))) + require.True(t, IsMetaDDLJobHistoryKey([]byte("mDDLJobHistory"))) + require.False(t, IsMetaDDLJobHistoryKey([]byte("mDDL"))) + require.True(t, IsDBOrDDLJobHistoryKey([]byte("mDL"))) + require.True(t, IsDBOrDDLJobHistoryKey([]byte("mDB:"))) + require.True(t, IsDBOrDDLJobHistoryKey([]byte("mDDLHistory"))) + require.False(t, IsDBOrDDLJobHistoryKey([]byte("DDL"))) +} diff --git a/br/pkg/utils/misc.go b/br/pkg/utils/misc.go index a6a872696266d..7a58c30b13289 100644 --- a/br/pkg/utils/misc.go +++ b/br/pkg/utils/misc.go @@ -216,3 +216,23 @@ func StartExitSingleListener(ctx context.Context) (context.Context, context.Canc }() return cx, cancel } + +func Values[K comparable, V any](m map[K]V) []V { + values := make([]V, 0, len(m)) + for _, v := range m { + values = append(values, v) + } + return values +} + +func FlattenValues[K comparable, V any](m map[K][]V) []V { + total := 0 + for _, v := range m { + total += len(v) + } + result := make([]V, 0, total) + for _, v := range m { + result = append(result, v...) + } + return result +} diff --git a/br/pkg/utils/schema.go b/br/pkg/utils/schema.go index 7c01b1538338c..af065dbc95384 100644 --- a/br/pkg/utils/schema.go +++ b/br/pkg/utils/schema.go @@ -38,7 +38,9 @@ func IsTemplateSysDB(dbname ast.CIStr) bool { // IsSysDB tests whether the database is system DB. // Currently, both `mysql` and `sys` are system DB. -func IsSysDB(dbLowerName string) bool { +func IsSysDB(dbName string) bool { + // just in case + dbLowerName := strings.ToLower(dbName) return dbLowerName == mysql.SystemDB || dbLowerName == mysql.SysDB } @@ -47,12 +49,12 @@ func TemporaryDBName(db string) ast.CIStr { return ast.NewCIStr(temporaryDBNamePrefix + db) } -// GetSysDBName get the original name of system DB -func GetSysDBName(tempDB ast.CIStr) (string, bool) { - if ok := strings.HasPrefix(tempDB.O, temporaryDBNamePrefix); !ok { - return tempDB.O, false +// StripTempTableNamePrefixIfNeeded get the original name of system DB +func StripTempTableNamePrefixIfNeeded(tempDB string) (string, bool) { + if ok := strings.HasPrefix(tempDB, temporaryDBNamePrefix); !ok { + return tempDB, false } - return tempDB.O[len(temporaryDBNamePrefix):], true + return tempDB[len(temporaryDBNamePrefix):], true } // GetSysDBCIStrName get the CIStr name of system DB diff --git a/br/tests/br_pitr/run.sh b/br/tests/br_pitr/run.sh index 6ee6a70c0885c..adb4e29d6463a 100644 --- a/br/tests/br_pitr/run.sh +++ b/br/tests/br_pitr/run.sh @@ -102,7 +102,7 @@ check_result() { # start a new cluster restart_services_allowing_huge_index -# non-compliant operation +# non-compliant operation, need full backup specified for the first time PiTR echo "non compliant operation" restore_fail=0 run_br --pd $PD_ADDR restore point -s "local://$TEST_DIR/$PREFIX/log" --start-ts $current_ts || restore_fail=1 diff --git a/br/tests/br_pitr_table_filter/run.sh b/br/tests/br_pitr_table_filter/run.sh new file mode 100755 index 0000000000000..c03c086d1816f --- /dev/null +++ b/br/tests/br_pitr_table_filter/run.sh @@ -0,0 +1,534 @@ +#!/bin/sh +# +# Copyright 2024 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 -eux +DB="$TEST_NAME" +CUR=$(cd `dirname $0`; pwd) +TASK_NAME="pitr_table_filter" +. run_services + +# helper methods +create_tables_with_values() { + local prefix=$1 # table name prefix + local count=$2 # number of tables to create + + for i in $(seq 1 $count); do + run_sql "create table $DB.${prefix}_${i}(c int); insert into $DB.${prefix}_${i} values ($i);" + done +} + +verify_tables() { + local prefix=$1 # table name prefix + local count=$2 # number of tables to verify + local should_exist=$3 # true/false - whether tables should exist + + for i in $(seq 1 $count); do + if [ "$should_exist" = "true" ]; then + run_sql "select count(*) = 1 from $DB.${prefix}_${i} where c = $i" || { + echo "Table $DB.${prefix}_${i} doesn't have expected value $i" + exit 1 + } + else + if run_sql "select * from $DB.${prefix}_${i}" 2>/dev/null; then + echo "Table $DB.${prefix}_${i} exists but should not" + exit 1 + fi + fi + done +} + +rename_tables() { + local old_prefix=$1 # original table name prefix + local new_prefix=$2 # new table name prefix + local count=$3 # number of tables to rename + + for i in $(seq 1 $count); do + run_sql "rename table $DB.${old_prefix}_${i} to $DB.${new_prefix}_${i};" + done +} + +drop_tables() { + local prefix=$1 # table name prefix + local count=$2 # number of tables to drop + + for i in $(seq 1 $count); do + run_sql "drop table $DB.${prefix}_${i};" + done +} + +verify_other_db_tables() { + local should_exist=$1 # true/false - whether tables should exist + + if [ "$should_exist" = "true" ]; then + run_sql "select count(*) = 1 from ${DB}_other.test_table where c = 42" || { + echo "Table ${DB}_other.test_table doesn't have expected value 42" + exit 1 + } + else + if run_sql "select * from ${DB}_other.test_table" 2>/dev/null; then + echo "Table ${DB}_other.test_table exists but should not" + exit 1 + fi + fi +} + +test_basic_filter() { + restart_services || { echo "Failed to restart services"; exit 1; } + + echo "start basic filter testing" + run_br --pd $PD_ADDR log start --task-name $TASK_NAME -s "local://$TEST_DIR/$TASK_NAME/log" + + run_sql "create schema $DB;" + run_sql "create schema ${DB}_other;" + + echo "write initial data and do snapshot backup" + create_tables_with_values "full_backup" 3 + create_tables_with_values "table_to_drop" 3 + + run_br backup full -s "local://$TEST_DIR/$TASK_NAME/full" --pd $PD_ADDR + + echo "write more data and wait for log backup to catch up" + run_sql "create table ${DB}_other.test_table(c int); insert into ${DB}_other.test_table values (42);" + create_tables_with_values "log_backup_lower" 3 + create_tables_with_values "LOG_BACKUP_UPPER" 3 + create_tables_with_values "other" 3 + drop_tables "table_to_drop" 3 + + . "$CUR/../br_test_utils.sh" && wait_log_checkpoint_advance "$TASK_NAME" + + # restart services to clean up the cluster + restart_services || { echo "Failed to restart services"; exit 1; } + + echo "case 1 sanity check, zero filter" + run_br --pd "$PD_ADDR" restore point -s "local://$TEST_DIR/$TASK_NAME/log" --full-backup-storage "local://$TEST_DIR/$TASK_NAME/full" + + verify_tables "log_backup_lower" 3 true + verify_tables "LOG_BACKUP_UPPER" 3 true + verify_tables "full_backup" 3 true + verify_tables "other" 3 true + verify_tables "table_to_drop" 3 false + verify_other_db_tables true + + echo "case 2 with log restore table filter" + run_sql "drop schema $DB;" + run_sql "drop schema ${DB}_other;" + run_br --pd "$PD_ADDR" restore point -s "local://$TEST_DIR/$TASK_NAME/log" --full-backup-storage "local://$TEST_DIR/$TASK_NAME/full" -f "$DB.log*" + + verify_tables "log_backup_lower" 3 true + verify_tables "LOG_BACKUP_UPPER" 3 true + verify_tables "full_backup" 3 false + verify_tables "other" 3 false + verify_tables "table_to_drop" 3 false + verify_other_db_tables false + + echo "case 3 with multiple filters" + run_sql "drop schema $DB;" + run_br --pd "$PD_ADDR" restore point -s "local://$TEST_DIR/$TASK_NAME/log" --full-backup-storage "local://$TEST_DIR/$TASK_NAME/full" -f "$DB.log*" -f "$DB.full*" + + verify_tables "log_backup_lower" 3 true + verify_tables "LOG_BACKUP_UPPER" 3 true + verify_tables "full_backup" 3 true + verify_tables "other" 3 false + verify_tables "table_to_drop" 3 false + verify_other_db_tables false + + echo "case 4 with negative filters" + run_sql "drop schema $DB;" + # have to use a match all filter before using negative filters + run_br --pd "$PD_ADDR" restore point -s "local://$TEST_DIR/$TASK_NAME/log" --full-backup-storage "local://$TEST_DIR/$TASK_NAME/full" -f "*.*" -f "!mysql.*" -f "!$DB.log*" + + verify_tables "log_backup_lower" 3 false + verify_tables "LOG_BACKUP_UPPER" 3 false + verify_tables "full_backup" 3 true + verify_tables "other" 3 true + verify_tables "table_to_drop" 3 false + verify_other_db_tables true + + echo "case 5 restore dropped table" + run_sql "drop schema $DB;" + run_sql "drop schema ${DB}_other;" + run_br --pd "$PD_ADDR" restore point -s "local://$TEST_DIR/$TASK_NAME/log" --full-backup-storage "local://$TEST_DIR/$TASK_NAME/full" -f "$DB.table*" + + verify_tables "log_backup_lower" 3 false + verify_tables "LOG_BACKUP_UPPER" 3 false + verify_tables "full_backup" 3 false + verify_tables "other" 3 false + verify_tables "table_to_drop" 3 false + verify_other_db_tables false + + echo "case 6 restore only other database" + run_sql "drop schema $DB;" + run_br --pd "$PD_ADDR" restore point -s "local://$TEST_DIR/$TASK_NAME/log" --full-backup-storage "local://$TEST_DIR/$TASK_NAME/full" -f "${DB}_other.*" + + verify_tables "log_backup_lower" 3 false + verify_tables "LOG_BACKUP_UPPER" 3 false + verify_tables "full_backup" 3 false + verify_tables "other" 3 false + verify_tables "table_to_drop" 3 false + verify_other_db_tables true + + # cleanup + rm -rf "$TEST_DIR/$TASK_NAME" + + echo "basic filter test cases passed" +} + +test_with_full_backup_filter() { + restart_services || { echo "Failed to restart services"; exit 1; } + + echo "start with full backup filter testing" + run_br --pd $PD_ADDR log start --task-name $TASK_NAME -s "local://$TEST_DIR/$TASK_NAME/log" + + run_sql "create schema $DB;" + run_sql "create schema ${DB}_other;" + + echo "write initial data and do snapshot backup" + create_tables_with_values "full_backup" 3 + + run_br backup full -f "${DB}_other.*" -s "local://$TEST_DIR/$TASK_NAME/full" --pd $PD_ADDR + + echo "write more data and wait for log backup to catch up" + run_sql "create table ${DB}_other.test_table(c int); insert into ${DB}_other.test_table values (42);" + create_tables_with_values "log_backup" 3 + + . "$CUR/../br_test_utils.sh" && wait_log_checkpoint_advance "$TASK_NAME" + + # restart services to clean up the cluster + restart_services || { echo "Failed to restart services"; exit 1; } + + echo "case 1 sanity check, zero filter" + run_br --pd "$PD_ADDR" restore point -s "local://$TEST_DIR/$TASK_NAME/log" --full-backup-storage "local://$TEST_DIR/$TASK_NAME/full" + + verify_tables "log_backup" 3 false + verify_tables "full_backup" 3 false + verify_other_db_tables true + + echo "case 2 with log backup table same filter" + run_sql "drop schema ${DB}_other;" + run_br --pd "$PD_ADDR" restore point -s "local://$TEST_DIR/$TASK_NAME/log" --full-backup-storage "local://$TEST_DIR/$TASK_NAME/full" -f "${DB}_other.*" + + verify_tables "log_backup" 3 false + verify_tables "full_backup" 3 false + verify_other_db_tables true + + echo "case 3 with log backup filter include nothing" + run_sql "drop schema ${DB}_other;" + run_br --pd "$PD_ADDR" restore point -s "local://$TEST_DIR/$TASK_NAME/log" --full-backup-storage "local://$TEST_DIR/$TASK_NAME/full" -f "${DB}_nothing.*" + + verify_tables "log_backup" 3 false + verify_tables "full_backup" 3 false + verify_other_db_tables false + + # cleanup + rm -rf "$TEST_DIR/$TASK_NAME" + + echo "with full backup filter test cases passed" +} + +test_table_rename() { + restart_services || { echo "Failed to restart services"; exit 1; } + + echo "start table rename with filter testing" + run_br --pd $PD_ADDR log start --task-name $TASK_NAME -s "local://$TEST_DIR/$TASK_NAME/log" + + # create multiple schemas for cross-db rename testing + run_sql "create schema $DB;" + run_sql "create schema ${DB}_other1;" + run_sql "create schema ${DB}_other2;" + + echo "write initial data and do snapshot backup" + create_tables_with_values "full_backup" 3 + create_tables_with_values "renamed_in" 3 + create_tables_with_values "log_renamed_out" 3 + # add table for multiple rename test + run_sql "create table ${DB}_other1.multi_rename(c int); insert into ${DB}_other1.multi_rename values (42);" + + run_br backup full -s "local://$TEST_DIR/$TASK_NAME/full" --pd $PD_ADDR + + echo "write more data and wait for log backup to catch up" + create_tables_with_values "log_backup" 3 + rename_tables "full_backup" "full_backup_renamed" 3 + rename_tables "log_backup" "log_backup_renamed" 3 + rename_tables "renamed_in" "log_backup_renamed_in" 3 + rename_tables "log_renamed_out" "renamed_out" 3 + + # multiple renames across different databases + run_sql "rename table ${DB}_other1.multi_rename to ${DB}_other2.multi_rename;" + run_sql "rename table ${DB}_other2.multi_rename to $DB.log_multi_rename;" + + . "$CUR/../br_test_utils.sh" && wait_log_checkpoint_advance "$TASK_NAME" + + # restart services to clean up the cluster + restart_services || { echo "Failed to restart services"; exit 1; } + + run_br --pd "$PD_ADDR" restore point -s "local://$TEST_DIR/$TASK_NAME/log" --full-backup-storage "local://$TEST_DIR/$TASK_NAME/full" -f "$DB.log*" + + verify_tables "log_backup" 3 false + verify_tables "log_backup_renamed" 3 true + verify_tables "log_backup_renamed_in" 3 true + + verify_tables "full_backup" 3 false + # has been renamed, should not visible anymore + verify_tables "renamed_in" 3 false + # also renamed out of filter range, should not be visible for both + verify_tables "renamed_out" 3 false + verify_tables "log_renamed_out" 3 false + + # verify multi-renamed table + run_sql "select count(*) = 1 from $DB.log_multi_rename where c = 42" || { + echo "Table multi_rename doesn't have expected value after multiple renames" + exit 1 + } + + # Verify table doesn't exist in intermediate databases + if run_sql "select * from ${DB}_other1.multi_rename" 2>/dev/null; then + echo "Table exists in ${DB}_other1 but should not" + exit 1 + fi + if run_sql "select * from ${DB}_other2.multi_rename" 2>/dev/null; then + echo "Table exists in ${DB}_other2 but should not" + exit 1 + fi + + # cleanup + rm -rf "$TEST_DIR/$TASK_NAME" + + echo "table rename with filter passed" +} + +test_with_checkpoint() { + restart_services || { echo "Failed to restart services"; exit 1; } + + echo "start table filter with checkpoint" + run_br --pd $PD_ADDR log start --task-name $TASK_NAME -s "local://$TEST_DIR/$TASK_NAME/log" + + run_sql "create schema $DB;" + + echo "write initial data and do snapshot backup" + create_tables_with_values "full_backup" 3 + create_tables_with_values "renamed_in" 3 + create_tables_with_values "log_renamed_out" 3 + + run_br backup full -f "$DB.*" -s "local://$TEST_DIR/$TASK_NAME/full" --pd $PD_ADDR + + echo "write more data and wait for log backup to catch up" + create_tables_with_values "log_backup" 3 + rename_tables "renamed_in" "log_backup_renamed_in" 3 + rename_tables "log_renamed_out" "renamed_out" 3 + + . "$CUR/../br_test_utils.sh" && wait_log_checkpoint_advance "$TASK_NAME" + + # restart services to clean up the cluster + restart_services || { echo "Failed to restart services"; exit 1; } + + # Using single quotes to prevent shell interpretation + export GO_FAILPOINTS='github.com/pingcap/tidb/br/pkg/restore/snap_client/corrupt-files=return("corrupt-last-table-files")' + restore_fail=0 + run_br --pd $PD_ADDR restore point --full-backup-storage "local://$TEST_DIR/$TASK_NAME/full" -s "local://$TEST_DIR/$TASK_NAME/log" -f "$DB.log*" || restore_fail=1 + export GO_FAILPOINTS="" + if [ $restore_fail -ne 1 ]; then + echo 'expecting full backup last table corruption but success' + exit 1 + fi + + # PITR with checkpoint but failed in the log restore metakv stage + export GO_FAILPOINTS='github.com/pingcap/tidb/br/pkg/restore/snap_client/corrupt-files=return("only-last-table-files");github.com/pingcap/tidb/br/pkg/restore/log_client/failed-after-id-maps-saved=return(true)' + restore_fail=0 + run_br --pd $PD_ADDR restore point --full-backup-storage "local://$TEST_DIR/$TASK_NAME/full" -s "local://$TEST_DIR/$TASK_NAME/log" -f "$DB.log*" || restore_fail=1 + export GO_FAILPOINTS="" + if [ $restore_fail -ne 1 ]; then + echo 'expecting failed after id map saved but success' + exit 1 + fi + + # PITR with checkpoint but failed in the log restore datakv stage + # skip the snapshot restore stage + export GO_FAILPOINTS='github.com/pingcap/tidb/br/pkg/task/corrupt-files=return("corrupt-last-table-files")' + restore_fail=0 + run_br --pd $PD_ADDR restore point --full-backup-storage "local://$TEST_DIR/$TASK_NAME/full" -s "local://$TEST_DIR/$TASK_NAME/log" -f "$DB.log*" || restore_fail=1 + export GO_FAILPOINTS="" + if [ $restore_fail -ne 1 ]; then + echo 'expecting log restore last table corruption but success' + exit 1 + fi + + # PITR with checkpoint + export GO_FAILPOINTS='github.com/pingcap/tidb/br/pkg/task/corrupt-files=return("only-last-table-files")' + run_br --pd $PD_ADDR restore point --full-backup-storage "local://$TEST_DIR/$TASK_NAME/full" -s "local://$TEST_DIR/$TASK_NAME/log" -f "$DB.log*" + export GO_FAILPOINTS="" + + verify_tables "log_backup" 3 true + verify_tables "log_backup_renamed_in" 3 true + + verify_tables "full_backup" 3 false + # has been renamed, should not visible anymore + verify_tables "renamed_in" 3 false + # also renamed out of filter range, should not be visible for both + verify_tables "renamed_out" 3 false + verify_tables "log_renamed_out" 3 false + + # cleanup + rm -rf "$TEST_DIR/$TASK_NAME" + + echo "table filter checkpoint passed" +} + +test_exchange_partition() { + restart_services || { echo "Failed to restart services"; exit 1; } + + echo "start testing exchange partition with filter" + run_br --pd $PD_ADDR log start --task-name $TASK_NAME -s "local://$TEST_DIR/$TASK_NAME/log" + + run_sql "create schema $DB;" + + # create a partitioned table and a normal table for exchange + run_sql "CREATE TABLE $DB.full_partitioned ( + id INT, + value INT + ) PARTITION BY RANGE (id) ( + PARTITION p0 VALUES LESS THAN (100), + PARTITION p1 VALUES LESS THAN (200) + );" + + run_sql "CREATE TABLE $DB.log_table ( + id INT, + value INT + );" + + run_sql "INSERT INTO $DB.full_partitioned VALUES (50, 1), (150, 2);" + run_sql "INSERT INTO $DB.log_table VALUES (75, 3);" + + run_br backup full -f "$DB.*" -s "local://$TEST_DIR/$TASK_NAME/full" --pd $PD_ADDR + + # exchange partition and create some new tables with log_ prefix + run_sql "ALTER TABLE $DB.full_partitioned EXCHANGE PARTITION p0 WITH TABLE $DB.log_table;" + run_sql "CREATE TABLE $DB.log_after_exchange (id INT, value INT);" + run_sql "INSERT INTO $DB.log_after_exchange VALUES (1, 1);" + + . "$CUR/../br_test_utils.sh" && wait_log_checkpoint_advance "$TASK_NAME" + + restart_services || { echo "Failed to restart services"; exit 1; } + + # Test case 1: Restore with full* filter + echo "Testing restoration with full* filter" + run_br --pd "$PD_ADDR" restore point -s "local://$TEST_DIR/$TASK_NAME/log" --full-backup-storage "local://$TEST_DIR/$TASK_NAME/full" -f "$DB.full*" + + # Verify full_partitioned table has the exchanged data + run_sql "SELECT count(*) = 1 FROM $DB.full_partitioned WHERE id = 75 AND value = 3" || { + echo "full_partitioned doesn't have the exchanged partition data (75,3)" + exit 1 + } + run_sql "SELECT count(*) = 1 FROM $DB.full_partitioned WHERE id = 150 AND value = 2" || { + echo "full_partitioned missing original data (150,2)" + exit 1 + } + + # log_table and log_after_exchange should not exist with full* filter + if run_sql "SELECT * FROM $DB.log_table" 2>/dev/null; then + echo "log_table exists but should not with full* filter" + exit 1 + fi + if run_sql "SELECT * FROM $DB.log_after_exchange" 2>/dev/null; then + echo "log_after_exchange exists but should not with full* filter" + exit 1 + fi + + # Clean up for next test + run_sql "drop schema $DB;" + + # Test case 2: Restore with log* filter + echo "Testing restoration with log* filter" + run_br --pd "$PD_ADDR" restore point -s "local://$TEST_DIR/$TASK_NAME/log" --full-backup-storage "local://$TEST_DIR/$TASK_NAME/full" -f "$DB.log*" + + # Verify log tables exist with correct data + run_sql "SELECT count(*) = 1 FROM $DB.log_table WHERE id = 50 AND value = 1" || { + echo "log_table doesn't have the exchanged partition data (50,1)" + exit 1 + } + run_sql "SELECT count(*) = 1 FROM $DB.log_after_exchange WHERE id = 1 AND value = 1" || { + echo "log_after_exchange missing its data (1,1)" + exit 1 + } + + # full_partitioned should not exist with log* filter + if run_sql "SELECT * FROM $DB.full_partitioned" 2>/dev/null; then + echo "full_partitioned exists but should not with log* filter" + exit 1 + fi + + # cleanup + rm -rf "$TEST_DIR/$TASK_NAME" + + echo "exchange partition with filter test passed" +} + +test_system_tables() { + restart_services || { echo "Failed to restart services"; exit 1; } + + echo "start system tables testing" + run_br --pd $PD_ADDR log start --task-name $TASK_NAME -s "local://$TEST_DIR/$TASK_NAME/log" + + run_sql "create schema $DB;" + + echo "write initial data and do snapshot backup" + # create and populate a user table for reference + run_sql "create table $DB.user_table(id int primary key);" + run_sql "insert into $DB.user_table values (1);" + + # make some changes to system tables + run_sql "create user 'test_user'@'%' identified by 'password';" + run_sql "grant select on $DB.* to 'test_user'@'%';" + + run_br backup full -s "local://$TEST_DIR/$TASK_NAME/full" --pd $PD_ADDR + + echo "make more changes to system tables and wait for log backup" + run_sql "revoke select on $DB.* from 'test_user'@'%';" + run_sql "grant insert on $DB.* to 'test_user'@'%';" + run_sql "alter user 'test_user'@'%' identified by 'newpassword';" + + . "$CUR/../br_test_utils.sh" && wait_log_checkpoint_advance "$TASK_NAME" + + restart_services || { echo "Failed to restart services"; exit 1; } + + echo "PiTR should error out when system tables are included with explicit filter" + restore_fail=0 + run_br --pd "$PD_ADDR" restore point -f "*.*" -s "local://$TEST_DIR/$TASK_NAME/log" --full-backup-storage "local://$TEST_DIR/$TASK_NAME/full" || restore_fail=1 + if [ $restore_fail -ne 1 ]; then + echo "Expected restore to fail when including system tables with filter" + exit 1 + fi + + # Also verify that specific system table filters fail + restore_fail=0 + run_br --pd "$PD_ADDR" restore point -f "mysql.*" -s "local://$TEST_DIR/$TASK_NAME/log" --full-backup-storage "local://$TEST_DIR/$TASK_NAME/full" || restore_fail=1 + if [ $restore_fail -ne 1 ]; then + echo "Expected restore to fail when explicitly filtering system tables" + exit 1 + fi + + rm -rf "$TEST_DIR/$TASK_NAME" + echo "system tables test passed" +} + +# run all test cases +test_basic_filter +test_with_full_backup_filter +test_table_rename +test_with_checkpoint +test_exchange_partition +test_system_tables + +echo "br pitr table filter all tests passed" diff --git a/br/tests/config/tikv.toml b/br/tests/config/tikv.toml index 22126549ab848..a469b389989e7 100644 --- a/br/tests/config/tikv.toml +++ b/br/tests/config/tikv.toml @@ -36,6 +36,3 @@ path = "/tmp/backup_restore_test/master-key-file" [log-backup] max-flush-interval = "50s" -[gc] -ratio-threshold = 1.1 - diff --git a/br/tests/run_group_br_tests.sh b/br/tests/run_group_br_tests.sh index 0c9518f69fb97..99ff3693bf78c 100755 --- a/br/tests/run_group_br_tests.sh +++ b/br/tests/run_group_br_tests.sh @@ -28,7 +28,7 @@ groups=( ["G05"]='br_skip_checksum br_split_region_fail br_systables br_table_filter br_txn br_stats br_clustered_index br_crypter br_partition_add_index' ["G06"]='br_tikv_outage br_tikv_outage3 br_restore_checkpoint br_encryption' ["G07"]='br_pitr' - ["G08"]='br_tikv_outage2 br_ttl br_views_and_sequences br_z_gc_safepoint br_autorandom br_file_corruption br_tiflash_conflict' + ["G08"]='br_tikv_outage2 br_ttl br_views_and_sequences br_z_gc_safepoint br_autorandom br_file_corruption br_tiflash_conflict br_pitr_table_filter' ) # Get other cases not in groups, to avoid missing any case diff --git a/br/tests/utils.go b/br/tests/utils.go index e8653aaaabb43..2033c712a756c 100644 --- a/br/tests/utils.go +++ b/br/tests/utils.go @@ -75,41 +75,42 @@ func runValidateBackupFiles(cmd *cobra.Command, args []string) { // as full backup will have backup files ready in the storage path after returning from the command // and log backup will not, so we can only use restore point to validate. func parseCommand(cmd string) (string, bool) { - // Create a temporary cobra command to parse the input - tempCmd := &cobra.Command{} - tempCmd.Flags().String("s", "", "Storage path (short)") - tempCmd.Flags().String("storage", "", "Storage path (long)") - - // Split the command string into args + // not using cobra since it has to define all the possible flags otherwise will report parsing error args := strings.Fields(cmd) - // Parse the args - if err := tempCmd.Flags().Parse(args); err != nil { - return "", false - } - - // Check for backup or restore point command + // check for backup or restore point command hasBackupOrRestorePoint := false - for i, arg := range args { + storagePath := "" + + for i := 0; i < len(args); i++ { + arg := args[i] if arg == "backup" { hasBackupOrRestorePoint = true - break + continue } if i < len(args)-1 && arg == "restore" && args[i+1] == "point" { hasBackupOrRestorePoint = true - break + continue } - } - // Get the storage path from either -s or -storage flag - storagePath, _ := tempCmd.Flags().GetString("s") - if storagePath == "" { - storagePath, _ = tempCmd.Flags().GetString("storage") + // check for storage path in various formats + if arg == "-s" || arg == "--storage" { + if i+1 < len(args) { + storagePath = args[i+1] + i++ // skip the next arg since we consumed it + } + } else if strings.HasPrefix(arg, "--storage=") { + storagePath = strings.TrimPrefix(arg, "--storage=") + } else if strings.HasPrefix(arg, "-s=") { + storagePath = strings.TrimPrefix(arg, "-s=") + } } - storagePath = strings.TrimPrefix(storagePath, "local://") - if hasBackupOrRestorePoint && storagePath != "" { - return storagePath, true + if strings.HasPrefix(storagePath, "local://") { + storagePath = strings.TrimPrefix(storagePath, "local://") + if hasBackupOrRestorePoint && storagePath != "" { + return storagePath, true + } } return "", false }