From a9861dffc07a7f719deba47f1c96c6e5d321d24e Mon Sep 17 00:00:00 2001 From: hillium Date: Mon, 25 Nov 2024 16:51:32 +0800 Subject: [PATCH 01/36] initial commit Signed-off-by: hillium --- br/pkg/restore/log_client/client.go | 2 +- br/pkg/restore/restorer.go | 27 + br/pkg/restore/snap_client/client.go | 13 + br/pkg/restore/snap_client/pitr_collector.go | 255 ++++ br/pkg/restore/utils/rewrite_rule.go | 23 +- br/pkg/storage/local.go | 8 + br/pkg/storage/s3.go | 17 + br/pkg/storage/storage.go | 9 + br/pkg/stream/stream_metas.go | 67 +- br/pkg/stream/stream_metas_test.go | 14 +- br/pkg/task/operator/list_migration.go | 2 +- br/pkg/task/operator/migrate_to.go | 2 +- br/pkg/task/restore.go | 19 +- go.mod | 2 + go.sum | 1254 +++++++++++++++++- pkg/util/util.go | 7 + pkg/util/util_test.go | 16 + 17 files changed, 1706 insertions(+), 31 deletions(-) create mode 100644 br/pkg/restore/snap_client/pitr_collector.go diff --git a/br/pkg/restore/log_client/client.go b/br/pkg/restore/log_client/client.go index 4faf59a316657..8c3621eed6d2e 100644 --- a/br/pkg/restore/log_client/client.go +++ b/br/pkg/restore/log_client/client.go @@ -491,7 +491,7 @@ func (rc *LogClient) InitCheckpointMetadataForLogRestore( } func (rc *LogClient) GetMigrations(ctx context.Context) ([]*backuppb.Migration, error) { - ext := stream.MigerationExtension(rc.storage) + ext := stream.MigrationExtension(rc.storage) migs, err := ext.Load(ctx) if err != nil { return nil, errors.Trace(err) diff --git a/br/pkg/restore/restorer.go b/br/pkg/restore/restorer.go index 75a21b583eb1f..9f27df71f34e4 100644 --- a/br/pkg/restore/restorer.go +++ b/br/pkg/restore/restorer.go @@ -31,6 +31,7 @@ import ( "github.com/pingcap/tidb/br/pkg/summary" "github.com/pingcap/tidb/br/pkg/utils/iter" "github.com/pingcap/tidb/pkg/util" + "go.uber.org/multierr" "go.uber.org/zap" "go.uber.org/zap/zapcore" "golang.org/x/sync/errgroup" @@ -109,6 +110,32 @@ func NewFileSet(files []*backuppb.File, rules *utils.RewriteRules) BackupFileSet } } +// DualRestorer is a composite restorer that combines two SstRestorers into a single restorer. +type DualRestorer struct { + A SstRestorer + B SstRestorer +} + +// Dual creates a new DualRestorer from two SstRestorers. +func Dual(a, b SstRestorer) *DualRestorer { + return &DualRestorer{ + A: a, + B: b, + } +} + +func (c *DualRestorer) Close() error { + return multierr.Combine(c.A.Close(), c.B.Close()) +} + +func (c *DualRestorer) WaitUntilFinish() error { + return multierr.Combine(c.A.WaitUntilFinish(), c.B.WaitUntilFinish()) +} + +func (c *DualRestorer) GoRestore(onProgress func(int64), batchFileSets ...BatchBackupFileSet) error { + return multierr.Combine(c.A.GoRestore(onProgress, batchFileSets...), c.B.GoRestore(onProgress, batchFileSets...)) +} + // SstRestorer defines the essential methods required for restoring SST files in various backup formats: // 1. Raw backup SST files // 2. Transactional (Txn) backup SST files diff --git a/br/pkg/restore/snap_client/client.go b/br/pkg/restore/snap_client/client.go index a7e0ecab3d230..0bcc03c5d85c0 100644 --- a/br/pkg/restore/snap_client/client.go +++ b/br/pkg/restore/snap_client/client.go @@ -150,6 +150,8 @@ type SnapClient struct { // checkpoint information for snapshot restore checkpointRunner *checkpoint.CheckpointRunner[checkpoint.RestoreKeyType, checkpoint.RestoreValueType] checkpointChecksum map[int64]*checkpoint.ChecksumItem + + pitrColl *pitrCollector } // NewRestoreClient returns a new RestoreClient. @@ -165,6 +167,7 @@ func NewRestoreClient( tlsConf: tlsConf, keepaliveConf: keepaliveConf, switchCh: make(chan struct{}), + pitrColl: new(pitrCollector), } } @@ -413,6 +416,16 @@ func makeDBPool(size uint, dbFactory func() (*tidallocdb.DB, error)) ([]*tidallo return dbPool, nil } +func (rc *SnapClient) InstallPiTRSupport(ctx context.Context, deps PiTRCollDep) error { + collector, err := newPiTRColl(ctx, deps) + if err != nil { + return errors.Trace(err) + } + rc.pitrColl = collector + rc.restorer = restore.Dual(rc.pitrColl.createRestorer(ctx), rc.restorer) + return nil +} + // Init create db connection and domain for storage. func (rc *SnapClient) Init(g glue.Glue, store kv.Storage) error { // setDB must happen after set PolicyMode. diff --git a/br/pkg/restore/snap_client/pitr_collector.go b/br/pkg/restore/snap_client/pitr_collector.go new file mode 100644 index 0000000000000..8410f5111618e --- /dev/null +++ b/br/pkg/restore/snap_client/pitr_collector.go @@ -0,0 +1,255 @@ +package snapclient + +import ( + "context" + "fmt" + "path/filepath" + "sync" + + "github.com/pingcap/errors" + pb "github.com/pingcap/kvproto/pkg/brpb" + berrors "github.com/pingcap/tidb/br/pkg/errors" + "github.com/pingcap/tidb/br/pkg/restore" + "github.com/pingcap/tidb/br/pkg/storage" + "github.com/pingcap/tidb/br/pkg/stream" + "github.com/pingcap/tidb/br/pkg/streamhelper" + "github.com/pingcap/tidb/pkg/util" + "github.com/tikv/client-go/v2/oracle" + pd "github.com/tikv/pd/client" + clientv3 "go.etcd.io/etcd/client/v3" + "golang.org/x/sync/errgroup" +) + +type pitrCollectorRestorer struct { + restore.SstRestorer + // the context used for committing. + cx context.Context + // the context bound to the errgroup. + ecx context.Context + + coll *pitrCollector + wg *errgroup.Group +} + +// wrapRestorer wraps a restorer and the restorer will upload the SST file to the collector during restoring. +func (c *pitrCollector) createRestorer(ctx context.Context) *pitrCollectorRestorer { + wg, ecx := errgroup.WithContext(ctx) + return &pitrCollectorRestorer{ + cx: ctx, + ecx: ecx, + coll: c, + wg: wg, + } +} + +// GoRestore imports the specified backup file sets into TiKV asynchronously. +// The onProgress function is called with progress updates as files are processed. +func (p pitrCollectorRestorer) GoRestore(onProgress func(int64), batchFileSets ...restore.BatchBackupFileSet) error { + p.wg.Go(func() error { + for _, fileSets := range batchFileSets { + for _, fileSet := range fileSets { + for _, file := range fileSet.SSTFiles { + if err := p.coll.PutSST(p.ecx, file); err != nil { + return errors.Annotatef(err, "failed to put sst %s", file.GetName()) + } + } + for _, hint := range fileSet.RewriteRules.TableIDRemapHint { + if err := p.coll.PutRewriteRule(p.ecx, hint.Origin, hint.Rewritten); err != nil { + return errors.Annotatef(err, "failed to put rewrite rule of %v", fileSet.RewriteRules) + } + } + } + } + return nil + }) + return nil +} + +// WaitUntilFinish blocks until all pending restore files have completed processing. +func (p pitrCollectorRestorer) WaitUntilFinish() error { + return errors.Annotate(p.wg.Wait(), "failed to wait on wait pitrCollector") +} + +// Close releases any resources associated with the restoration process. +func (p pitrCollectorRestorer) Close() error { + return errors.Annotate(p.coll.Commit(p.cx), "failed to commit pitrCollector") +} + +type pitrCollector struct { + // Immutable state. + taskStorage storage.ExternalStorage + restoreStorage storage.ExternalStorage + name string + enabled bool + + // Mutable state. + committing committing + committingLock sync.Mutex + + // Delegates. + tso func(ctx context.Context) (uint64, error) +} + +type committing struct { + msg pb.ExtraFullBackup + rewrites map[int64]int64 +} + +func (c *committing) genMsg() *pb.ExtraFullBackup { + msg := util.ProtoV1Clone(&c.msg) + for old, new := range c.rewrites { + msg.RewrittenTables = append(msg.RewrittenTables, &pb.RewrittenTableID{UpstreamOfUpstream: old, Upstream: new}) + } + return msg +} + +// doWithCommittingLock edits the committing ExtraFullBackup. +func (c *pitrCollector) doWithCommittingLock(f func()) { + c.committingLock.Lock() + f() + c.committingLock.Unlock() +} + +// outputPath constructs the path by a relative path for outputting. +func (c *pitrCollector) outputPath(segs ...string) string { + return filepath.Join(append([]string{"v1", "ext_backups", c.name}, segs...)...) +} + +// PutSST records an SST file. +func (c *pitrCollector) PutSST(ctx context.Context, f *pb.File) error { + if !c.enabled { + return nil + } + + f = util.ProtoV1Clone(f) + out := c.outputPath(f.GetName()) + + copier, ok := c.taskStorage.(storage.Copier) + if !ok { + return errors.Annotatef(berrors.ErrInvalidArgument, "storage %T does not support copying", c.taskStorage) + } + spec := storage.CopySpec{ + From: f.GetName(), + To: out, + } + if err := copier.CopyFrom(ctx, c.restoreStorage, spec); err != nil { + return err + } + + f.Name = out + c.doWithCommittingLock(func() { c.committing.msg.Files = append(c.committing.msg.Files, f) }) + return nil +} + +// PutRewriteRule records a rewrite rule. +func (c *pitrCollector) PutRewriteRule(_ context.Context, oldID int64, newID int64) error { + if !c.enabled { + return nil + } + var err error + c.doWithCommittingLock(func() { + if oldVal, ok := c.committing.rewrites[oldID]; ok && oldVal != newID { + err = errors.Annotatef( + berrors.ErrInvalidArgument, + "pitr coll rewrite rule conflict: we had %v -> %v, but you want rewrite to %v", + oldID, + oldVal, + newID, + ) + return + } + c.committing.rewrites[oldID] = newID + }) + return err +} + +// Commit commits the collected SSTs to a migration. +func (c *pitrCollector) Commit(ctx context.Context) error { + if !c.enabled { + return nil + } + + est := stream.MigrationExtension(c.taskStorage) + m := stream.NewMigration() + var msg *pb.ExtraFullBackup + tso, err := c.tso(ctx) + if err != nil { + return errors.Trace(err) + } + + c.doWithCommittingLock(func() { + msg = c.committing.genMsg() + c.committing.msg.AsIfTs = tso + }) + m.ExtraFullBackups = append(m.ExtraFullBackups, msg) + + _, err = est.AppendMigration(ctx, m) + if err != nil { + return errors.Trace(err) + } + + c.doWithCommittingLock(func() { + c.resetCommitting() + }) + return nil +} + +func (c *pitrCollector) resetCommitting() { + c.committing = committing{ + rewrites: map[int64]int64{}, + } + c.committing.msg.FilesPrefixHint = c.outputPath() +} + +// PiTRCollDep is the dependencies of a PiTR collector. +type PiTRCollDep struct { + PDCli pd.Client + EtcdCli *clientv3.Client + Storage *pb.StorageBackend +} + +// newPiTRColl creates a new PiTR collector. +func newPiTRColl(ctx context.Context, deps PiTRCollDep) (*pitrCollector, error) { + mcli := streamhelper.NewMetaDataClient(deps.EtcdCli) + ts, err := mcli.GetAllTasks(ctx) + if err != nil { + return nil, errors.Trace(err) + } + if len(ts) > 1 { + return nil, errors.Annotatef(berrors.ErrInvalidArgument, "more than one task found, pitr collector doesn't support that") + } + if len(ts) == 0 { + return &pitrCollector{}, nil + } + + coll := &pitrCollector{ + enabled: true, + } + + strg, err := storage.Create(ctx, ts[0].Info.Storage, false) + if err != nil { + return nil, errors.Trace(err) + } + coll.taskStorage = strg + + tso := func(ctx context.Context) (uint64, error) { + l, o, err := deps.PDCli.GetTS(ctx) + return oracle.ComposeTS(l, o), err + } + coll.tso = tso + + t, err := tso(ctx) + if err != nil { + return nil, errors.Trace(err) + } + coll.name = fmt.Sprintf("backup-%016X", t) + + restoreStrg, err := storage.Create(ctx, deps.Storage, false) + if err != nil { + return nil, errors.Trace(err) + } + coll.restoreStorage = restoreStrg + + coll.resetCommitting() + return coll, nil +} diff --git a/br/pkg/restore/utils/rewrite_rule.go b/br/pkg/restore/utils/rewrite_rule.go index a664d97a5f11d..9afa639b8ca8b 100644 --- a/br/pkg/restore/utils/rewrite_rule.go +++ b/br/pkg/restore/utils/rewrite_rule.go @@ -47,6 +47,15 @@ type RewriteRules struct { NewKeyspace []byte // used to record checkpoint data NewTableID int64 + // used to record backup files to pitr. + // note: should NewTableID merged with this? + TableIDRemapHint []TableIDRemap +} + +// TableIDRemap presents a remapping of table id during rewriting. +type TableIDRemap struct { + Origin int64 + Rewritten int64 } // Append append its argument to this rewrite rules. @@ -75,9 +84,11 @@ func GetRewriteRules( ) *RewriteRules { tableIDs := GetTableIDMap(newTable, oldTable) indexIDs := GetIndexIDMap(newTable, oldTable) + remaps := make([]TableIDRemap, 0) dataRules := make([]*import_sstpb.RewriteRule, 0) for oldTableID, newTableID := range tableIDs { + remaps = append(remaps, TableIDRemap{Origin: oldTableID, Rewritten: newTableID}) if getDetailRule { dataRules = append(dataRules, &import_sstpb.RewriteRule{ OldKeyPrefix: tablecodec.GenTableRecordPrefix(oldTableID), @@ -101,7 +112,8 @@ func GetRewriteRules( } return &RewriteRules{ - Data: dataRules, + Data: dataRules, + TableIDRemapHint: remaps, } } @@ -112,8 +124,10 @@ func GetRewriteRulesMap( tableIDs := GetTableIDMap(newTable, oldTable) indexIDs := GetIndexIDMap(newTable, oldTable) + remaps := make([]TableIDRemap, 0) for oldTableID, newTableID := range tableIDs { + remaps = append(remaps, TableIDRemap{Origin: oldTableID, Rewritten: newTableID}) dataRules := make([]*import_sstpb.RewriteRule, 0) if getDetailRule { dataRules = append(dataRules, &import_sstpb.RewriteRule{ @@ -137,7 +151,8 @@ func GetRewriteRulesMap( } rules[oldTableID] = &RewriteRules{ - Data: dataRules, + Data: dataRules, + TableIDRemapHint: remaps, } } @@ -152,7 +167,7 @@ func GetRewriteRuleOfTable( getDetailRule bool, ) *RewriteRules { dataRules := make([]*import_sstpb.RewriteRule, 0) - + remaps := []TableIDRemap{{Origin: oldTableID, Rewritten: newTableID}} if getDetailRule { dataRules = append(dataRules, &import_sstpb.RewriteRule{ OldKeyPrefix: tablecodec.GenTableRecordPrefix(oldTableID), @@ -174,7 +189,7 @@ func GetRewriteRuleOfTable( }) } - return &RewriteRules{Data: dataRules, NewTableID: newTableID} + return &RewriteRules{Data: dataRules, NewTableID: newTableID, TableIDRemapHint: remaps} } // ValidateFileRewriteRule uses rewrite rules to validate the ranges of a file. diff --git a/br/pkg/storage/local.go b/br/pkg/storage/local.go index b825c79e90381..35f6a94cd8cbe 100644 --- a/br/pkg/storage/local.go +++ b/br/pkg/storage/local.go @@ -258,6 +258,14 @@ func (l *LocalStorage) Rename(_ context.Context, oldFileName, newFileName string // Close implements ExternalStorage interface. func (*LocalStorage) Close() {} +func (l *LocalStorage) CopyFrom(ctx context.Context, e ExternalStorage, spec CopySpec) error { + bs, err := e.ReadFile(ctx, spec.From) + if err != nil { + return errors.Trace(err) + } + return l.WriteFile(ctx, spec.To, bs) +} + func pathExists(_path string) (bool, error) { _, err := os.Stat(_path) if err != nil { diff --git a/br/pkg/storage/s3.go b/br/pkg/storage/s3.go index 3987512b2a0a2..0158979411f1a 100644 --- a/br/pkg/storage/s3.go +++ b/br/pkg/storage/s3.go @@ -93,6 +93,23 @@ func (rs *S3Storage) GetOptions() *backuppb.S3 { return rs.options } +func (rs *S3Storage) CopyFrom(ctx context.Context, e ExternalStorage, spec CopySpec) error { + s, ok := e.(*S3Storage) + if !ok { + return errors.Annotatef(berrors.ErrStorageInvalidConfig, "S3Storage.CopyFrom supports S3 storage only, get %T", e) + } + + copyInput := &s3.CopyObjectInput{ + Bucket: aws.String(rs.options.Bucket), + // NOTE: Perhaps we need to allow copy cross regions / accounts. + CopySource: aws.String(path.Join(s.options.Bucket, s.options.Prefix, spec.From)), + Key: aws.String(rs.options.Prefix + spec.To), + } + + _, err := s.svc.CopyObjectWithContext(ctx, copyInput) + return err +} + // S3Uploader does multi-part upload to s3. type S3Uploader struct { svc s3iface.S3API diff --git a/br/pkg/storage/storage.go b/br/pkg/storage/storage.go index 042d4a1f4d715..82857a98bdcd8 100644 --- a/br/pkg/storage/storage.go +++ b/br/pkg/storage/storage.go @@ -101,6 +101,15 @@ type ReaderOption struct { PrefetchSize int } +type Copier interface { + CopyFrom(ctx context.Context, e ExternalStorage, spec CopySpec) error +} + +type CopySpec struct { + From string + To string +} + // ExternalStorage represents a kind of file system storage. type ExternalStorage interface { // WriteFile writes a complete file to storage, similar to os.WriteFile, but WriteFile should be atomic diff --git a/br/pkg/stream/stream_metas.go b/br/pkg/stream/stream_metas.go index bfbebafebd7e4..ec5514fd88a76 100644 --- a/br/pkg/stream/stream_metas.go +++ b/br/pkg/stream/stream_metas.go @@ -39,13 +39,14 @@ const ( baseTmp = "BASE_TMP" metaSuffix = ".meta" migrationPrefix = "v1/migrations" + lockPrefix = "v1/LOCK" - SupportedMigVersion = pb.MigrationVersion_M1 + SupportedMigVersion = pb.MigrationVersion_M2 ) func NewMigration() *pb.Migration { return &pb.Migration{ - Version: pb.MigrationVersion_M1, + Version: pb.MigrationVersion_M2, Creator: fmt.Sprintf("br;commit=%s;branch=%s", versioninfo.TiDBGitHash, versioninfo.TiDBGitBranch), } } @@ -204,7 +205,7 @@ func (ms *StreamMetadataSet) RemoveDataFilesAndUpdateMetadataInBatch( updateFn func(num int64), ) ([]string, error) { hst := ms.hook(st) - est := MigerationExtension(hst) + est := MigrationExtension(hst) est.Hooks = updateFnHook{updateFn: updateFn} res := MigratedTo{NewBase: NewMigration()} est.doTruncateLogs(ctx, ms, from, &res) @@ -516,7 +517,7 @@ func (NoHooks) HandledAMetaEdit(*pb.MetaEdit) func (NoHooks) HandingMetaEditDone() {} // MigrateionExtnsion installs the extension methods to an `ExternalStorage`. -func MigerationExtension(s storage.ExternalStorage) MigrationExt { +func MigrationExtension(s storage.ExternalStorage) MigrationExt { return MigrationExt{ s: s, prefix: migrationPrefix, @@ -534,6 +535,7 @@ func MergeMigrations(m1 *pb.Migration, m2 *pb.Migration) *pb.Migration { out.TruncatedTo = max(m1.GetTruncatedTo(), m2.GetTruncatedTo()) out.DestructPrefix = append(out.DestructPrefix, m1.GetDestructPrefix()...) out.DestructPrefix = append(out.DestructPrefix, m2.GetDestructPrefix()...) + out.ExtraFullBackups = append(out.ExtraFullBackups, m1.GetExtraFullBackups()...) return out } @@ -656,11 +658,20 @@ func (m MigrationExt) DryRun(f func(MigrationExt)) []storage.Effect { } func (m MigrationExt) AppendMigration(ctx context.Context, mig *pb.Migration) (int, error) { + lock, err := storage.TryLockRemoteWrite(ctx, m.s, lockPrefix, "AppendMigration") + if err != nil { + return 0, err + } + defer lock.Unlock(ctx) + migs, err := m.Load(ctx) if err != nil { return 0, err } - newSN := migs.Layers[len(migs.Layers)-1].SeqNum + 1 + newSN := 1 + if len(migs.Layers) > 0 { + newSN = migs.Layers[len(migs.Layers)-1].SeqNum + 1 + } name := path.Join(migrationPrefix, nameOf(mig, newSN)) data, err := mig.Marshal() if err != nil { @@ -738,6 +749,16 @@ func (m MigrationExt) MergeAndMigrateTo( targetSpec int, opts ...MergeAndMigrateToOpt, ) (result MergeAndMigratedTo) { + lock, err := storage.TryLockRemoteWrite(ctx, m.s, lockPrefix, "AppendMigration") + if err != nil { + result.MigratedTo = MigratedTo{ + Warnings: []error{ + errors.Annotate(err, "failed to get the lock, nothing will happen"), + }} + return + } + defer lock.Unlock(ctx) + config := mergeAndMigrateToConfig{} for _, o := range opts { o(&config) @@ -1064,6 +1085,16 @@ func (m MigrationExt) doTruncating(ctx context.Context, mig *pb.Migration, resul m.tryRemovePrefix(ctx, pfx, result) } + for _, extraBackup := range mig.ExtraFullBackups { + if extraBackup.AsIfTs > mig.TruncatedTo { + result.NewBase.ExtraFullBackups = append(result.NewBase.ExtraFullBackups, extraBackup) + } else { + for _, pfx := range extraBackup.Files { + m.tryRemovePrefix(ctx, pfx.Name, result) + } + } + } + result.NewBase.TruncatedTo = mig.TruncatedTo m.Hooks.StartLoadingMetaForTruncating() @@ -1320,6 +1351,21 @@ func isEmptyMetadata(md *pb.Metadata) bool { return len(md.FileGroups) == 0 && len(md.Files) == 0 } +/* Below are hash algorithms for hashing a component of the migration. + * Sadly there isn't a document describes the behavior of the algorithms. + * Perhaps we can standardlize them in the future. + * Maybe by defining a ordering-insensitive object hash algorithm for protocol buffer. + * + * Note: For now, the canon of the hash algorithm for a message should follow the following rules: + * - If a hash algorithm for a message exists both in TiKV and BR and conflicting, we + * follow the implementation at where the message firstly creates (say, for compactions, + * TiKV will be the canonical implementation. while for extra full backups, BR is canonical.). + * - For commonly used fields, follow the implementation in BR. + * + * Another note: nowadays, the hash of a migration is mainly used for detecting duplicated works, + * so the difference between hash algorithms won't result in something too bad... + */ + func hashMigration(m *pb.Migration) uint64 { var crc64 uint64 = 0 for _, compaction := range m.Compactions { @@ -1328,9 +1374,20 @@ func hashMigration(m *pb.Migration) uint64 { for _, metaEdit := range m.EditMeta { crc64 ^= hashMetaEdit(metaEdit) } + for _, extBkup := range m.ExtraFullBackups { + crc64 ^= hashExtraBackup(extBkup) + } return crc64 ^ m.TruncatedTo } +func hashExtraBackup(extBkup *pb.ExtraFullBackup) uint64 { + bs, err := extBkup.Marshal() + if err != nil { + panic(fmt.Sprintf("failed to marshal message, this shouldn't happen: %s", err)) + } + return crc64.Checksum(bs, crc64.MakeTable(crc64.ISO)) +} + func hashMetaEdit(metaEdit *pb.MetaEdit) uint64 { var res uint64 = 0 for _, df := range metaEdit.DeletePhysicalFiles { diff --git a/br/pkg/stream/stream_metas_test.go b/br/pkg/stream/stream_metas_test.go index b292a3f8a9025..95dd645ec3d52 100644 --- a/br/pkg/stream/stream_metas_test.go +++ b/br/pkg/stream/stream_metas_test.go @@ -2566,7 +2566,7 @@ func TestBasicMigration(t *testing.T) { ) bs := storage.Batch(s) - est := MigerationExtension(bs) + est := MigrationExtension(bs) res := MergeMigrations(mig1, mig2) resE := mig( @@ -2632,7 +2632,7 @@ func TestMergeAndMigrateTo(t *testing.T) { mig3p := pmig(s, 3, mig3) bs := storage.Batch(s) - est := MigerationExtension(bs) + est := MigrationExtension(bs) ctx := context.Background() migs, err := est.Load(ctx) @@ -2700,7 +2700,7 @@ func TestRemoveCompaction(t *testing.T) { mTruncatedTo(20), ) bs := storage.Batch(s) - est := MigerationExtension(bs) + est := MigrationExtension(bs) merged := MergeMigrations(mig1, mig2) requireMigrationsEqual(t, merged, mig( @@ -2739,7 +2739,7 @@ func TestRetry(t *testing.T) { require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/br/pkg/storage/local_write_file_err", `1*return("this disk remembers nothing")`)) ctx := context.Background() - est := MigerationExtension(s) + est := MigrationExtension(s) mg := est.MergeAndMigrateTo(ctx, 2) require.Len(t, mg.Warnings, 1) require.Error(t, mg.Warnings[0], "this disk remembers nothing") @@ -2770,7 +2770,7 @@ func TestRetryRemoveCompaction(t *testing.T) { ) require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/br/pkg/storage/local_delete_file_err", `1*return("this disk will never forget")`)) - est := MigerationExtension(s) + est := MigrationExtension(s) mg := est.MigrateTo(ctx, mig1) require.Len(t, mg.Warnings, 1) require.Error(t, mg.Warnings[0], "this disk will never forget") @@ -2814,7 +2814,7 @@ func TestWithSimpleTruncate(t *testing.T) { }, })) - est := MigerationExtension(s) + est := MigrationExtension(s) m := mig(mTruncatedTo(65)) var res MigratedTo effs := est.DryRun(func(me MigrationExt) { res = me.MigrateTo(ctx, m) }) @@ -2842,7 +2842,7 @@ func TestUnsupportedVersion(t *testing.T) { m := mig(mVersion(backuppb.MigrationVersion(65535))) pmig(s, 1, m) - est := MigerationExtension(s) + est := MigrationExtension(s) ctx := context.Background() _, err := est.Load(ctx) require.Error(t, err) diff --git a/br/pkg/task/operator/list_migration.go b/br/pkg/task/operator/list_migration.go index d6c7efd57197a..591638656ddbf 100644 --- a/br/pkg/task/operator/list_migration.go +++ b/br/pkg/task/operator/list_migration.go @@ -26,7 +26,7 @@ func RunListMigrations(ctx context.Context, cfg ListMigrationConfig) error { if err != nil { return err } - ext := stream.MigerationExtension(st) + ext := stream.MigrationExtension(st) migs, err := ext.Load(ctx) if err != nil { return err diff --git a/br/pkg/task/operator/migrate_to.go b/br/pkg/task/operator/migrate_to.go index 20f76b0f86967..c00d5bbccfb28 100644 --- a/br/pkg/task/operator/migrate_to.go +++ b/br/pkg/task/operator/migrate_to.go @@ -90,7 +90,7 @@ func RunMigrateTo(ctx context.Context, cfg MigrateToConfig) error { console := glue.ConsoleOperations{ConsoleGlue: glue.StdIOGlue{}} - est := stream.MigerationExtension(st) + est := stream.MigrationExtension(st) est.Hooks = stream.NewProgressBarHooks(console) migs, err := est.Load(ctx) if err != nil { diff --git a/br/pkg/task/restore.go b/br/pkg/task/restore.go index 90f85cf15a7c7..bf66bd2d6234c 100644 --- a/br/pkg/task/restore.go +++ b/br/pkg/task/restore.go @@ -39,6 +39,7 @@ import ( "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta/model" pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/util" "github.com/pingcap/tidb/pkg/util/collate" "github.com/pingcap/tidb/pkg/util/engine" "github.com/spf13/cobra" @@ -676,9 +677,9 @@ func RunRestore(c context.Context, g glue.Glue, cmdName string, cfg *RestoreConf log.Error("failed to close the etcd client", zap.Error(err)) } }() - if err := checkTaskExists(c, cfg, etcdCLI); err != nil { - return errors.Annotate(err, "failed to check task exists") - } + // if err := checkTaskExists(c, cfg, etcdCLI); err != nil { + // return errors.Annotate(err, "failed to check task exists") + // } closeF, err := registerTaskToPD(c, etcdCLI) if err != nil { return errors.Annotate(err, "failed to register task to pd") @@ -782,14 +783,15 @@ func runSnapshotRestore(c context.Context, mgr *conn.Mgr, g glue.Glue, cmdName s // Init DB connection sessions err = client.Init(g, mgr.GetStorage()) defer client.Close() - if err != nil { return errors.Trace(err) } + u, s, backupMeta, err := ReadBackupMeta(ctx, metautil.MetaFile, &cfg.Config) if err != nil { return errors.Trace(err) } + if cfg.CheckRequirements { err := checkIncompatibleChangefeed(ctx, backupMeta.EndVersion, mgr.GetDomain().GetEtcdClient()) log.Info("Checking incompatible TiCDC changefeeds before restoring.", @@ -814,6 +816,15 @@ func runSnapshotRestore(c context.Context, mgr *conn.Mgr, g glue.Glue, cmdName s return errors.Trace(err) } + err = client.InstallPiTRSupport(ctx, snapclient.PiTRCollDep{ + PDCli: mgr.GetPDClient(), + EtcdCli: mgr.GetDomain().GetEtcdClient(), + Storage: util.ProtoV1Clone(u), + }) + if err != nil { + return errors.Trace(err) + } + if client.IsRawKvMode() { return errors.Annotate(berrors.ErrRestoreModeMismatch, "cannot do transactional restore from raw kv data") } diff --git a/go.mod b/go.mod index 278aa736baca2..66e2baa3d01d2 100644 --- a/go.mod +++ b/go.mod @@ -327,3 +327,5 @@ replace ( sourcegraph.com/sourcegraph/appdash => github.com/sourcegraph/appdash v0.0.0-20190731080439-ebfcffb1b5c0 sourcegraph.com/sourcegraph/appdash-data => github.com/sourcegraph/appdash-data v0.0.0-20151005221446-73f23eafcf67 ) + +replace github.com/pingcap/kvproto => github.com/yujuncen/kvproto v0.0.0-20241125070101-be2e12233a90 diff --git a/go.sum b/go.sum index 1d79f276736c5..0d69cdbee952a 100644 --- a/go.sum +++ b/go.sum @@ -3,6 +3,7 @@ cloud.google.com/go v0.34.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMT cloud.google.com/go v0.38.0/go.mod h1:990N+gfupTy94rShfmMCWGDn0LpTmnzTp2qbd1dvSRU= cloud.google.com/go v0.44.1/go.mod h1:iSa0KzasP4Uvy3f1mN/7PiObzGgflwredwwASm/v6AU= cloud.google.com/go v0.44.2/go.mod h1:60680Gw3Yr4ikxnPRS/oxxkBccT6SA1yMk63TGekxKY= +cloud.google.com/go v0.44.3/go.mod h1:60680Gw3Yr4ikxnPRS/oxxkBccT6SA1yMk63TGekxKY= cloud.google.com/go v0.45.1/go.mod h1:RpBamKRgapWJb87xiFSdk4g1CME7QZg3uwTez+TSTjc= cloud.google.com/go v0.46.3/go.mod h1:a6bKKbmY7er1mI7TEI4lsAkts/mkhTSZK8w33B4RAg0= cloud.google.com/go v0.50.0/go.mod h1:r9sluTvynVuxRIOHXQEHMFffphuXHOMZMycpNR5e6To= @@ -13,35 +14,810 @@ cloud.google.com/go v0.56.0/go.mod h1:jr7tqZxxKOVYizybht9+26Z/gUq7tiRzu+ACVAMbKV cloud.google.com/go v0.57.0/go.mod h1:oXiQ6Rzq3RAkkY7N6t3TcE6jE+CIBBbA36lwQ1JyzZs= cloud.google.com/go v0.62.0/go.mod h1:jmCYTdRCQuc1PHIIJ/maLInMho30T/Y0M4hTdTShOYc= cloud.google.com/go v0.65.0/go.mod h1:O5N8zS7uWy9vkA9vayVHs65eM1ubvY4h553ofrNHObY= +cloud.google.com/go v0.72.0/go.mod h1:M+5Vjvlc2wnp6tjzE102Dw08nGShTscUx2nZMufOKPI= +cloud.google.com/go v0.74.0/go.mod h1:VV1xSbzvo+9QJOxLDaJfTjx5e+MePCpCWwvftOeQmWk= +cloud.google.com/go v0.75.0/go.mod h1:VGuuCn7PG0dwsd5XPVm2Mm3wlh3EL55/79EKB6hlPTY= +cloud.google.com/go v0.78.0/go.mod h1:QjdrLG0uq+YwhjoVOLsS1t7TW8fs36kLs4XO5R5ECHg= +cloud.google.com/go v0.79.0/go.mod h1:3bzgcEeQlzbuEAYu4mrWhKqWjmpprinYgKJLgKHnbb8= +cloud.google.com/go v0.81.0/go.mod h1:mk/AM35KwGk/Nm2YSeZbxXdrNK3KZOYHmLkOqC2V6E0= +cloud.google.com/go v0.83.0/go.mod h1:Z7MJUsANfY0pYPdw0lbnivPx4/vhy/e2FEkSkF7vAVY= +cloud.google.com/go v0.84.0/go.mod h1:RazrYuxIK6Kb7YrzzhPoLmCVzl7Sup4NrbKPg8KHSUM= +cloud.google.com/go v0.87.0/go.mod h1:TpDYlFy7vuLzZMMZ+B6iRiELaY7z/gJPaqbMx6mlWcY= +cloud.google.com/go v0.90.0/go.mod h1:kRX0mNRHe0e2rC6oNakvwQqzyDmg57xJ+SZU1eT2aDQ= +cloud.google.com/go v0.93.3/go.mod h1:8utlLll2EF5XMAV15woO4lSbWQlk8rer9aLOfLh7+YI= +cloud.google.com/go v0.94.1/go.mod h1:qAlAugsXlC+JWO+Bke5vCtc9ONxjQT3drlTTnAplMW4= +cloud.google.com/go v0.97.0/go.mod h1:GF7l59pYBVlXQIBLx3a761cZ41F9bBH3JUlihCt2Udc= +cloud.google.com/go v0.99.0/go.mod h1:w0Xx2nLzqWJPuozYQX+hFfCSI8WioryfRDzkoI/Y2ZA= +cloud.google.com/go v0.100.1/go.mod h1:fs4QogzfH5n2pBXBP9vRiU+eCny7lD2vmFZy79Iuw1U= +cloud.google.com/go v0.100.2/go.mod h1:4Xra9TjzAeYHrl5+oeLlzbM2k3mjVhZh4UqTZ//w99A= +cloud.google.com/go v0.102.0/go.mod h1:oWcCzKlqJ5zgHQt9YsaeTY9KzIvjyy0ArmiBUgpQ+nc= +cloud.google.com/go v0.102.1/go.mod h1:XZ77E9qnTEnrgEOvr4xzfdX5TRo7fB4T2F4O6+34hIU= +cloud.google.com/go v0.104.0/go.mod h1:OO6xxXdJyvuJPcEPBLN9BJPD+jep5G1+2U5B5gkRYtA= +cloud.google.com/go v0.105.0/go.mod h1:PrLgOJNe5nfE9UMxKxgXj4mD3voiP+YQ6gdt6KMFOKM= +cloud.google.com/go v0.107.0/go.mod h1:wpc2eNrD7hXUTy8EKS10jkxpZBjASrORK7goS+3YX2I= +cloud.google.com/go v0.110.0/go.mod h1:SJnCLqQ0FCFGSZMUNUf84MV3Aia54kn7pi8st7tMzaY= +cloud.google.com/go v0.110.2/go.mod h1:k04UEeEtb6ZBRTv3dZz4CeJC3jKGxyhl0sAiVVquxiw= +cloud.google.com/go v0.110.4/go.mod h1:+EYjdK8e5RME/VY/qLCAtuyALQ9q67dvuum8i+H5xsI= +cloud.google.com/go v0.110.6/go.mod h1:+EYjdK8e5RME/VY/qLCAtuyALQ9q67dvuum8i+H5xsI= +cloud.google.com/go v0.110.7/go.mod h1:+EYjdK8e5RME/VY/qLCAtuyALQ9q67dvuum8i+H5xsI= +cloud.google.com/go v0.110.8/go.mod h1:Iz8AkXJf1qmxC3Oxoep8R1T36w8B92yU29PcBhHO5fk= cloud.google.com/go v0.112.1 h1:uJSeirPke5UNZHIb4SxfZklVSiWWVqW4oXlETwZziwM= cloud.google.com/go v0.112.1/go.mod h1:+Vbu+Y1UU+I1rjmzeMOb/8RfkKJK2Gyxi1X6jJCZLo4= +cloud.google.com/go/accessapproval v1.4.0/go.mod h1:zybIuC3KpDOvotz59lFe5qxRZx6C75OtwbisN56xYB4= +cloud.google.com/go/accessapproval v1.5.0/go.mod h1:HFy3tuiGvMdcd/u+Cu5b9NkO1pEICJ46IR82PoUdplw= +cloud.google.com/go/accessapproval v1.6.0/go.mod h1:R0EiYnwV5fsRFiKZkPHr6mwyk2wxUJ30nL4j2pcFY2E= +cloud.google.com/go/accessapproval v1.7.1/go.mod h1:JYczztsHRMK7NTXb6Xw+dwbs/WnOJxbo/2mTI+Kgg68= +cloud.google.com/go/accesscontextmanager v1.3.0/go.mod h1:TgCBehyr5gNMz7ZaH9xubp+CE8dkrszb4oK9CWyvD4o= +cloud.google.com/go/accesscontextmanager v1.4.0/go.mod h1:/Kjh7BBu/Gh83sv+K60vN9QE5NJcd80sU33vIe2IFPE= +cloud.google.com/go/accesscontextmanager v1.6.0/go.mod h1:8XCvZWfYw3K/ji0iVnp+6pu7huxoQTLmxAbVjbloTtM= +cloud.google.com/go/accesscontextmanager v1.7.0/go.mod h1:CEGLewx8dwa33aDAZQujl7Dx+uYhS0eay198wB/VumQ= +cloud.google.com/go/accesscontextmanager v1.8.0/go.mod h1:uI+AI/r1oyWK99NN8cQ3UK76AMelMzgZCvJfsi2c+ps= +cloud.google.com/go/accesscontextmanager v1.8.1/go.mod h1:JFJHfvuaTC+++1iL1coPiG1eu5D24db2wXCDWDjIrxo= +cloud.google.com/go/aiplatform v1.22.0/go.mod h1:ig5Nct50bZlzV6NvKaTwmplLLddFx0YReh9WfTO5jKw= +cloud.google.com/go/aiplatform v1.24.0/go.mod h1:67UUvRBKG6GTayHKV8DBv2RtR1t93YRu5B1P3x99mYY= +cloud.google.com/go/aiplatform v1.27.0/go.mod h1:Bvxqtl40l0WImSb04d0hXFU7gDOiq9jQmorivIiWcKg= +cloud.google.com/go/aiplatform v1.35.0/go.mod h1:7MFT/vCaOyZT/4IIFfxH4ErVg/4ku6lKv3w0+tFTgXQ= +cloud.google.com/go/aiplatform v1.36.1/go.mod h1:WTm12vJRPARNvJ+v6P52RDHCNe4AhvjcIZ/9/RRHy/k= +cloud.google.com/go/aiplatform v1.37.0/go.mod h1:IU2Cv29Lv9oCn/9LkFiiuKfwrRTq+QQMbW+hPCxJGZw= +cloud.google.com/go/aiplatform v1.45.0/go.mod h1:Iu2Q7sC7QGhXUeOhAj/oCK9a+ULz1O4AotZiqjQ8MYA= +cloud.google.com/go/aiplatform v1.48.0/go.mod h1:Iu2Q7sC7QGhXUeOhAj/oCK9a+ULz1O4AotZiqjQ8MYA= +cloud.google.com/go/aiplatform v1.50.0/go.mod h1:IRc2b8XAMTa9ZmfJV1BCCQbieWWvDnP1A8znyz5N7y4= +cloud.google.com/go/aiplatform v1.51.0/go.mod h1:IRc2b8XAMTa9ZmfJV1BCCQbieWWvDnP1A8znyz5N7y4= +cloud.google.com/go/analytics v0.11.0/go.mod h1:DjEWCu41bVbYcKyvlws9Er60YE4a//bK6mnhWvQeFNI= +cloud.google.com/go/analytics v0.12.0/go.mod h1:gkfj9h6XRf9+TS4bmuhPEShsh3hH8PAZzm/41OOhQd4= +cloud.google.com/go/analytics v0.17.0/go.mod h1:WXFa3WSym4IZ+JiKmavYdJwGG/CvpqiqczmL59bTD9M= +cloud.google.com/go/analytics v0.18.0/go.mod h1:ZkeHGQlcIPkw0R/GW+boWHhCOR43xz9RN/jn7WcqfIE= +cloud.google.com/go/analytics v0.19.0/go.mod h1:k8liqf5/HCnOUkbawNtrWWc+UAzyDlW89doe8TtoDsE= +cloud.google.com/go/analytics v0.21.2/go.mod h1:U8dcUtmDmjrmUTnnnRnI4m6zKn/yaA5N9RlEkYFHpQo= +cloud.google.com/go/analytics v0.21.3/go.mod h1:U8dcUtmDmjrmUTnnnRnI4m6zKn/yaA5N9RlEkYFHpQo= +cloud.google.com/go/apigateway v1.3.0/go.mod h1:89Z8Bhpmxu6AmUxuVRg/ECRGReEdiP3vQtk4Z1J9rJk= +cloud.google.com/go/apigateway v1.4.0/go.mod h1:pHVY9MKGaH9PQ3pJ4YLzoj6U5FUDeDFBllIz7WmzJoc= +cloud.google.com/go/apigateway v1.5.0/go.mod h1:GpnZR3Q4rR7LVu5951qfXPJCHquZt02jf7xQx7kpqN8= +cloud.google.com/go/apigateway v1.6.1/go.mod h1:ufAS3wpbRjqfZrzpvLC2oh0MFlpRJm2E/ts25yyqmXA= +cloud.google.com/go/apigeeconnect v1.3.0/go.mod h1:G/AwXFAKo0gIXkPTVfZDd2qA1TxBXJ3MgMRBQkIi9jc= +cloud.google.com/go/apigeeconnect v1.4.0/go.mod h1:kV4NwOKqjvt2JYR0AoIWo2QGfoRtn/pkS3QlHp0Ni04= +cloud.google.com/go/apigeeconnect v1.5.0/go.mod h1:KFaCqvBRU6idyhSNyn3vlHXc8VMDJdRmwDF6JyFRqZ8= +cloud.google.com/go/apigeeconnect v1.6.1/go.mod h1:C4awq7x0JpLtrlQCr8AzVIzAaYgngRqWf9S5Uhg+wWs= +cloud.google.com/go/apigeeregistry v0.4.0/go.mod h1:EUG4PGcsZvxOXAdyEghIdXwAEi/4MEaoqLMLDMIwKXY= +cloud.google.com/go/apigeeregistry v0.5.0/go.mod h1:YR5+s0BVNZfVOUkMa5pAR2xGd0A473vA5M7j247o1wM= +cloud.google.com/go/apigeeregistry v0.6.0/go.mod h1:BFNzW7yQVLZ3yj0TKcwzb8n25CFBri51GVGOEUcgQsc= +cloud.google.com/go/apigeeregistry v0.7.1/go.mod h1:1XgyjZye4Mqtw7T9TsY4NW10U7BojBvG4RMD+vRDrIw= +cloud.google.com/go/apikeys v0.4.0/go.mod h1:XATS/yqZbaBK0HOssf+ALHp8jAlNHUgyfprvNcBIszU= +cloud.google.com/go/apikeys v0.5.0/go.mod h1:5aQfwY4D+ewMMWScd3hm2en3hCj+BROlyrt3ytS7KLI= +cloud.google.com/go/apikeys v0.6.0/go.mod h1:kbpXu5upyiAlGkKrJgQl8A0rKNNJ7dQ377pdroRSSi8= +cloud.google.com/go/appengine v1.4.0/go.mod h1:CS2NhuBuDXM9f+qscZ6V86m1MIIqPj3WC/UoEuR1Sno= +cloud.google.com/go/appengine v1.5.0/go.mod h1:TfasSozdkFI0zeoxW3PTBLiNqRmzraodCWatWI9Dmak= +cloud.google.com/go/appengine v1.6.0/go.mod h1:hg6i0J/BD2cKmDJbaFSYHFyZkgBEfQrDg/X0V5fJn84= +cloud.google.com/go/appengine v1.7.0/go.mod h1:eZqpbHFCqRGa2aCdope7eC0SWLV1j0neb/QnMJVWx6A= +cloud.google.com/go/appengine v1.7.1/go.mod h1:IHLToyb/3fKutRysUlFO0BPt5j7RiQ45nrzEJmKTo6E= +cloud.google.com/go/appengine v1.8.1/go.mod h1:6NJXGLVhZCN9aQ/AEDvmfzKEfoYBlfB80/BHiKVputY= +cloud.google.com/go/area120 v0.5.0/go.mod h1:DE/n4mp+iqVyvxHN41Vf1CR602GiHQjFPusMFW6bGR4= +cloud.google.com/go/area120 v0.6.0/go.mod h1:39yFJqWVgm0UZqWTOdqkLhjoC7uFfgXRC8g/ZegeAh0= +cloud.google.com/go/area120 v0.7.0/go.mod h1:a3+8EUD1SX5RUcCs3MY5YasiO1z6yLiNLRiFrykbynY= +cloud.google.com/go/area120 v0.7.1/go.mod h1:j84i4E1RboTWjKtZVWXPqvK5VHQFJRF2c1Nm69pWm9k= +cloud.google.com/go/area120 v0.8.1/go.mod h1:BVfZpGpB7KFVNxPiQBuHkX6Ed0rS51xIgmGyjrAfzsg= +cloud.google.com/go/artifactregistry v1.6.0/go.mod h1:IYt0oBPSAGYj/kprzsBjZ/4LnG/zOcHyFHjWPCi6SAQ= +cloud.google.com/go/artifactregistry v1.7.0/go.mod h1:mqTOFOnGZx8EtSqK/ZWcsm/4U8B77rbcLP6ruDU2Ixk= +cloud.google.com/go/artifactregistry v1.8.0/go.mod h1:w3GQXkJX8hiKN0v+at4b0qotwijQbYUqF2GWkZzAhC0= +cloud.google.com/go/artifactregistry v1.9.0/go.mod h1:2K2RqvA2CYvAeARHRkLDhMDJ3OXy26h3XW+3/Jh2uYc= +cloud.google.com/go/artifactregistry v1.11.1/go.mod h1:lLYghw+Itq9SONbCa1YWBoWs1nOucMH0pwXN1rOBZFI= +cloud.google.com/go/artifactregistry v1.11.2/go.mod h1:nLZns771ZGAwVLzTX/7Al6R9ehma4WUEhZGWV6CeQNQ= +cloud.google.com/go/artifactregistry v1.12.0/go.mod h1:o6P3MIvtzTOnmvGagO9v/rOjjA0HmhJ+/6KAXrmYDCI= +cloud.google.com/go/artifactregistry v1.13.0/go.mod h1:uy/LNfoOIivepGhooAUpL1i30Hgee3Cu0l4VTWHUC08= +cloud.google.com/go/artifactregistry v1.14.1/go.mod h1:nxVdG19jTaSTu7yA7+VbWL346r3rIdkZ142BSQqhn5E= +cloud.google.com/go/artifactregistry v1.14.2/go.mod h1:Xk+QbsKEb0ElmyeMfdHAey41B+qBq3q5R5f5xD4XT3U= +cloud.google.com/go/asset v1.5.0/go.mod h1:5mfs8UvcM5wHhqtSv8J1CtxxaQq3AdBxxQi2jGW/K4o= +cloud.google.com/go/asset v1.7.0/go.mod h1:YbENsRK4+xTiL+Ofoj5Ckf+O17kJtgp3Y3nn4uzZz5s= +cloud.google.com/go/asset v1.8.0/go.mod h1:mUNGKhiqIdbr8X7KNayoYvyc4HbbFO9URsjbytpUaW0= +cloud.google.com/go/asset v1.9.0/go.mod h1:83MOE6jEJBMqFKadM9NLRcs80Gdw76qGuHn8m3h8oHQ= +cloud.google.com/go/asset v1.10.0/go.mod h1:pLz7uokL80qKhzKr4xXGvBQXnzHn5evJAEAtZiIb0wY= +cloud.google.com/go/asset v1.11.1/go.mod h1:fSwLhbRvC9p9CXQHJ3BgFeQNM4c9x10lqlrdEUYXlJo= +cloud.google.com/go/asset v1.12.0/go.mod h1:h9/sFOa4eDIyKmH6QMpm4eUK3pDojWnUhTgJlk762Hg= +cloud.google.com/go/asset v1.13.0/go.mod h1:WQAMyYek/b7NBpYq/K4KJWcRqzoalEsxz/t/dTk4THw= +cloud.google.com/go/asset v1.14.1/go.mod h1:4bEJ3dnHCqWCDbWJ/6Vn7GVI9LerSi7Rfdi03hd+WTQ= +cloud.google.com/go/asset v1.15.0/go.mod h1:tpKafV6mEut3+vN9ScGvCHXHj7FALFVta+okxFECHcg= +cloud.google.com/go/assuredworkloads v1.5.0/go.mod h1:n8HOZ6pff6re5KYfBXcFvSViQjDwxFkAkmUFffJRbbY= +cloud.google.com/go/assuredworkloads v1.6.0/go.mod h1:yo2YOk37Yc89Rsd5QMVECvjaMKymF9OP+QXWlKXUkXw= +cloud.google.com/go/assuredworkloads v1.7.0/go.mod h1:z/736/oNmtGAyU47reJgGN+KVoYoxeLBoj4XkKYscNI= +cloud.google.com/go/assuredworkloads v1.8.0/go.mod h1:AsX2cqyNCOvEQC8RMPnoc0yEarXQk6WEKkxYfL6kGIo= +cloud.google.com/go/assuredworkloads v1.9.0/go.mod h1:kFuI1P78bplYtT77Tb1hi0FMxM0vVpRC7VVoJC3ZoT0= +cloud.google.com/go/assuredworkloads v1.10.0/go.mod h1:kwdUQuXcedVdsIaKgKTp9t0UJkE5+PAVNhdQm4ZVq2E= +cloud.google.com/go/assuredworkloads v1.11.1/go.mod h1:+F04I52Pgn5nmPG36CWFtxmav6+7Q+c5QyJoL18Lry0= +cloud.google.com/go/automl v1.5.0/go.mod h1:34EjfoFGMZ5sgJ9EoLsRtdPSNZLcfflJR39VbVNS2M0= +cloud.google.com/go/automl v1.6.0/go.mod h1:ugf8a6Fx+zP0D59WLhqgTDsQI9w07o64uf/Is3Nh5p8= +cloud.google.com/go/automl v1.7.0/go.mod h1:RL9MYCCsJEOmt0Wf3z9uzG0a7adTT1fe+aObgSpkCt8= +cloud.google.com/go/automl v1.8.0/go.mod h1:xWx7G/aPEe/NP+qzYXktoBSDfjO+vnKMGgsApGJJquM= +cloud.google.com/go/automl v1.12.0/go.mod h1:tWDcHDp86aMIuHmyvjuKeeHEGq76lD7ZqfGLN6B0NuU= +cloud.google.com/go/automl v1.13.1/go.mod h1:1aowgAHWYZU27MybSCFiukPO7xnyawv7pt3zK4bheQE= +cloud.google.com/go/baremetalsolution v0.3.0/go.mod h1:XOrocE+pvK1xFfleEnShBlNAXf+j5blPPxrhjKgnIFc= +cloud.google.com/go/baremetalsolution v0.4.0/go.mod h1:BymplhAadOO/eBa7KewQ0Ppg4A4Wplbn+PsFKRLo0uI= +cloud.google.com/go/baremetalsolution v0.5.0/go.mod h1:dXGxEkmR9BMwxhzBhV0AioD0ULBmuLZI8CdwalUxuss= +cloud.google.com/go/baremetalsolution v1.1.1/go.mod h1:D1AV6xwOksJMV4OSlWHtWuFNZZYujJknMAP4Qa27QIA= +cloud.google.com/go/baremetalsolution v1.2.0/go.mod h1:68wi9AwPYkEWIUT4SvSGS9UJwKzNpshjHsH4lzk8iOw= +cloud.google.com/go/batch v0.3.0/go.mod h1:TR18ZoAekj1GuirsUsR1ZTKN3FC/4UDnScjT8NXImFE= +cloud.google.com/go/batch v0.4.0/go.mod h1:WZkHnP43R/QCGQsZ+0JyG4i79ranE2u8xvjq/9+STPE= +cloud.google.com/go/batch v0.7.0/go.mod h1:vLZN95s6teRUqRQ4s3RLDsH8PvboqBK+rn1oevL159g= +cloud.google.com/go/batch v1.3.1/go.mod h1:VguXeQKXIYaeeIYbuozUmBR13AfL4SJP7IltNPS+A4A= +cloud.google.com/go/batch v1.4.1/go.mod h1:KdBmDD61K0ovcxoRHGrN6GmOBWeAOyCgKD0Mugx4Fkk= +cloud.google.com/go/batch v1.5.0/go.mod h1:KdBmDD61K0ovcxoRHGrN6GmOBWeAOyCgKD0Mugx4Fkk= +cloud.google.com/go/beyondcorp v0.2.0/go.mod h1:TB7Bd+EEtcw9PCPQhCJtJGjk/7TC6ckmnSFS+xwTfm4= +cloud.google.com/go/beyondcorp v0.3.0/go.mod h1:E5U5lcrcXMsCuoDNyGrpyTm/hn7ne941Jz2vmksAxW8= +cloud.google.com/go/beyondcorp v0.4.0/go.mod h1:3ApA0mbhHx6YImmuubf5pyW8srKnCEPON32/5hj+RmM= +cloud.google.com/go/beyondcorp v0.5.0/go.mod h1:uFqj9X+dSfrheVp7ssLTaRHd2EHqSL4QZmH4e8WXGGU= +cloud.google.com/go/beyondcorp v0.6.1/go.mod h1:YhxDWw946SCbmcWo3fAhw3V4XZMSpQ/VYfcKGAEU8/4= +cloud.google.com/go/beyondcorp v1.0.0/go.mod h1:YhxDWw946SCbmcWo3fAhw3V4XZMSpQ/VYfcKGAEU8/4= cloud.google.com/go/bigquery v1.0.1/go.mod h1:i/xbL2UlR5RvWAURpBYZTtm/cXjCha9lbfbpx4poX+o= cloud.google.com/go/bigquery v1.3.0/go.mod h1:PjpwJnslEMmckchkHFfq+HTD2DmtT67aNFKH1/VBDHE= cloud.google.com/go/bigquery v1.4.0/go.mod h1:S8dzgnTigyfTmLBfrtrhyYhwRxG72rYxvftPBK2Dvzc= cloud.google.com/go/bigquery v1.5.0/go.mod h1:snEHRnqQbz117VIFhE8bmtwIDY80NLUZUMb4Nv6dBIg= cloud.google.com/go/bigquery v1.7.0/go.mod h1://okPTzCYNXSlb24MZs83e2Do+h+VXtc4gLoIoXIAPc= cloud.google.com/go/bigquery v1.8.0/go.mod h1:J5hqkt3O0uAFnINi6JXValWIb1v0goeZM77hZzJN/fQ= +cloud.google.com/go/bigquery v1.42.0/go.mod h1:8dRTJxhtG+vwBKzE5OseQn/hiydoQN3EedCaOdYmxRA= +cloud.google.com/go/bigquery v1.43.0/go.mod h1:ZMQcXHsl+xmU1z36G2jNGZmKp9zNY5BUua5wDgmNCfw= +cloud.google.com/go/bigquery v1.44.0/go.mod h1:0Y33VqXTEsbamHJvJHdFmtqHvMIY28aK1+dFsvaChGc= +cloud.google.com/go/bigquery v1.47.0/go.mod h1:sA9XOgy0A8vQK9+MWhEQTY6Tix87M/ZurWFIxmF9I/E= +cloud.google.com/go/bigquery v1.48.0/go.mod h1:QAwSz+ipNgfL5jxiaK7weyOhzdoAy1zFm0Nf1fysJac= +cloud.google.com/go/bigquery v1.49.0/go.mod h1:Sv8hMmTFFYBlt/ftw2uN6dFdQPzBlREY9yBh7Oy7/4Q= +cloud.google.com/go/bigquery v1.50.0/go.mod h1:YrleYEh2pSEbgTBZYMJ5SuSr0ML3ypjRB1zgf7pvQLU= +cloud.google.com/go/bigquery v1.52.0/go.mod h1:3b/iXjRQGU4nKa87cXeg6/gogLjO8C6PmuM8i5Bi/u4= +cloud.google.com/go/bigquery v1.53.0/go.mod h1:3b/iXjRQGU4nKa87cXeg6/gogLjO8C6PmuM8i5Bi/u4= +cloud.google.com/go/bigquery v1.55.0/go.mod h1:9Y5I3PN9kQWuid6183JFhOGOW3GcirA5LpsKCUn+2ec= +cloud.google.com/go/bigquery v1.56.0/go.mod h1:KDcsploXTEY7XT3fDQzMUZlpQLHzE4itubHrnmhUrZA= +cloud.google.com/go/billing v1.4.0/go.mod h1:g9IdKBEFlItS8bTtlrZdVLWSSdSyFUZKXNS02zKMOZY= +cloud.google.com/go/billing v1.5.0/go.mod h1:mztb1tBc3QekhjSgmpf/CV4LzWXLzCArwpLmP2Gm88s= +cloud.google.com/go/billing v1.6.0/go.mod h1:WoXzguj+BeHXPbKfNWkqVtDdzORazmCjraY+vrxcyvI= +cloud.google.com/go/billing v1.7.0/go.mod h1:q457N3Hbj9lYwwRbnlD7vUpyjq6u5U1RAOArInEiD5Y= +cloud.google.com/go/billing v1.12.0/go.mod h1:yKrZio/eu+okO/2McZEbch17O5CB5NpZhhXG6Z766ss= +cloud.google.com/go/billing v1.13.0/go.mod h1:7kB2W9Xf98hP9Sr12KfECgfGclsH3CQR0R08tnRlRbc= +cloud.google.com/go/billing v1.16.0/go.mod h1:y8vx09JSSJG02k5QxbycNRrN7FGZB6F3CAcgum7jvGA= +cloud.google.com/go/billing v1.17.0/go.mod h1:Z9+vZXEq+HwH7bhJkyI4OQcR6TSbeMrjlpEjO2vzY64= +cloud.google.com/go/billing v1.17.1/go.mod h1:Z9+vZXEq+HwH7bhJkyI4OQcR6TSbeMrjlpEjO2vzY64= +cloud.google.com/go/binaryauthorization v1.1.0/go.mod h1:xwnoWu3Y84jbuHa0zd526MJYmtnVXn0syOjaJgy4+dM= +cloud.google.com/go/binaryauthorization v1.2.0/go.mod h1:86WKkJHtRcv5ViNABtYMhhNWRrD1Vpi//uKEy7aYEfI= +cloud.google.com/go/binaryauthorization v1.3.0/go.mod h1:lRZbKgjDIIQvzYQS1p99A7/U1JqvqeZg0wiI5tp6tg0= +cloud.google.com/go/binaryauthorization v1.4.0/go.mod h1:tsSPQrBd77VLplV70GUhBf/Zm3FsKmgSqgm4UmiDItk= +cloud.google.com/go/binaryauthorization v1.5.0/go.mod h1:OSe4OU1nN/VswXKRBmciKpo9LulY41gch5c68htf3/Q= +cloud.google.com/go/binaryauthorization v1.6.1/go.mod h1:TKt4pa8xhowwffiBmbrbcxijJRZED4zrqnwZ1lKH51U= +cloud.google.com/go/binaryauthorization v1.7.0/go.mod h1:Zn+S6QqTMn6odcMU1zDZCJxPjU2tZPV1oDl45lWY154= +cloud.google.com/go/certificatemanager v1.3.0/go.mod h1:n6twGDvcUBFu9uBgt4eYvvf3sQ6My8jADcOVwHmzadg= +cloud.google.com/go/certificatemanager v1.4.0/go.mod h1:vowpercVFyqs8ABSmrdV+GiFf2H/ch3KyudYQEMM590= +cloud.google.com/go/certificatemanager v1.6.0/go.mod h1:3Hh64rCKjRAX8dXgRAyOcY5vQ/fE1sh8o+Mdd6KPgY8= +cloud.google.com/go/certificatemanager v1.7.1/go.mod h1:iW8J3nG6SaRYImIa+wXQ0g8IgoofDFRp5UMzaNk1UqI= +cloud.google.com/go/channel v1.8.0/go.mod h1:W5SwCXDJsq/rg3tn3oG0LOxpAo6IMxNa09ngphpSlnk= +cloud.google.com/go/channel v1.9.0/go.mod h1:jcu05W0my9Vx4mt3/rEHpfxc9eKi9XwsdDL8yBMbKUk= +cloud.google.com/go/channel v1.11.0/go.mod h1:IdtI0uWGqhEeatSB62VOoJ8FSUhJ9/+iGkJVqp74CGE= +cloud.google.com/go/channel v1.12.0/go.mod h1:VkxCGKASi4Cq7TbXxlaBezonAYpp1GCnKMY6tnMQnLU= +cloud.google.com/go/channel v1.16.0/go.mod h1:eN/q1PFSl5gyu0dYdmxNXscY/4Fi7ABmeHCJNf/oHmc= +cloud.google.com/go/channel v1.17.0/go.mod h1:RpbhJsGi/lXWAUM1eF4IbQGbsfVlg2o8Iiy2/YLfVT0= +cloud.google.com/go/cloudbuild v1.3.0/go.mod h1:WequR4ULxlqvMsjDEEEFnOG5ZSRSgWOywXYDb1vPE6U= +cloud.google.com/go/cloudbuild v1.4.0/go.mod h1:5Qwa40LHiOXmz3386FrjrYM93rM/hdRr7b53sySrTqA= +cloud.google.com/go/cloudbuild v1.6.0/go.mod h1:UIbc/w9QCbH12xX+ezUsgblrWv+Cv4Tw83GiSMHOn9M= +cloud.google.com/go/cloudbuild v1.7.0/go.mod h1:zb5tWh2XI6lR9zQmsm1VRA+7OCuve5d8S+zJUul8KTg= +cloud.google.com/go/cloudbuild v1.9.0/go.mod h1:qK1d7s4QlO0VwfYn5YuClDGg2hfmLZEb4wQGAbIgL1s= +cloud.google.com/go/cloudbuild v1.10.1/go.mod h1:lyJg7v97SUIPq4RC2sGsz/9tNczhyv2AjML/ci4ulzU= +cloud.google.com/go/cloudbuild v1.13.0/go.mod h1:lyJg7v97SUIPq4RC2sGsz/9tNczhyv2AjML/ci4ulzU= +cloud.google.com/go/cloudbuild v1.14.0/go.mod h1:lyJg7v97SUIPq4RC2sGsz/9tNczhyv2AjML/ci4ulzU= +cloud.google.com/go/clouddms v1.3.0/go.mod h1:oK6XsCDdW4Ib3jCCBugx+gVjevp2TMXFtgxvPSee3OM= +cloud.google.com/go/clouddms v1.4.0/go.mod h1:Eh7sUGCC+aKry14O1NRljhjyrr0NFC0G2cjwX0cByRk= +cloud.google.com/go/clouddms v1.5.0/go.mod h1:QSxQnhikCLUw13iAbffF2CZxAER3xDGNHjsTAkQJcQA= +cloud.google.com/go/clouddms v1.6.1/go.mod h1:Ygo1vL52Ov4TBZQquhz5fiw2CQ58gvu+PlS6PVXCpZI= +cloud.google.com/go/clouddms v1.7.0/go.mod h1:MW1dC6SOtI/tPNCciTsXtsGNEM0i0OccykPvv3hiYeM= +cloud.google.com/go/cloudtasks v1.5.0/go.mod h1:fD92REy1x5woxkKEkLdvavGnPJGEn8Uic9nWuLzqCpY= +cloud.google.com/go/cloudtasks v1.6.0/go.mod h1:C6Io+sxuke9/KNRkbQpihnW93SWDU3uXt92nu85HkYI= +cloud.google.com/go/cloudtasks v1.7.0/go.mod h1:ImsfdYWwlWNJbdgPIIGJWC+gemEGTBK/SunNQQNCAb4= +cloud.google.com/go/cloudtasks v1.8.0/go.mod h1:gQXUIwCSOI4yPVK7DgTVFiiP0ZW/eQkydWzwVMdHxrI= +cloud.google.com/go/cloudtasks v1.9.0/go.mod h1:w+EyLsVkLWHcOaqNEyvcKAsWp9p29dL6uL9Nst1cI7Y= +cloud.google.com/go/cloudtasks v1.10.0/go.mod h1:NDSoTLkZ3+vExFEWu2UJV1arUyzVDAiZtdWcsUyNwBs= +cloud.google.com/go/cloudtasks v1.11.1/go.mod h1:a9udmnou9KO2iulGscKR0qBYjreuX8oHwpmFsKspEvM= +cloud.google.com/go/cloudtasks v1.12.1/go.mod h1:a9udmnou9KO2iulGscKR0qBYjreuX8oHwpmFsKspEvM= +cloud.google.com/go/compute v0.1.0/go.mod h1:GAesmwr110a34z04OlxYkATPBEfVhkymfTBXtfbBFow= +cloud.google.com/go/compute v1.3.0/go.mod h1:cCZiE1NHEtai4wiufUhW8I8S1JKkAnhnQJWM7YD99wM= +cloud.google.com/go/compute v1.5.0/go.mod h1:9SMHyhJlzhlkJqrPAc839t2BZFTSk6Jdj6mkzQJeu0M= +cloud.google.com/go/compute v1.6.0/go.mod h1:T29tfhtVbq1wvAPo0E3+7vhgmkOYeXjhFvz/FMzPu0s= +cloud.google.com/go/compute v1.6.1/go.mod h1:g85FgpzFvNULZ+S8AYq87axRKuf2Kh7deLqV/jJ3thU= +cloud.google.com/go/compute v1.7.0/go.mod h1:435lt8av5oL9P3fv1OEzSbSUe+ybHXGMPQHHZWZxy9U= +cloud.google.com/go/compute v1.10.0/go.mod h1:ER5CLbMxl90o2jtNbGSbtfOpQKR0t15FOtRsugnLrlU= +cloud.google.com/go/compute v1.12.0/go.mod h1:e8yNOBcBONZU1vJKCvCoDw/4JQsA0dpM4x/6PIIOocU= +cloud.google.com/go/compute v1.12.1/go.mod h1:e8yNOBcBONZU1vJKCvCoDw/4JQsA0dpM4x/6PIIOocU= +cloud.google.com/go/compute v1.13.0/go.mod h1:5aPTS0cUNMIc1CE546K+Th6weJUNQErARyZtRXDJ8GE= +cloud.google.com/go/compute v1.14.0/go.mod h1:YfLtxrj9sU4Yxv+sXzZkyPjEyPBZfXHUvjxega5vAdo= +cloud.google.com/go/compute v1.15.1/go.mod h1:bjjoF/NtFUrkD/urWfdHaKuOPDR5nWIs63rR+SXhcpA= +cloud.google.com/go/compute v1.18.0/go.mod h1:1X7yHxec2Ga+Ss6jPyjxRxpu2uu7PLgsOVXvgU0yacs= +cloud.google.com/go/compute v1.19.0/go.mod h1:rikpw2y+UMidAe9tISo04EHNOIf42RLYF/q8Bs93scU= +cloud.google.com/go/compute v1.19.1/go.mod h1:6ylj3a05WF8leseCdIf77NK0g1ey+nj5IKd5/kvShxE= +cloud.google.com/go/compute v1.19.3/go.mod h1:qxvISKp/gYnXkSAD1ppcSOveRAmzxicEv/JlizULFrI= +cloud.google.com/go/compute v1.20.1/go.mod h1:4tCnrn48xsqlwSAiLf1HXMQk8CONslYbdiEZc9FEIbM= +cloud.google.com/go/compute v1.21.0/go.mod h1:4tCnrn48xsqlwSAiLf1HXMQk8CONslYbdiEZc9FEIbM= +cloud.google.com/go/compute v1.23.0/go.mod h1:4tCnrn48xsqlwSAiLf1HXMQk8CONslYbdiEZc9FEIbM= +cloud.google.com/go/compute/metadata v0.1.0/go.mod h1:Z1VN+bulIf6bt4P/C37K4DyZYZEXYonfTBHHFPO/4UU= +cloud.google.com/go/compute/metadata v0.2.0/go.mod h1:zFmK7XCadkQkj6TtorcaGlCW1hT1fIilQDwofLpJ20k= +cloud.google.com/go/compute/metadata v0.2.1/go.mod h1:jgHgmJd2RKBGzXqF5LR2EZMGxBkeanZ9wwa75XHJgOM= +cloud.google.com/go/compute/metadata v0.2.3/go.mod h1:VAV5nSsACxMJvgaAuX6Pk2AawlZn8kiOGuCv6gTkwuA= cloud.google.com/go/compute/metadata v0.3.0 h1:Tz+eQXMEqDIKRsmY3cHTL6FVaynIjX2QxYC4trgAKZc= cloud.google.com/go/compute/metadata v0.3.0/go.mod h1:zFmK7XCadkQkj6TtorcaGlCW1hT1fIilQDwofLpJ20k= +cloud.google.com/go/contactcenterinsights v1.3.0/go.mod h1:Eu2oemoePuEFc/xKFPjbTuPSj0fYJcPls9TFlPNnHHY= +cloud.google.com/go/contactcenterinsights v1.4.0/go.mod h1:L2YzkGbPsv+vMQMCADxJoT9YiTTnSEd6fEvCeHTYVck= +cloud.google.com/go/contactcenterinsights v1.6.0/go.mod h1:IIDlT6CLcDoyv79kDv8iWxMSTZhLxSCofVV5W6YFM/w= +cloud.google.com/go/contactcenterinsights v1.9.1/go.mod h1:bsg/R7zGLYMVxFFzfh9ooLTruLRCG9fnzhH9KznHhbM= +cloud.google.com/go/contactcenterinsights v1.10.0/go.mod h1:bsg/R7zGLYMVxFFzfh9ooLTruLRCG9fnzhH9KznHhbM= +cloud.google.com/go/contactcenterinsights v1.11.0/go.mod h1:hutBdImE4XNZ1NV4vbPJKSFOnQruhC5Lj9bZqWMTKiU= +cloud.google.com/go/container v1.6.0/go.mod h1:Xazp7GjJSeUYo688S+6J5V+n/t+G5sKBTFkKNudGRxg= +cloud.google.com/go/container v1.7.0/go.mod h1:Dp5AHtmothHGX3DwwIHPgq45Y8KmNsgN3amoYfxVkLo= +cloud.google.com/go/container v1.13.1/go.mod h1:6wgbMPeQRw9rSnKBCAJXnds3Pzj03C4JHamr8asWKy4= +cloud.google.com/go/container v1.14.0/go.mod h1:3AoJMPhHfLDxLvrlVWaK57IXzaPnLaZq63WX59aQBfM= +cloud.google.com/go/container v1.15.0/go.mod h1:ft+9S0WGjAyjDggg5S06DXj+fHJICWg8L7isCQe9pQA= +cloud.google.com/go/container v1.22.1/go.mod h1:lTNExE2R7f+DLbAN+rJiKTisauFCaoDq6NURZ83eVH4= +cloud.google.com/go/container v1.24.0/go.mod h1:lTNExE2R7f+DLbAN+rJiKTisauFCaoDq6NURZ83eVH4= +cloud.google.com/go/container v1.26.0/go.mod h1:YJCmRet6+6jnYYRS000T6k0D0xUXQgBSaJ7VwI8FBj4= +cloud.google.com/go/containeranalysis v0.5.1/go.mod h1:1D92jd8gRR/c0fGMlymRgxWD3Qw9C1ff6/T7mLgVL8I= +cloud.google.com/go/containeranalysis v0.6.0/go.mod h1:HEJoiEIu+lEXM+k7+qLCci0h33lX3ZqoYFdmPcoO7s4= +cloud.google.com/go/containeranalysis v0.7.0/go.mod h1:9aUL+/vZ55P2CXfuZjS4UjQ9AgXoSw8Ts6lemfmxBxI= +cloud.google.com/go/containeranalysis v0.9.0/go.mod h1:orbOANbwk5Ejoom+s+DUCTTJ7IBdBQJDcSylAx/on9s= +cloud.google.com/go/containeranalysis v0.10.1/go.mod h1:Ya2jiILITMY68ZLPaogjmOMNkwsDrWBSTyBubGXO7j0= +cloud.google.com/go/containeranalysis v0.11.0/go.mod h1:4n2e99ZwpGxpNcz+YsFT1dfOHPQFGcAC8FN2M2/ne/U= +cloud.google.com/go/datacatalog v1.3.0/go.mod h1:g9svFY6tuR+j+hrTw3J2dNcmI0dzmSiyOzm8kpLq0a0= +cloud.google.com/go/datacatalog v1.5.0/go.mod h1:M7GPLNQeLfWqeIm3iuiruhPzkt65+Bx8dAKvScX8jvs= +cloud.google.com/go/datacatalog v1.6.0/go.mod h1:+aEyF8JKg+uXcIdAmmaMUmZ3q1b/lKLtXCmXdnc0lbc= +cloud.google.com/go/datacatalog v1.7.0/go.mod h1:9mEl4AuDYWw81UGc41HonIHH7/sn52H0/tc8f8ZbZIE= +cloud.google.com/go/datacatalog v1.8.0/go.mod h1:KYuoVOv9BM8EYz/4eMFxrr4DUKhGIOXxZoKYF5wdISM= +cloud.google.com/go/datacatalog v1.8.1/go.mod h1:RJ58z4rMp3gvETA465Vg+ag8BGgBdnRPEMMSTr5Uv+M= +cloud.google.com/go/datacatalog v1.12.0/go.mod h1:CWae8rFkfp6LzLumKOnmVh4+Zle4A3NXLzVJ1d1mRm0= +cloud.google.com/go/datacatalog v1.13.0/go.mod h1:E4Rj9a5ZtAxcQJlEBTLgMTphfP11/lNaAshpoBgemX8= +cloud.google.com/go/datacatalog v1.14.0/go.mod h1:h0PrGtlihoutNMp/uvwhawLQ9+c63Kz65UFqh49Yo+E= +cloud.google.com/go/datacatalog v1.14.1/go.mod h1:d2CevwTG4yedZilwe+v3E3ZBDRMobQfSG/a6cCCN5R4= +cloud.google.com/go/datacatalog v1.16.0/go.mod h1:d2CevwTG4yedZilwe+v3E3ZBDRMobQfSG/a6cCCN5R4= +cloud.google.com/go/datacatalog v1.17.1/go.mod h1:nCSYFHgtxh2MiEktWIz71s/X+7ds/UT9kp0PC7waCzE= +cloud.google.com/go/datacatalog v1.18.0/go.mod h1:nCSYFHgtxh2MiEktWIz71s/X+7ds/UT9kp0PC7waCzE= +cloud.google.com/go/dataflow v0.6.0/go.mod h1:9QwV89cGoxjjSR9/r7eFDqqjtvbKxAK2BaYU6PVk9UM= +cloud.google.com/go/dataflow v0.7.0/go.mod h1:PX526vb4ijFMesO1o202EaUmouZKBpjHsTlCtB4parQ= +cloud.google.com/go/dataflow v0.8.0/go.mod h1:Rcf5YgTKPtQyYz8bLYhFoIV/vP39eL7fWNcSOyFfLJE= +cloud.google.com/go/dataflow v0.9.1/go.mod h1:Wp7s32QjYuQDWqJPFFlnBKhkAtiFpMTdg00qGbnIHVw= +cloud.google.com/go/dataform v0.3.0/go.mod h1:cj8uNliRlHpa6L3yVhDOBrUXH+BPAO1+KFMQQNSThKo= +cloud.google.com/go/dataform v0.4.0/go.mod h1:fwV6Y4Ty2yIFL89huYlEkwUPtS7YZinZbzzj5S9FzCE= +cloud.google.com/go/dataform v0.5.0/go.mod h1:GFUYRe8IBa2hcomWplodVmUx/iTL0FrsauObOM3Ipr0= +cloud.google.com/go/dataform v0.6.0/go.mod h1:QPflImQy33e29VuapFdf19oPbE4aYTJxr31OAPV+ulA= +cloud.google.com/go/dataform v0.7.0/go.mod h1:7NulqnVozfHvWUBpMDfKMUESr+85aJsC/2O0o3jWPDE= +cloud.google.com/go/dataform v0.8.1/go.mod h1:3BhPSiw8xmppbgzeBbmDvmSWlwouuJkXsXsb8UBih9M= +cloud.google.com/go/datafusion v1.4.0/go.mod h1:1Zb6VN+W6ALo85cXnM1IKiPw+yQMKMhB9TsTSRDo/38= +cloud.google.com/go/datafusion v1.5.0/go.mod h1:Kz+l1FGHB0J+4XF2fud96WMmRiq/wj8N9u007vyXZ2w= +cloud.google.com/go/datafusion v1.6.0/go.mod h1:WBsMF8F1RhSXvVM8rCV3AeyWVxcC2xY6vith3iw3S+8= +cloud.google.com/go/datafusion v1.7.1/go.mod h1:KpoTBbFmoToDExJUso/fcCiguGDk7MEzOWXUsJo0wsI= +cloud.google.com/go/datalabeling v0.5.0/go.mod h1:TGcJ0G2NzcsXSE/97yWjIZO0bXj0KbVlINXMG9ud42I= +cloud.google.com/go/datalabeling v0.6.0/go.mod h1:WqdISuk/+WIGeMkpw/1q7bK/tFEZxsrFJOJdY2bXvTQ= +cloud.google.com/go/datalabeling v0.7.0/go.mod h1:WPQb1y08RJbmpM3ww0CSUAGweL0SxByuW2E+FU+wXcM= +cloud.google.com/go/datalabeling v0.8.1/go.mod h1:XS62LBSVPbYR54GfYQsPXZjTW8UxCK2fkDciSrpRFdY= +cloud.google.com/go/dataplex v1.3.0/go.mod h1:hQuRtDg+fCiFgC8j0zV222HvzFQdRd+SVX8gdmFcZzA= +cloud.google.com/go/dataplex v1.4.0/go.mod h1:X51GfLXEMVJ6UN47ESVqvlsRplbLhcsAt0kZCCKsU0A= +cloud.google.com/go/dataplex v1.5.2/go.mod h1:cVMgQHsmfRoI5KFYq4JtIBEUbYwc3c7tXmIDhRmNNVQ= +cloud.google.com/go/dataplex v1.6.0/go.mod h1:bMsomC/aEJOSpHXdFKFGQ1b0TDPIeL28nJObeO1ppRs= +cloud.google.com/go/dataplex v1.8.1/go.mod h1:7TyrDT6BCdI8/38Uvp0/ZxBslOslP2X2MPDucliyvSE= +cloud.google.com/go/dataplex v1.9.0/go.mod h1:7TyrDT6BCdI8/38Uvp0/ZxBslOslP2X2MPDucliyvSE= +cloud.google.com/go/dataplex v1.9.1/go.mod h1:7TyrDT6BCdI8/38Uvp0/ZxBslOslP2X2MPDucliyvSE= +cloud.google.com/go/dataproc v1.7.0/go.mod h1:CKAlMjII9H90RXaMpSxQ8EU6dQx6iAYNPcYPOkSbi8s= +cloud.google.com/go/dataproc v1.8.0/go.mod h1:5OW+zNAH0pMpw14JVrPONsxMQYMBqJuzORhIBfBn9uI= +cloud.google.com/go/dataproc v1.12.0/go.mod h1:zrF3aX0uV3ikkMz6z4uBbIKyhRITnxvr4i3IjKsKrw4= +cloud.google.com/go/dataproc/v2 v2.0.1/go.mod h1:7Ez3KRHdFGcfY7GcevBbvozX+zyWGcwLJvvAMwCaoZ4= +cloud.google.com/go/dataproc/v2 v2.2.0/go.mod h1:lZR7AQtwZPvmINx5J87DSOOpTfof9LVZju6/Qo4lmcY= +cloud.google.com/go/dataqna v0.5.0/go.mod h1:90Hyk596ft3zUQ8NkFfvICSIfHFh1Bc7C4cK3vbhkeo= +cloud.google.com/go/dataqna v0.6.0/go.mod h1:1lqNpM7rqNLVgWBJyk5NF6Uen2PHym0jtVJonplVsDA= +cloud.google.com/go/dataqna v0.7.0/go.mod h1:Lx9OcIIeqCrw1a6KdO3/5KMP1wAmTc0slZWwP12Qq3c= +cloud.google.com/go/dataqna v0.8.1/go.mod h1:zxZM0Bl6liMePWsHA8RMGAfmTG34vJMapbHAxQ5+WA8= cloud.google.com/go/datastore v1.0.0/go.mod h1:LXYbyblFSglQ5pkeyhO+Qmw7ukd3C+pD7TKLgZqpHYE= cloud.google.com/go/datastore v1.1.0/go.mod h1:umbIZjpQpHh4hmRpGhH4tLFup+FVzqBi1b3c64qFpCk= +cloud.google.com/go/datastore v1.10.0/go.mod h1:PC5UzAmDEkAmkfaknstTYbNpgE49HAgW2J1gcgUfmdM= +cloud.google.com/go/datastore v1.11.0/go.mod h1:TvGxBIHCS50u8jzG+AW/ppf87v1of8nwzFNgEZU1D3c= +cloud.google.com/go/datastore v1.12.0/go.mod h1:KjdB88W897MRITkvWWJrg2OUtrR5XVj1EoLgSp6/N70= +cloud.google.com/go/datastore v1.12.1/go.mod h1:KjdB88W897MRITkvWWJrg2OUtrR5XVj1EoLgSp6/N70= +cloud.google.com/go/datastore v1.13.0/go.mod h1:KjdB88W897MRITkvWWJrg2OUtrR5XVj1EoLgSp6/N70= +cloud.google.com/go/datastore v1.14.0/go.mod h1:GAeStMBIt9bPS7jMJA85kgkpsMkvseWWXiaHya9Jes8= +cloud.google.com/go/datastore v1.15.0/go.mod h1:GAeStMBIt9bPS7jMJA85kgkpsMkvseWWXiaHya9Jes8= +cloud.google.com/go/datastream v1.2.0/go.mod h1:i/uTP8/fZwgATHS/XFu0TcNUhuA0twZxxQ3EyCUQMwo= +cloud.google.com/go/datastream v1.3.0/go.mod h1:cqlOX8xlyYF/uxhiKn6Hbv6WjwPPuI9W2M9SAXwaLLQ= +cloud.google.com/go/datastream v1.4.0/go.mod h1:h9dpzScPhDTs5noEMQVWP8Wx8AFBRyS0s8KWPx/9r0g= +cloud.google.com/go/datastream v1.5.0/go.mod h1:6TZMMNPwjUqZHBKPQ1wwXpb0d5VDVPl2/XoS5yi88q4= +cloud.google.com/go/datastream v1.6.0/go.mod h1:6LQSuswqLa7S4rPAOZFVjHIG3wJIjZcZrw8JDEDJuIs= +cloud.google.com/go/datastream v1.7.0/go.mod h1:uxVRMm2elUSPuh65IbZpzJNMbuzkcvu5CjMqVIUHrww= +cloud.google.com/go/datastream v1.9.1/go.mod h1:hqnmr8kdUBmrnk65k5wNRoHSCYksvpdZIcZIEl8h43Q= +cloud.google.com/go/datastream v1.10.0/go.mod h1:hqnmr8kdUBmrnk65k5wNRoHSCYksvpdZIcZIEl8h43Q= +cloud.google.com/go/deploy v1.4.0/go.mod h1:5Xghikd4VrmMLNaF6FiRFDlHb59VM59YoDQnOUdsH/c= +cloud.google.com/go/deploy v1.5.0/go.mod h1:ffgdD0B89tToyW/U/D2eL0jN2+IEV/3EMuXHA0l4r+s= +cloud.google.com/go/deploy v1.6.0/go.mod h1:f9PTHehG/DjCom3QH0cntOVRm93uGBDt2vKzAPwpXQI= +cloud.google.com/go/deploy v1.8.0/go.mod h1:z3myEJnA/2wnB4sgjqdMfgxCA0EqC3RBTNcVPs93mtQ= +cloud.google.com/go/deploy v1.11.0/go.mod h1:tKuSUV5pXbn67KiubiUNUejqLs4f5cxxiCNCeyl0F2g= +cloud.google.com/go/deploy v1.13.0/go.mod h1:tKuSUV5pXbn67KiubiUNUejqLs4f5cxxiCNCeyl0F2g= +cloud.google.com/go/dialogflow v1.15.0/go.mod h1:HbHDWs33WOGJgn6rfzBW1Kv807BE3O1+xGbn59zZWI4= +cloud.google.com/go/dialogflow v1.16.1/go.mod h1:po6LlzGfK+smoSmTBnbkIZY2w8ffjz/RcGSS+sh1el0= +cloud.google.com/go/dialogflow v1.17.0/go.mod h1:YNP09C/kXA1aZdBgC/VtXX74G/TKn7XVCcVumTflA+8= +cloud.google.com/go/dialogflow v1.18.0/go.mod h1:trO7Zu5YdyEuR+BhSNOqJezyFQ3aUzz0njv7sMx/iek= +cloud.google.com/go/dialogflow v1.19.0/go.mod h1:JVmlG1TwykZDtxtTXujec4tQ+D8SBFMoosgy+6Gn0s0= +cloud.google.com/go/dialogflow v1.29.0/go.mod h1:b+2bzMe+k1s9V+F2jbJwpHPzrnIyHihAdRFMtn2WXuM= +cloud.google.com/go/dialogflow v1.31.0/go.mod h1:cuoUccuL1Z+HADhyIA7dci3N5zUssgpBJmCzI6fNRB4= +cloud.google.com/go/dialogflow v1.32.0/go.mod h1:jG9TRJl8CKrDhMEcvfcfFkkpp8ZhgPz3sBGmAUYJ2qE= +cloud.google.com/go/dialogflow v1.38.0/go.mod h1:L7jnH+JL2mtmdChzAIcXQHXMvQkE3U4hTaNltEuxXn4= +cloud.google.com/go/dialogflow v1.40.0/go.mod h1:L7jnH+JL2mtmdChzAIcXQHXMvQkE3U4hTaNltEuxXn4= +cloud.google.com/go/dialogflow v1.43.0/go.mod h1:pDUJdi4elL0MFmt1REMvFkdsUTYSHq+rTCS8wg0S3+M= +cloud.google.com/go/dialogflow v1.44.0/go.mod h1:pDUJdi4elL0MFmt1REMvFkdsUTYSHq+rTCS8wg0S3+M= +cloud.google.com/go/dlp v1.6.0/go.mod h1:9eyB2xIhpU0sVwUixfBubDoRwP+GjeUoxxeueZmqvmM= +cloud.google.com/go/dlp v1.7.0/go.mod h1:68ak9vCiMBjbasxeVD17hVPxDEck+ExiHavX8kiHG+Q= +cloud.google.com/go/dlp v1.9.0/go.mod h1:qdgmqgTyReTz5/YNSSuueR8pl7hO0o9bQ39ZhtgkWp4= +cloud.google.com/go/dlp v1.10.1/go.mod h1:IM8BWz1iJd8njcNcG0+Kyd9OPnqnRNkDV8j42VT5KOI= +cloud.google.com/go/documentai v1.7.0/go.mod h1:lJvftZB5NRiFSX4moiye1SMxHx0Bc3x1+p9e/RfXYiU= +cloud.google.com/go/documentai v1.8.0/go.mod h1:xGHNEB7CtsnySCNrCFdCyyMz44RhFEEX2Q7UD0c5IhU= +cloud.google.com/go/documentai v1.9.0/go.mod h1:FS5485S8R00U10GhgBC0aNGrJxBP8ZVpEeJ7PQDZd6k= +cloud.google.com/go/documentai v1.10.0/go.mod h1:vod47hKQIPeCfN2QS/jULIvQTugbmdc0ZvxxfQY1bg4= +cloud.google.com/go/documentai v1.16.0/go.mod h1:o0o0DLTEZ+YnJZ+J4wNfTxmDVyrkzFvttBXXtYRMHkM= +cloud.google.com/go/documentai v1.18.0/go.mod h1:F6CK6iUH8J81FehpskRmhLq/3VlwQvb7TvwOceQ2tbs= +cloud.google.com/go/documentai v1.20.0/go.mod h1:yJkInoMcK0qNAEdRnqY/D5asy73tnPe88I1YTZT+a8E= +cloud.google.com/go/documentai v1.22.0/go.mod h1:yJkInoMcK0qNAEdRnqY/D5asy73tnPe88I1YTZT+a8E= +cloud.google.com/go/documentai v1.22.1/go.mod h1:LKs22aDHbJv7ufXuPypzRO7rG3ALLJxzdCXDPutw4Qc= +cloud.google.com/go/documentai v1.23.0/go.mod h1:LKs22aDHbJv7ufXuPypzRO7rG3ALLJxzdCXDPutw4Qc= +cloud.google.com/go/domains v0.6.0/go.mod h1:T9Rz3GasrpYk6mEGHh4rymIhjlnIuB4ofT1wTxDeT4Y= +cloud.google.com/go/domains v0.7.0/go.mod h1:PtZeqS1xjnXuRPKE/88Iru/LdfoRyEHYA9nFQf4UKpg= +cloud.google.com/go/domains v0.8.0/go.mod h1:M9i3MMDzGFXsydri9/vW+EWz9sWb4I6WyHqdlAk0idE= +cloud.google.com/go/domains v0.9.1/go.mod h1:aOp1c0MbejQQ2Pjf1iJvnVyT+z6R6s8pX66KaCSDYfE= +cloud.google.com/go/edgecontainer v0.1.0/go.mod h1:WgkZ9tp10bFxqO8BLPqv2LlfmQF1X8lZqwW4r1BTajk= +cloud.google.com/go/edgecontainer v0.2.0/go.mod h1:RTmLijy+lGpQ7BXuTDa4C4ssxyXT34NIuHIgKuP4s5w= +cloud.google.com/go/edgecontainer v0.3.0/go.mod h1:FLDpP4nykgwwIfcLt6zInhprzw0lEi2P1fjO6Ie0qbc= +cloud.google.com/go/edgecontainer v1.0.0/go.mod h1:cttArqZpBB2q58W/upSG++ooo6EsblxDIolxa3jSjbY= +cloud.google.com/go/edgecontainer v1.1.1/go.mod h1:O5bYcS//7MELQZs3+7mabRqoWQhXCzenBu0R8bz2rwk= +cloud.google.com/go/errorreporting v0.3.0/go.mod h1:xsP2yaAp+OAW4OIm60An2bbLpqIhKXdWR/tawvl7QzU= +cloud.google.com/go/essentialcontacts v1.3.0/go.mod h1:r+OnHa5jfj90qIfZDO/VztSFqbQan7HV75p8sA+mdGI= +cloud.google.com/go/essentialcontacts v1.4.0/go.mod h1:8tRldvHYsmnBCHdFpvU+GL75oWiBKl80BiqlFh9tp+8= +cloud.google.com/go/essentialcontacts v1.5.0/go.mod h1:ay29Z4zODTuwliK7SnX8E86aUF2CTzdNtvv42niCX0M= +cloud.google.com/go/essentialcontacts v1.6.2/go.mod h1:T2tB6tX+TRak7i88Fb2N9Ok3PvY3UNbUsMag9/BARh4= +cloud.google.com/go/eventarc v1.7.0/go.mod h1:6ctpF3zTnaQCxUjHUdcfgcA1A2T309+omHZth7gDfmc= +cloud.google.com/go/eventarc v1.8.0/go.mod h1:imbzxkyAU4ubfsaKYdQg04WS1NvncblHEup4kvF+4gw= +cloud.google.com/go/eventarc v1.10.0/go.mod h1:u3R35tmZ9HvswGRBnF48IlYgYeBcPUCjkr4BTdem2Kw= +cloud.google.com/go/eventarc v1.11.0/go.mod h1:PyUjsUKPWoRBCHeOxZd/lbOOjahV41icXyUY5kSTvVY= +cloud.google.com/go/eventarc v1.12.1/go.mod h1:mAFCW6lukH5+IZjkvrEss+jmt2kOdYlN8aMx3sRJiAI= +cloud.google.com/go/eventarc v1.13.0/go.mod h1:mAFCW6lukH5+IZjkvrEss+jmt2kOdYlN8aMx3sRJiAI= +cloud.google.com/go/filestore v1.3.0/go.mod h1:+qbvHGvXU1HaKX2nD0WEPo92TP/8AQuCVEBXNY9z0+w= +cloud.google.com/go/filestore v1.4.0/go.mod h1:PaG5oDfo9r224f8OYXURtAsY+Fbyq/bLYoINEK8XQAI= +cloud.google.com/go/filestore v1.5.0/go.mod h1:FqBXDWBp4YLHqRnVGveOkHDf8svj9r5+mUDLupOWEDs= +cloud.google.com/go/filestore v1.6.0/go.mod h1:di5unNuss/qfZTw2U9nhFqo8/ZDSc466dre85Kydllg= +cloud.google.com/go/filestore v1.7.1/go.mod h1:y10jsorq40JJnjR/lQ8AfFbbcGlw3g+Dp8oN7i7FjV4= +cloud.google.com/go/firestore v1.9.0/go.mod h1:HMkjKHNTtRyZNiMzu7YAsLr9K3X2udY2AMwDaMEQiiE= +cloud.google.com/go/firestore v1.11.0/go.mod h1:b38dKhgzlmNNGTNZZwe7ZRFEuRab1Hay3/DBsIGKKy4= +cloud.google.com/go/firestore v1.12.0/go.mod h1:b38dKhgzlmNNGTNZZwe7ZRFEuRab1Hay3/DBsIGKKy4= +cloud.google.com/go/firestore v1.13.0/go.mod h1:QojqqOh8IntInDUSTAh0c8ZsPYAr68Ma8c5DWOy8xb8= +cloud.google.com/go/functions v1.6.0/go.mod h1:3H1UA3qiIPRWD7PeZKLvHZ9SaQhR26XIJcC0A5GbvAk= +cloud.google.com/go/functions v1.7.0/go.mod h1:+d+QBcWM+RsrgZfV9xo6KfA1GlzJfxcfZcRPEhDDfzg= +cloud.google.com/go/functions v1.8.0/go.mod h1:RTZ4/HsQjIqIYP9a9YPbU+QFoQsAlYgrwOXJWHn1POY= +cloud.google.com/go/functions v1.9.0/go.mod h1:Y+Dz8yGguzO3PpIjhLTbnqV1CWmgQ5UwtlpzoyquQ08= +cloud.google.com/go/functions v1.10.0/go.mod h1:0D3hEOe3DbEvCXtYOZHQZmD+SzYsi1YbI7dGvHfldXw= +cloud.google.com/go/functions v1.12.0/go.mod h1:AXWGrF3e2C/5ehvwYo/GH6O5s09tOPksiKhz+hH8WkA= +cloud.google.com/go/functions v1.13.0/go.mod h1:EU4O007sQm6Ef/PwRsI8N2umygGqPBS/IZQKBQBcJ3c= +cloud.google.com/go/functions v1.15.1/go.mod h1:P5yNWUTkyU+LvW/S9O6V+V423VZooALQlqoXdoPz5AE= +cloud.google.com/go/gaming v1.5.0/go.mod h1:ol7rGcxP/qHTRQE/RO4bxkXq+Fix0j6D4LFPzYTIrDM= +cloud.google.com/go/gaming v1.6.0/go.mod h1:YMU1GEvA39Qt3zWGyAVA9bpYz/yAhTvaQ1t2sK4KPUA= +cloud.google.com/go/gaming v1.7.0/go.mod h1:LrB8U7MHdGgFG851iHAfqUdLcKBdQ55hzXy9xBJz0+w= +cloud.google.com/go/gaming v1.8.0/go.mod h1:xAqjS8b7jAVW0KFYeRUxngo9My3f33kFmua++Pi+ggM= +cloud.google.com/go/gaming v1.9.0/go.mod h1:Fc7kEmCObylSWLO334NcO+O9QMDyz+TKC4v1D7X+Bc0= +cloud.google.com/go/gaming v1.10.1/go.mod h1:XQQvtfP8Rb9Rxnxm5wFVpAp9zCQkJi2bLIb7iHGwB3s= +cloud.google.com/go/gkebackup v0.2.0/go.mod h1:XKvv/4LfG829/B8B7xRkk8zRrOEbKtEam6yNfuQNH60= +cloud.google.com/go/gkebackup v0.3.0/go.mod h1:n/E671i1aOQvUxT541aTkCwExO/bTer2HDlj4TsBRAo= +cloud.google.com/go/gkebackup v0.4.0/go.mod h1:byAyBGUwYGEEww7xsbnUTBHIYcOPy/PgUWUtOeRm9Vg= +cloud.google.com/go/gkebackup v1.3.0/go.mod h1:vUDOu++N0U5qs4IhG1pcOnD1Mac79xWy6GoBFlWCWBU= +cloud.google.com/go/gkebackup v1.3.1/go.mod h1:vUDOu++N0U5qs4IhG1pcOnD1Mac79xWy6GoBFlWCWBU= +cloud.google.com/go/gkeconnect v0.5.0/go.mod h1:c5lsNAg5EwAy7fkqX/+goqFsU1Da/jQFqArp+wGNr/o= +cloud.google.com/go/gkeconnect v0.6.0/go.mod h1:Mln67KyU/sHJEBY8kFZ0xTeyPtzbq9StAVvEULYK16A= +cloud.google.com/go/gkeconnect v0.7.0/go.mod h1:SNfmVqPkaEi3bF/B3CNZOAYPYdg7sU+obZ+QTky2Myw= +cloud.google.com/go/gkeconnect v0.8.1/go.mod h1:KWiK1g9sDLZqhxB2xEuPV8V9NYzrqTUmQR9shJHpOZw= +cloud.google.com/go/gkehub v0.9.0/go.mod h1:WYHN6WG8w9bXU0hqNxt8rm5uxnk8IH+lPY9J2TV7BK0= +cloud.google.com/go/gkehub v0.10.0/go.mod h1:UIPwxI0DsrpsVoWpLB0stwKCP+WFVG9+y977wO+hBH0= +cloud.google.com/go/gkehub v0.11.0/go.mod h1:JOWHlmN+GHyIbuWQPl47/C2RFhnFKH38jH9Ascu3n0E= +cloud.google.com/go/gkehub v0.12.0/go.mod h1:djiIwwzTTBrF5NaXCGv3mf7klpEMcST17VBTVVDcuaw= +cloud.google.com/go/gkehub v0.14.1/go.mod h1:VEXKIJZ2avzrbd7u+zeMtW00Y8ddk/4V9511C9CQGTY= +cloud.google.com/go/gkemulticloud v0.3.0/go.mod h1:7orzy7O0S+5kq95e4Hpn7RysVA7dPs8W/GgfUtsPbrA= +cloud.google.com/go/gkemulticloud v0.4.0/go.mod h1:E9gxVBnseLWCk24ch+P9+B2CoDFJZTyIgLKSalC7tuI= +cloud.google.com/go/gkemulticloud v0.5.0/go.mod h1:W0JDkiyi3Tqh0TJr//y19wyb1yf8llHVto2Htf2Ja3Y= +cloud.google.com/go/gkemulticloud v0.6.1/go.mod h1:kbZ3HKyTsiwqKX7Yw56+wUGwwNZViRnxWK2DVknXWfw= +cloud.google.com/go/gkemulticloud v1.0.0/go.mod h1:kbZ3HKyTsiwqKX7Yw56+wUGwwNZViRnxWK2DVknXWfw= +cloud.google.com/go/grafeas v0.2.0/go.mod h1:KhxgtF2hb0P191HlY5besjYm6MqTSTj3LSI+M+ByZHc= +cloud.google.com/go/grafeas v0.3.0/go.mod h1:P7hgN24EyONOTMyeJH6DxG4zD7fwiYa5Q6GUgyFSOU8= +cloud.google.com/go/gsuiteaddons v1.3.0/go.mod h1:EUNK/J1lZEZO8yPtykKxLXI6JSVN2rg9bN8SXOa0bgM= +cloud.google.com/go/gsuiteaddons v1.4.0/go.mod h1:rZK5I8hht7u7HxFQcFei0+AtfS9uSushomRlg+3ua1o= +cloud.google.com/go/gsuiteaddons v1.5.0/go.mod h1:TFCClYLd64Eaa12sFVmUyG62tk4mdIsI7pAnSXRkcFo= +cloud.google.com/go/gsuiteaddons v1.6.1/go.mod h1:CodrdOqRZcLp5WOwejHWYBjZvfY0kOphkAKpF/3qdZY= +cloud.google.com/go/iam v0.1.0/go.mod h1:vcUNEa0pEm0qRVpmWepWaFMIAI8/hjB9mO8rNCJtF6c= +cloud.google.com/go/iam v0.3.0/go.mod h1:XzJPvDayI+9zsASAFO68Hk07u3z+f+JrT2xXNdp4bnY= +cloud.google.com/go/iam v0.5.0/go.mod h1:wPU9Vt0P4UmCux7mqtRu6jcpPAb74cP1fh50J3QpkUc= +cloud.google.com/go/iam v0.6.0/go.mod h1:+1AH33ueBne5MzYccyMHtEKqLE4/kJOibtffMHDMFMc= +cloud.google.com/go/iam v0.7.0/go.mod h1:H5Br8wRaDGNc8XP3keLc4unfUUZeyH3Sfl9XpQEYOeg= +cloud.google.com/go/iam v0.8.0/go.mod h1:lga0/y3iH6CX7sYqypWJ33hf7kkfXJag67naqGESjkE= +cloud.google.com/go/iam v0.11.0/go.mod h1:9PiLDanza5D+oWFZiH1uG+RnRCfEGKoyl6yo4cgWZGY= +cloud.google.com/go/iam v0.12.0/go.mod h1:knyHGviacl11zrtZUoDuYpDgLjvr28sLQaG0YB2GYAY= +cloud.google.com/go/iam v0.13.0/go.mod h1:ljOg+rcNfzZ5d6f1nAUJ8ZIxOaZUVoS14bKCtaLZ/D0= +cloud.google.com/go/iam v1.0.1/go.mod h1:yR3tmSL8BcZB4bxByRv2jkSIahVmCtfKZwLYGBalRE8= +cloud.google.com/go/iam v1.1.0/go.mod h1:nxdHjaKfCr7fNYx/HJMM8LgiMugmveWlkatear5gVyk= +cloud.google.com/go/iam v1.1.1/go.mod h1:A5avdyVL2tCppe4unb0951eI9jreack+RJ0/d+KUZOU= +cloud.google.com/go/iam v1.1.2/go.mod h1:A5avdyVL2tCppe4unb0951eI9jreack+RJ0/d+KUZOU= cloud.google.com/go/iam v1.1.6 h1:bEa06k05IO4f4uJonbB5iAgKTPpABy1ayxaIZV/GHVc= cloud.google.com/go/iam v1.1.6/go.mod h1:O0zxdPeGBoFdWW3HWmBxJsk0pfvNM/p/qa82rWOGTwI= +cloud.google.com/go/iap v1.4.0/go.mod h1:RGFwRJdihTINIe4wZ2iCP0zF/qu18ZwyKxrhMhygBEc= +cloud.google.com/go/iap v1.5.0/go.mod h1:UH/CGgKd4KyohZL5Pt0jSKE4m3FR51qg6FKQ/z/Ix9A= +cloud.google.com/go/iap v1.6.0/go.mod h1:NSuvI9C/j7UdjGjIde7t7HBz+QTwBcapPE07+sSRcLk= +cloud.google.com/go/iap v1.7.0/go.mod h1:beqQx56T9O1G1yNPph+spKpNibDlYIiIixiqsQXxLIo= +cloud.google.com/go/iap v1.7.1/go.mod h1:WapEwPc7ZxGt2jFGB/C/bm+hP0Y6NXzOYGjpPnmMS74= +cloud.google.com/go/iap v1.8.1/go.mod h1:sJCbeqg3mvWLqjZNsI6dfAtbbV1DL2Rl7e1mTyXYREQ= +cloud.google.com/go/iap v1.9.0/go.mod h1:01OFxd1R+NFrg78S+hoPV5PxEzv22HXaNqUUlmNHFuY= +cloud.google.com/go/ids v1.1.0/go.mod h1:WIuwCaYVOzHIj2OhN9HAwvW+DBdmUAdcWlFxRl+KubM= +cloud.google.com/go/ids v1.2.0/go.mod h1:5WXvp4n25S0rA/mQWAg1YEEBBq6/s+7ml1RDCW1IrcY= +cloud.google.com/go/ids v1.3.0/go.mod h1:JBdTYwANikFKaDP6LtW5JAi4gubs57SVNQjemdt6xV4= +cloud.google.com/go/ids v1.4.1/go.mod h1:np41ed8YMU8zOgv53MMMoCntLTn2lF+SUzlM+O3u/jw= +cloud.google.com/go/iot v1.3.0/go.mod h1:r7RGh2B61+B8oz0AGE+J72AhA0G7tdXItODWsaA2oLs= +cloud.google.com/go/iot v1.4.0/go.mod h1:dIDxPOn0UvNDUMD8Ger7FIaTuvMkj+aGk94RPP0iV+g= +cloud.google.com/go/iot v1.5.0/go.mod h1:mpz5259PDl3XJthEmh9+ap0affn/MqNSP4My77Qql9o= +cloud.google.com/go/iot v1.6.0/go.mod h1:IqdAsmE2cTYYNO1Fvjfzo9po179rAtJeVGUvkLN3rLE= +cloud.google.com/go/iot v1.7.1/go.mod h1:46Mgw7ev1k9KqK1ao0ayW9h0lI+3hxeanz+L1zmbbbk= +cloud.google.com/go/kms v1.4.0/go.mod h1:fajBHndQ+6ubNw6Ss2sSd+SWvjL26RNo/dr7uxsnnOA= +cloud.google.com/go/kms v1.5.0/go.mod h1:QJS2YY0eJGBg3mnDfuaCyLauWwBJiHRboYxJ++1xJNg= +cloud.google.com/go/kms v1.6.0/go.mod h1:Jjy850yySiasBUDi6KFUwUv2n1+o7QZFyuUJg6OgjA0= +cloud.google.com/go/kms v1.8.0/go.mod h1:4xFEhYFqvW+4VMELtZyxomGSYtSQKzM178ylFW4jMAg= +cloud.google.com/go/kms v1.9.0/go.mod h1:qb1tPTgfF9RQP8e1wq4cLFErVuTJv7UsSC915J8dh3w= +cloud.google.com/go/kms v1.10.0/go.mod h1:ng3KTUtQQU9bPX3+QGLsflZIHlkbn8amFAMY63m8d24= +cloud.google.com/go/kms v1.10.1/go.mod h1:rIWk/TryCkR59GMC3YtHtXeLzd634lBbKenvyySAyYI= +cloud.google.com/go/kms v1.11.0/go.mod h1:hwdiYC0xjnWsKQQCQQmIQnS9asjYVSK6jtXm+zFqXLM= +cloud.google.com/go/kms v1.12.1/go.mod h1:c9J991h5DTl+kg7gi3MYomh12YEENGrf48ee/N/2CDM= +cloud.google.com/go/kms v1.15.0/go.mod h1:c9J991h5DTl+kg7gi3MYomh12YEENGrf48ee/N/2CDM= +cloud.google.com/go/kms v1.15.2/go.mod h1:3hopT4+7ooWRCjc2DxgnpESFxhIraaI2IpAVUEhbT/w= cloud.google.com/go/kms v1.15.7 h1:7caV9K3yIxvlQPAcaFffhlT7d1qpxjB1wHBtjWa13SM= cloud.google.com/go/kms v1.15.7/go.mod h1:ub54lbsa6tDkUwnu4W7Yt1aAIFLnspgh0kPGToDukeI= +cloud.google.com/go/language v1.4.0/go.mod h1:F9dRpNFQmJbkaop6g0JhSBXCNlO90e1KWx5iDdxbWic= +cloud.google.com/go/language v1.6.0/go.mod h1:6dJ8t3B+lUYfStgls25GusK04NLh3eDLQnWM3mdEbhI= +cloud.google.com/go/language v1.7.0/go.mod h1:DJ6dYN/W+SQOjF8e1hLQXMF21AkH2w9wiPzPCJa2MIE= +cloud.google.com/go/language v1.8.0/go.mod h1:qYPVHf7SPoNNiCL2Dr0FfEFNil1qi3pQEyygwpgVKB8= +cloud.google.com/go/language v1.9.0/go.mod h1:Ns15WooPM5Ad/5no/0n81yUetis74g3zrbeJBE+ptUY= +cloud.google.com/go/language v1.10.1/go.mod h1:CPp94nsdVNiQEt1CNjF5WkTcisLiHPyIbMhvR8H2AW0= +cloud.google.com/go/language v1.11.0/go.mod h1:uDx+pFDdAKTY8ehpWbiXyQdz8tDSYLJbQcXsCkjYyvQ= +cloud.google.com/go/lifesciences v0.5.0/go.mod h1:3oIKy8ycWGPUyZDR/8RNnTOYevhaMLqh5vLUXs9zvT8= +cloud.google.com/go/lifesciences v0.6.0/go.mod h1:ddj6tSX/7BOnhxCSd3ZcETvtNr8NZ6t/iPhY2Tyfu08= +cloud.google.com/go/lifesciences v0.8.0/go.mod h1:lFxiEOMqII6XggGbOnKiyZ7IBwoIqA84ClvoezaA/bo= +cloud.google.com/go/lifesciences v0.9.1/go.mod h1:hACAOd1fFbCGLr/+weUKRAJas82Y4vrL3O5326N//Wc= +cloud.google.com/go/logging v1.6.1/go.mod h1:5ZO0mHHbvm8gEmeEUHrmDlTDSu5imF6MUP9OfilNXBw= +cloud.google.com/go/logging v1.7.0/go.mod h1:3xjP2CjkM3ZkO73aj4ASA5wRPGGCRrPIAeNqVNkzY8M= +cloud.google.com/go/logging v1.8.1/go.mod h1:TJjR+SimHwuC8MZ9cjByQulAMgni+RkXeI3wwctHJEI= +cloud.google.com/go/longrunning v0.1.1/go.mod h1:UUFxuDWkv22EuY93jjmDMFT5GPQKeFVJBIF6QlTqdsE= +cloud.google.com/go/longrunning v0.3.0/go.mod h1:qth9Y41RRSUE69rDcOn6DdK3HfQfsUI0YSmW3iIlLJc= +cloud.google.com/go/longrunning v0.4.1/go.mod h1:4iWDqhBZ70CvZ6BfETbvam3T8FMvLK+eFj0E6AaRQTo= +cloud.google.com/go/longrunning v0.4.2/go.mod h1:OHrnaYyLUV6oqwh0xiS7e5sLQhP1m0QU9R+WhGDMgIQ= +cloud.google.com/go/longrunning v0.5.0/go.mod h1:0JNuqRShmscVAhIACGtskSAWtqtOoPkwP0YF1oVEchc= +cloud.google.com/go/longrunning v0.5.1/go.mod h1:spvimkwdz6SPWKEt/XBij79E9fiTkHSQl/fRUUQJYJc= +cloud.google.com/go/managedidentities v1.3.0/go.mod h1:UzlW3cBOiPrzucO5qWkNkh0w33KFtBJU281hacNvsdE= +cloud.google.com/go/managedidentities v1.4.0/go.mod h1:NWSBYbEMgqmbZsLIyKvxrYbtqOsxY1ZrGM+9RgDqInM= +cloud.google.com/go/managedidentities v1.5.0/go.mod h1:+dWcZ0JlUmpuxpIDfyP5pP5y0bLdRwOS4Lp7gMni/LA= +cloud.google.com/go/managedidentities v1.6.1/go.mod h1:h/irGhTN2SkZ64F43tfGPMbHnypMbu4RB3yl8YcuEak= +cloud.google.com/go/maps v0.1.0/go.mod h1:BQM97WGyfw9FWEmQMpZ5T6cpovXXSd1cGmFma94eubI= +cloud.google.com/go/maps v0.6.0/go.mod h1:o6DAMMfb+aINHz/p/jbcY+mYeXBoZoxTfdSQ8VAJaCw= +cloud.google.com/go/maps v0.7.0/go.mod h1:3GnvVl3cqeSvgMcpRlQidXsPYuDGQ8naBis7MVzpXsY= +cloud.google.com/go/maps v1.3.0/go.mod h1:6mWTUv+WhnOwAgjVsSW2QPPECmW+s3PcRyOa9vgG/5s= +cloud.google.com/go/maps v1.4.0/go.mod h1:6mWTUv+WhnOwAgjVsSW2QPPECmW+s3PcRyOa9vgG/5s= +cloud.google.com/go/mediatranslation v0.5.0/go.mod h1:jGPUhGTybqsPQn91pNXw0xVHfuJ3leR1wj37oU3y1f4= +cloud.google.com/go/mediatranslation v0.6.0/go.mod h1:hHdBCTYNigsBxshbznuIMFNe5QXEowAuNmmC7h8pu5w= +cloud.google.com/go/mediatranslation v0.7.0/go.mod h1:LCnB/gZr90ONOIQLgSXagp8XUW1ODs2UmUMvcgMfI2I= +cloud.google.com/go/mediatranslation v0.8.1/go.mod h1:L/7hBdEYbYHQJhX2sldtTO5SZZ1C1vkapubj0T2aGig= +cloud.google.com/go/memcache v1.4.0/go.mod h1:rTOfiGZtJX1AaFUrOgsMHX5kAzaTQ8azHiuDoTPzNsE= +cloud.google.com/go/memcache v1.5.0/go.mod h1:dk3fCK7dVo0cUU2c36jKb4VqKPS22BTkf81Xq617aWM= +cloud.google.com/go/memcache v1.6.0/go.mod h1:XS5xB0eQZdHtTuTF9Hf8eJkKtR3pVRCcvJwtm68T3rA= +cloud.google.com/go/memcache v1.7.0/go.mod h1:ywMKfjWhNtkQTxrWxCkCFkoPjLHPW6A7WOTVI8xy3LY= +cloud.google.com/go/memcache v1.9.0/go.mod h1:8oEyzXCu+zo9RzlEaEjHl4KkgjlNDaXbCQeQWlzNFJM= +cloud.google.com/go/memcache v1.10.1/go.mod h1:47YRQIarv4I3QS5+hoETgKO40InqzLP6kpNLvyXuyaA= +cloud.google.com/go/metastore v1.5.0/go.mod h1:2ZNrDcQwghfdtCwJ33nM0+GrBGlVuh8rakL3vdPY3XY= +cloud.google.com/go/metastore v1.6.0/go.mod h1:6cyQTls8CWXzk45G55x57DVQ9gWg7RiH65+YgPsNh9s= +cloud.google.com/go/metastore v1.7.0/go.mod h1:s45D0B4IlsINu87/AsWiEVYbLaIMeUSoxlKKDqBGFS8= +cloud.google.com/go/metastore v1.8.0/go.mod h1:zHiMc4ZUpBiM7twCIFQmJ9JMEkDSyZS9U12uf7wHqSI= +cloud.google.com/go/metastore v1.10.0/go.mod h1:fPEnH3g4JJAk+gMRnrAnoqyv2lpUCqJPWOodSaf45Eo= +cloud.google.com/go/metastore v1.11.1/go.mod h1:uZuSo80U3Wd4zi6C22ZZliOUJ3XeM/MlYi/z5OAOWRA= +cloud.google.com/go/metastore v1.12.0/go.mod h1:uZuSo80U3Wd4zi6C22ZZliOUJ3XeM/MlYi/z5OAOWRA= +cloud.google.com/go/metastore v1.13.0/go.mod h1:URDhpG6XLeh5K+Glq0NOt74OfrPKTwS62gEPZzb5SOk= +cloud.google.com/go/monitoring v1.7.0/go.mod h1:HpYse6kkGo//7p6sT0wsIC6IBDET0RhIsnmlA53dvEk= +cloud.google.com/go/monitoring v1.8.0/go.mod h1:E7PtoMJ1kQXWxPjB6mv2fhC5/15jInuulFdYYtlcvT4= +cloud.google.com/go/monitoring v1.12.0/go.mod h1:yx8Jj2fZNEkL/GYZyTLS4ZtZEZN8WtDEiEqG4kLK50w= +cloud.google.com/go/monitoring v1.13.0/go.mod h1:k2yMBAB1H9JT/QETjNkgdCGD9bPF712XiLTVr+cBrpw= +cloud.google.com/go/monitoring v1.15.1/go.mod h1:lADlSAlFdbqQuwwpaImhsJXu1QSdd3ojypXrFSMr2rM= +cloud.google.com/go/monitoring v1.16.0/go.mod h1:Ptp15HgAyM1fNICAojDMoNc/wUmn67mLHQfyqbw+poY= +cloud.google.com/go/networkconnectivity v1.4.0/go.mod h1:nOl7YL8odKyAOtzNX73/M5/mGZgqqMeryi6UPZTk/rA= +cloud.google.com/go/networkconnectivity v1.5.0/go.mod h1:3GzqJx7uhtlM3kln0+x5wyFvuVH1pIBJjhCpjzSt75o= +cloud.google.com/go/networkconnectivity v1.6.0/go.mod h1:OJOoEXW+0LAxHh89nXd64uGG+FbQoeH8DtxCHVOMlaM= +cloud.google.com/go/networkconnectivity v1.7.0/go.mod h1:RMuSbkdbPwNMQjB5HBWD5MpTBnNm39iAVpC3TmsExt8= +cloud.google.com/go/networkconnectivity v1.10.0/go.mod h1:UP4O4sWXJG13AqrTdQCD9TnLGEbtNRqjuaaA7bNjF5E= +cloud.google.com/go/networkconnectivity v1.11.0/go.mod h1:iWmDD4QF16VCDLXUqvyspJjIEtBR/4zq5hwnY2X3scM= +cloud.google.com/go/networkconnectivity v1.12.1/go.mod h1:PelxSWYM7Sh9/guf8CFhi6vIqf19Ir/sbfZRUwXh92E= +cloud.google.com/go/networkconnectivity v1.13.0/go.mod h1:SAnGPes88pl7QRLUen2HmcBSE9AowVAcdug8c0RSBFk= +cloud.google.com/go/networkconnectivity v1.14.0/go.mod h1:SAnGPes88pl7QRLUen2HmcBSE9AowVAcdug8c0RSBFk= +cloud.google.com/go/networkmanagement v1.4.0/go.mod h1:Q9mdLLRn60AsOrPc8rs8iNV6OHXaGcDdsIQe1ohekq8= +cloud.google.com/go/networkmanagement v1.5.0/go.mod h1:ZnOeZ/evzUdUsnvRt792H0uYEnHQEMaz+REhhzJRcf4= +cloud.google.com/go/networkmanagement v1.6.0/go.mod h1:5pKPqyXjB/sgtvB5xqOemumoQNB7y95Q7S+4rjSOPYY= +cloud.google.com/go/networkmanagement v1.8.0/go.mod h1:Ho/BUGmtyEqrttTgWEe7m+8vDdK74ibQc+Be0q7Fof0= +cloud.google.com/go/networkmanagement v1.9.0/go.mod h1:UTUaEU9YwbCAhhz3jEOHr+2/K/MrBk2XxOLS89LQzFw= +cloud.google.com/go/networksecurity v0.5.0/go.mod h1:xS6fOCoqpVC5zx15Z/MqkfDwH4+m/61A3ODiDV1xmiQ= +cloud.google.com/go/networksecurity v0.6.0/go.mod h1:Q5fjhTr9WMI5mbpRYEbiexTzROf7ZbDzvzCrNl14nyU= +cloud.google.com/go/networksecurity v0.7.0/go.mod h1:mAnzoxx/8TBSyXEeESMy9OOYwo1v+gZ5eMRnsT5bC8k= +cloud.google.com/go/networksecurity v0.8.0/go.mod h1:B78DkqsxFG5zRSVuwYFRZ9Xz8IcQ5iECsNrPn74hKHU= +cloud.google.com/go/networksecurity v0.9.1/go.mod h1:MCMdxOKQ30wsBI1eI659f9kEp4wuuAueoC9AJKSPWZQ= +cloud.google.com/go/notebooks v1.2.0/go.mod h1:9+wtppMfVPUeJ8fIWPOq1UnATHISkGXGqTkxeieQ6UY= +cloud.google.com/go/notebooks v1.3.0/go.mod h1:bFR5lj07DtCPC7YAAJ//vHskFBxA5JzYlH68kXVdk34= +cloud.google.com/go/notebooks v1.4.0/go.mod h1:4QPMngcwmgb6uw7Po99B2xv5ufVoIQ7nOGDyL4P8AgA= +cloud.google.com/go/notebooks v1.5.0/go.mod h1:q8mwhnP9aR8Hpfnrc5iN5IBhrXUy8S2vuYs+kBJ/gu0= +cloud.google.com/go/notebooks v1.7.0/go.mod h1:PVlaDGfJgj1fl1S3dUwhFMXFgfYGhYQt2164xOMONmE= +cloud.google.com/go/notebooks v1.8.0/go.mod h1:Lq6dYKOYOWUCTvw5t2q1gp1lAp0zxAxRycayS0iJcqQ= +cloud.google.com/go/notebooks v1.9.1/go.mod h1:zqG9/gk05JrzgBt4ghLzEepPHNwE5jgPcHZRKhlC1A8= +cloud.google.com/go/notebooks v1.10.0/go.mod h1:SOPYMZnttHxqot0SGSFSkRrwE29eqnKPBJFqgWmiK2k= +cloud.google.com/go/optimization v1.1.0/go.mod h1:5po+wfvX5AQlPznyVEZjGJTMr4+CAkJf2XSTQOOl9l4= +cloud.google.com/go/optimization v1.2.0/go.mod h1:Lr7SOHdRDENsh+WXVmQhQTrzdu9ybg0NecjHidBq6xs= +cloud.google.com/go/optimization v1.3.1/go.mod h1:IvUSefKiwd1a5p0RgHDbWCIbDFgKuEdB+fPPuP0IDLI= +cloud.google.com/go/optimization v1.4.1/go.mod h1:j64vZQP7h9bO49m2rVaTVoNM0vEBEN5eKPUPbZyXOrk= +cloud.google.com/go/optimization v1.5.0/go.mod h1:evo1OvTxeBRBu6ydPlrIRizKY/LJKo/drDMMRKqGEUU= +cloud.google.com/go/orchestration v1.3.0/go.mod h1:Sj5tq/JpWiB//X/q3Ngwdl5K7B7Y0KZ7bfv0wL6fqVA= +cloud.google.com/go/orchestration v1.4.0/go.mod h1:6W5NLFWs2TlniBphAViZEVhrXRSMgUGDfW7vrWKvsBk= +cloud.google.com/go/orchestration v1.6.0/go.mod h1:M62Bevp7pkxStDfFfTuCOaXgaaqRAga1yKyoMtEoWPQ= +cloud.google.com/go/orchestration v1.8.1/go.mod h1:4sluRF3wgbYVRqz7zJ1/EUNc90TTprliq9477fGobD8= +cloud.google.com/go/orgpolicy v1.4.0/go.mod h1:xrSLIV4RePWmP9P3tBl8S93lTmlAxjm06NSm2UTmKvE= +cloud.google.com/go/orgpolicy v1.5.0/go.mod h1:hZEc5q3wzwXJaKrsx5+Ewg0u1LxJ51nNFlext7Tanwc= +cloud.google.com/go/orgpolicy v1.10.0/go.mod h1:w1fo8b7rRqlXlIJbVhOMPrwVljyuW5mqssvBtU18ONc= +cloud.google.com/go/orgpolicy v1.11.0/go.mod h1:2RK748+FtVvnfuynxBzdnyu7sygtoZa1za/0ZfpOs1M= +cloud.google.com/go/orgpolicy v1.11.1/go.mod h1:8+E3jQcpZJQliP+zaFfayC2Pg5bmhuLK755wKhIIUCE= +cloud.google.com/go/osconfig v1.7.0/go.mod h1:oVHeCeZELfJP7XLxcBGTMBvRO+1nQ5tFG9VQTmYS2Fs= +cloud.google.com/go/osconfig v1.8.0/go.mod h1:EQqZLu5w5XA7eKizepumcvWx+m8mJUhEwiPqWiZeEdg= +cloud.google.com/go/osconfig v1.9.0/go.mod h1:Yx+IeIZJ3bdWmzbQU4fxNl8xsZ4amB+dygAwFPlvnNo= +cloud.google.com/go/osconfig v1.10.0/go.mod h1:uMhCzqC5I8zfD9zDEAfvgVhDS8oIjySWh+l4WK6GnWw= +cloud.google.com/go/osconfig v1.11.0/go.mod h1:aDICxrur2ogRd9zY5ytBLV89KEgT2MKB2L/n6x1ooPw= +cloud.google.com/go/osconfig v1.12.0/go.mod h1:8f/PaYzoS3JMVfdfTubkowZYGmAhUCjjwnjqWI7NVBc= +cloud.google.com/go/osconfig v1.12.1/go.mod h1:4CjBxND0gswz2gfYRCUoUzCm9zCABp91EeTtWXyz0tE= +cloud.google.com/go/oslogin v1.4.0/go.mod h1:YdgMXWRaElXz/lDk1Na6Fh5orF7gvmJ0FGLIs9LId4E= +cloud.google.com/go/oslogin v1.5.0/go.mod h1:D260Qj11W2qx/HVF29zBg+0fd6YCSjSqLUkY/qEenQU= +cloud.google.com/go/oslogin v1.6.0/go.mod h1:zOJ1O3+dTU8WPlGEkFSh7qeHPPSoxrcMbbK1Nm2iX70= +cloud.google.com/go/oslogin v1.7.0/go.mod h1:e04SN0xO1UNJ1M5GP0vzVBFicIe4O53FOfcixIqTyXo= +cloud.google.com/go/oslogin v1.9.0/go.mod h1:HNavntnH8nzrn8JCTT5fj18FuJLFJc4NaZJtBnQtKFs= +cloud.google.com/go/oslogin v1.10.1/go.mod h1:x692z7yAue5nE7CsSnoG0aaMbNoRJRXO4sn73R+ZqAs= +cloud.google.com/go/oslogin v1.11.0/go.mod h1:8GMTJs4X2nOAUVJiPGqIWVcDaF0eniEto3xlOxaboXE= +cloud.google.com/go/phishingprotection v0.5.0/go.mod h1:Y3HZknsK9bc9dMi+oE8Bim0lczMU6hrX0UpADuMefr0= +cloud.google.com/go/phishingprotection v0.6.0/go.mod h1:9Y3LBLgy0kDTcYET8ZH3bq/7qni15yVUoAxiFxnlSUA= +cloud.google.com/go/phishingprotection v0.7.0/go.mod h1:8qJI4QKHoda/sb/7/YmMQ2omRLSLYSu9bU0EKCNI+Lk= +cloud.google.com/go/phishingprotection v0.8.1/go.mod h1:AxonW7GovcA8qdEk13NfHq9hNx5KPtfxXNeUxTDxB6I= +cloud.google.com/go/policytroubleshooter v1.3.0/go.mod h1:qy0+VwANja+kKrjlQuOzmlvscn4RNsAc0e15GGqfMxg= +cloud.google.com/go/policytroubleshooter v1.4.0/go.mod h1:DZT4BcRw3QoO8ota9xw/LKtPa8lKeCByYeKTIf/vxdE= +cloud.google.com/go/policytroubleshooter v1.5.0/go.mod h1:Rz1WfV+1oIpPdN2VvvuboLVRsB1Hclg3CKQ53j9l8vw= +cloud.google.com/go/policytroubleshooter v1.6.0/go.mod h1:zYqaPTsmfvpjm5ULxAyD/lINQxJ0DDsnWOP/GZ7xzBc= +cloud.google.com/go/policytroubleshooter v1.7.1/go.mod h1:0NaT5v3Ag1M7U5r0GfDCpUFkWd9YqpubBWsQlhanRv0= +cloud.google.com/go/policytroubleshooter v1.8.0/go.mod h1:tmn5Ir5EToWe384EuboTcVQT7nTag2+DuH3uHmKd1HU= +cloud.google.com/go/policytroubleshooter v1.9.0/go.mod h1:+E2Lga7TycpeSTj2FsH4oXxTnrbHJGRlKhVZBLGgU64= +cloud.google.com/go/privatecatalog v0.5.0/go.mod h1:XgosMUvvPyxDjAVNDYxJ7wBW8//hLDDYmnsNcMGq1K0= +cloud.google.com/go/privatecatalog v0.6.0/go.mod h1:i/fbkZR0hLN29eEWiiwue8Pb+GforiEIBnV9yrRUOKI= +cloud.google.com/go/privatecatalog v0.7.0/go.mod h1:2s5ssIFO69F5csTXcwBP7NPFTZvps26xGzvQ2PQaBYg= +cloud.google.com/go/privatecatalog v0.8.0/go.mod h1:nQ6pfaegeDAq/Q5lrfCQzQLhubPiZhSaNhIgfJlnIXs= +cloud.google.com/go/privatecatalog v0.9.1/go.mod h1:0XlDXW2unJXdf9zFz968Hp35gl/bhF4twwpXZAW50JA= cloud.google.com/go/pubsub v1.0.1/go.mod h1:R0Gpsv3s54REJCy4fxDixWD93lHJMoZTyQ2kNxGRt3I= cloud.google.com/go/pubsub v1.1.0/go.mod h1:EwwdRX2sKPjnvnqCa270oGRyludottCI76h+R3AArQw= cloud.google.com/go/pubsub v1.2.0/go.mod h1:jhfEVHT8odbXTkndysNHCcx0awwzvfOlguIAii9o8iA= cloud.google.com/go/pubsub v1.3.1/go.mod h1:i+ucay31+CNRpDW4Lu78I4xXG+O1r/MAHgjpRVR+TSU= +cloud.google.com/go/pubsub v1.26.0/go.mod h1:QgBH3U/jdJy/ftjPhTkyXNj543Tin1pRYcdcPRnFIRI= +cloud.google.com/go/pubsub v1.27.1/go.mod h1:hQN39ymbV9geqBnfQq6Xf63yNhUAhv9CZhzp5O6qsW0= +cloud.google.com/go/pubsub v1.28.0/go.mod h1:vuXFpwaVoIPQMGXqRyUQigu/AX1S3IWugR9xznmcXX8= +cloud.google.com/go/pubsub v1.30.0/go.mod h1:qWi1OPS0B+b5L+Sg6Gmc9zD1Y+HaM0MdUr7LsupY1P4= +cloud.google.com/go/pubsub v1.32.0/go.mod h1:f+w71I33OMyxf9VpMVcZbnG5KSUkCOUHYpFd5U1GdRc= +cloud.google.com/go/pubsub v1.33.0/go.mod h1:f+w71I33OMyxf9VpMVcZbnG5KSUkCOUHYpFd5U1GdRc= cloud.google.com/go/pubsub v1.36.1 h1:dfEPuGCHGbWUhaMCTHUFjfroILEkx55iUmKBZTP5f+Y= cloud.google.com/go/pubsub v1.36.1/go.mod h1:iYjCa9EzWOoBiTdd4ps7QoMtMln5NwaZQpK1hbRfBDE= +cloud.google.com/go/pubsublite v1.5.0/go.mod h1:xapqNQ1CuLfGi23Yda/9l4bBCKz/wC3KIJ5gKcxveZg= +cloud.google.com/go/pubsublite v1.6.0/go.mod h1:1eFCS0U11xlOuMFV/0iBqw3zP12kddMeCbj/F3FSj9k= +cloud.google.com/go/pubsublite v1.7.0/go.mod h1:8hVMwRXfDfvGm3fahVbtDbiLePT3gpoiJYJY+vxWxVM= +cloud.google.com/go/pubsublite v1.8.1/go.mod h1:fOLdU4f5xldK4RGJrBMm+J7zMWNj/k4PxwEZXy39QS0= +cloud.google.com/go/recaptchaenterprise v1.3.1/go.mod h1:OdD+q+y4XGeAlxRaMn1Y7/GveP6zmq76byL6tjPE7d4= +cloud.google.com/go/recaptchaenterprise/v2 v2.1.0/go.mod h1:w9yVqajwroDNTfGuhmOjPDN//rZGySaf6PtFVcSCa7o= +cloud.google.com/go/recaptchaenterprise/v2 v2.2.0/go.mod h1:/Zu5jisWGeERrd5HnlS3EUGb/D335f9k51B/FVil0jk= +cloud.google.com/go/recaptchaenterprise/v2 v2.3.0/go.mod h1:O9LwGCjrhGHBQET5CA7dd5NwwNQUErSgEDit1DLNTdo= +cloud.google.com/go/recaptchaenterprise/v2 v2.4.0/go.mod h1:Am3LHfOuBstrLrNCBrlI5sbwx9LBg3te2N6hGvHn2mE= +cloud.google.com/go/recaptchaenterprise/v2 v2.5.0/go.mod h1:O8LzcHXN3rz0j+LBC91jrwI3R+1ZSZEWrfL7XHgNo9U= +cloud.google.com/go/recaptchaenterprise/v2 v2.6.0/go.mod h1:RPauz9jeLtB3JVzg6nCbe12qNoaa8pXc4d/YukAmcnA= +cloud.google.com/go/recaptchaenterprise/v2 v2.7.0/go.mod h1:19wVj/fs5RtYtynAPJdDTb69oW0vNHYDBTbB4NvMD9c= +cloud.google.com/go/recaptchaenterprise/v2 v2.7.2/go.mod h1:kR0KjsJS7Jt1YSyWFkseQ756D45kaYNTlDPPaRAvDBU= +cloud.google.com/go/recaptchaenterprise/v2 v2.8.0/go.mod h1:QuE8EdU9dEnesG8/kG3XuJyNsjEqMlMzg3v3scCJ46c= +cloud.google.com/go/recommendationengine v0.5.0/go.mod h1:E5756pJcVFeVgaQv3WNpImkFP8a+RptV6dDLGPILjvg= +cloud.google.com/go/recommendationengine v0.6.0/go.mod h1:08mq2umu9oIqc7tDy8sx+MNJdLG0fUi3vaSVbztHgJ4= +cloud.google.com/go/recommendationengine v0.7.0/go.mod h1:1reUcE3GIu6MeBz/h5xZJqNLuuVjNg1lmWMPyjatzac= +cloud.google.com/go/recommendationengine v0.8.1/go.mod h1:MrZihWwtFYWDzE6Hz5nKcNz3gLizXVIDI/o3G1DLcrE= +cloud.google.com/go/recommender v1.5.0/go.mod h1:jdoeiBIVrJe9gQjwd759ecLJbxCDED4A6p+mqoqDvTg= +cloud.google.com/go/recommender v1.6.0/go.mod h1:+yETpm25mcoiECKh9DEScGzIRyDKpZ0cEhWGo+8bo+c= +cloud.google.com/go/recommender v1.7.0/go.mod h1:XLHs/W+T8olwlGOgfQenXBTbIseGclClff6lhFVe9Bs= +cloud.google.com/go/recommender v1.8.0/go.mod h1:PkjXrTT05BFKwxaUxQmtIlrtj0kph108r02ZZQ5FE70= +cloud.google.com/go/recommender v1.9.0/go.mod h1:PnSsnZY7q+VL1uax2JWkt/UegHssxjUVVCrX52CuEmQ= +cloud.google.com/go/recommender v1.10.1/go.mod h1:XFvrE4Suqn5Cq0Lf+mCP6oBHD/yRMA8XxP5sb7Q7gpA= +cloud.google.com/go/recommender v1.11.0/go.mod h1:kPiRQhPyTJ9kyXPCG6u/dlPLbYfFlkwHNRwdzPVAoII= +cloud.google.com/go/redis v1.7.0/go.mod h1:V3x5Jq1jzUcg+UNsRvdmsfuFnit1cfe3Z/PGyq/lm4Y= +cloud.google.com/go/redis v1.8.0/go.mod h1:Fm2szCDavWzBk2cDKxrkmWBqoCiL1+Ctwq7EyqBCA/A= +cloud.google.com/go/redis v1.9.0/go.mod h1:HMYQuajvb2D0LvMgZmLDZW8V5aOC/WxstZHiy4g8OiA= +cloud.google.com/go/redis v1.10.0/go.mod h1:ThJf3mMBQtW18JzGgh41/Wld6vnDDc/F/F35UolRZPM= +cloud.google.com/go/redis v1.11.0/go.mod h1:/X6eicana+BWcUda5PpwZC48o37SiFVTFSs0fWAJ7uQ= +cloud.google.com/go/redis v1.13.1/go.mod h1:VP7DGLpE91M6bcsDdMuyCm2hIpB6Vp2hI090Mfd1tcg= +cloud.google.com/go/resourcemanager v1.3.0/go.mod h1:bAtrTjZQFJkiWTPDb1WBjzvc6/kifjj4QBYuKCCoqKA= +cloud.google.com/go/resourcemanager v1.4.0/go.mod h1:MwxuzkumyTX7/a3n37gmsT3py7LIXwrShilPh3P1tR0= +cloud.google.com/go/resourcemanager v1.5.0/go.mod h1:eQoXNAiAvCf5PXxWxXjhKQoTMaUSNrEfg+6qdf/wots= +cloud.google.com/go/resourcemanager v1.6.0/go.mod h1:YcpXGRs8fDzcUl1Xw8uOVmI8JEadvhRIkoXXUNVYcVo= +cloud.google.com/go/resourcemanager v1.7.0/go.mod h1:HlD3m6+bwhzj9XCouqmeiGuni95NTrExfhoSrkC/3EI= +cloud.google.com/go/resourcemanager v1.9.1/go.mod h1:dVCuosgrh1tINZ/RwBufr8lULmWGOkPS8gL5gqyjdT8= +cloud.google.com/go/resourcesettings v1.3.0/go.mod h1:lzew8VfESA5DQ8gdlHwMrqZs1S9V87v3oCnKCWoOuQU= +cloud.google.com/go/resourcesettings v1.4.0/go.mod h1:ldiH9IJpcrlC3VSuCGvjR5of/ezRrOxFtpJoJo5SmXg= +cloud.google.com/go/resourcesettings v1.5.0/go.mod h1:+xJF7QSG6undsQDfsCJyqWXyBwUoJLhetkRMDRnIoXA= +cloud.google.com/go/resourcesettings v1.6.1/go.mod h1:M7mk9PIZrC5Fgsu1kZJci6mpgN8o0IUzVx3eJU3y4Jw= +cloud.google.com/go/retail v1.8.0/go.mod h1:QblKS8waDmNUhghY2TI9O3JLlFk8jybHeV4BF19FrE4= +cloud.google.com/go/retail v1.9.0/go.mod h1:g6jb6mKuCS1QKnH/dpu7isX253absFl6iE92nHwlBUY= +cloud.google.com/go/retail v1.10.0/go.mod h1:2gDk9HsL4HMS4oZwz6daui2/jmKvqShXKQuB2RZ+cCc= +cloud.google.com/go/retail v1.11.0/go.mod h1:MBLk1NaWPmh6iVFSz9MeKG/Psyd7TAgm6y/9L2B4x9Y= +cloud.google.com/go/retail v1.12.0/go.mod h1:UMkelN/0Z8XvKymXFbD4EhFJlYKRx1FGhQkVPU5kF14= +cloud.google.com/go/retail v1.14.1/go.mod h1:y3Wv3Vr2k54dLNIrCzenyKG8g8dhvhncT2NcNjb/6gE= +cloud.google.com/go/run v0.2.0/go.mod h1:CNtKsTA1sDcnqqIFR3Pb5Tq0usWxJJvsWOCPldRU3Do= +cloud.google.com/go/run v0.3.0/go.mod h1:TuyY1+taHxTjrD0ZFk2iAR+xyOXEA0ztb7U3UNA0zBo= +cloud.google.com/go/run v0.8.0/go.mod h1:VniEnuBwqjigv0A7ONfQUaEItaiCRVujlMqerPPiktM= +cloud.google.com/go/run v0.9.0/go.mod h1:Wwu+/vvg8Y+JUApMwEDfVfhetv30hCG4ZwDR/IXl2Qg= +cloud.google.com/go/run v1.2.0/go.mod h1:36V1IlDzQ0XxbQjUx6IYbw8H3TJnWvhii963WW3B/bo= +cloud.google.com/go/run v1.3.0/go.mod h1:S/osX/4jIPZGg+ssuqh6GNgg7syixKe3YnprwehzHKU= +cloud.google.com/go/scheduler v1.4.0/go.mod h1:drcJBmxF3aqZJRhmkHQ9b3uSSpQoltBPGPxGAWROx6s= +cloud.google.com/go/scheduler v1.5.0/go.mod h1:ri073ym49NW3AfT6DZi21vLZrG07GXr5p3H1KxN5QlI= +cloud.google.com/go/scheduler v1.6.0/go.mod h1:SgeKVM7MIwPn3BqtcBntpLyrIJftQISRrYB5ZtT+KOk= +cloud.google.com/go/scheduler v1.7.0/go.mod h1:jyCiBqWW956uBjjPMMuX09n3x37mtyPJegEWKxRsn44= +cloud.google.com/go/scheduler v1.8.0/go.mod h1:TCET+Y5Gp1YgHT8py4nlg2Sew8nUHMqcpousDgXJVQc= +cloud.google.com/go/scheduler v1.9.0/go.mod h1:yexg5t+KSmqu+njTIh3b7oYPheFtBWGcbVUYF1GGMIc= +cloud.google.com/go/scheduler v1.10.1/go.mod h1:R63Ldltd47Bs4gnhQkmNDse5w8gBRrhObZ54PxgR2Oo= +cloud.google.com/go/secretmanager v1.6.0/go.mod h1:awVa/OXF6IiyaU1wQ34inzQNc4ISIDIrId8qE5QGgKA= +cloud.google.com/go/secretmanager v1.8.0/go.mod h1:hnVgi/bN5MYHd3Gt0SPuTPPp5ENina1/LxM+2W9U9J4= +cloud.google.com/go/secretmanager v1.9.0/go.mod h1:b71qH2l1yHmWQHt9LC80akm86mX8AL6X1MA01dW8ht4= +cloud.google.com/go/secretmanager v1.10.0/go.mod h1:MfnrdvKMPNra9aZtQFvBcvRU54hbPD8/HayQdlUgJpU= +cloud.google.com/go/secretmanager v1.11.1/go.mod h1:znq9JlXgTNdBeQk9TBW/FnR/W4uChEKGeqQWAJ8SXFw= +cloud.google.com/go/security v1.5.0/go.mod h1:lgxGdyOKKjHL4YG3/YwIL2zLqMFCKs0UbQwgyZmfJl4= +cloud.google.com/go/security v1.7.0/go.mod h1:mZklORHl6Bg7CNnnjLH//0UlAlaXqiG7Lb9PsPXLfD0= +cloud.google.com/go/security v1.8.0/go.mod h1:hAQOwgmaHhztFhiQ41CjDODdWP0+AE1B3sX4OFlq+GU= +cloud.google.com/go/security v1.9.0/go.mod h1:6Ta1bO8LXI89nZnmnsZGp9lVoVWXqsVbIq/t9dzI+2Q= +cloud.google.com/go/security v1.10.0/go.mod h1:QtOMZByJVlibUT2h9afNDWRZ1G96gVywH8T5GUSb9IA= +cloud.google.com/go/security v1.12.0/go.mod h1:rV6EhrpbNHrrxqlvW0BWAIawFWq3X90SduMJdFwtLB8= +cloud.google.com/go/security v1.13.0/go.mod h1:Q1Nvxl1PAgmeW0y3HTt54JYIvUdtcpYKVfIB8AOMZ+0= +cloud.google.com/go/security v1.15.1/go.mod h1:MvTnnbsWnehoizHi09zoiZob0iCHVcL4AUBj76h9fXA= +cloud.google.com/go/securitycenter v1.13.0/go.mod h1:cv5qNAqjY84FCN6Y9z28WlkKXyWsgLO832YiWwkCWcU= +cloud.google.com/go/securitycenter v1.14.0/go.mod h1:gZLAhtyKv85n52XYWt6RmeBdydyxfPeTrpToDPw4Auc= +cloud.google.com/go/securitycenter v1.15.0/go.mod h1:PeKJ0t8MoFmmXLXWm41JidyzI3PJjd8sXWaVqg43WWk= +cloud.google.com/go/securitycenter v1.16.0/go.mod h1:Q9GMaLQFUD+5ZTabrbujNWLtSLZIZF7SAR0wWECrjdk= +cloud.google.com/go/securitycenter v1.18.1/go.mod h1:0/25gAzCM/9OL9vVx4ChPeM/+DlfGQJDwBy/UC8AKK0= +cloud.google.com/go/securitycenter v1.19.0/go.mod h1:LVLmSg8ZkkyaNy4u7HCIshAngSQ8EcIRREP3xBnyfag= +cloud.google.com/go/securitycenter v1.23.0/go.mod h1:8pwQ4n+Y9WCWM278R8W3nF65QtY172h4S8aXyI9/hsQ= +cloud.google.com/go/servicecontrol v1.4.0/go.mod h1:o0hUSJ1TXJAmi/7fLJAedOovnujSEvjKCAFNXPQ1RaU= +cloud.google.com/go/servicecontrol v1.5.0/go.mod h1:qM0CnXHhyqKVuiZnGKrIurvVImCs8gmqWsDoqe9sU1s= +cloud.google.com/go/servicecontrol v1.10.0/go.mod h1:pQvyvSRh7YzUF2efw7H87V92mxU8FnFDawMClGCNuAA= +cloud.google.com/go/servicecontrol v1.11.0/go.mod h1:kFmTzYzTUIuZs0ycVqRHNaNhgR+UMUpw9n02l/pY+mc= +cloud.google.com/go/servicecontrol v1.11.1/go.mod h1:aSnNNlwEFBY+PWGQ2DoM0JJ/QUXqV5/ZD9DOLB7SnUk= +cloud.google.com/go/servicedirectory v1.4.0/go.mod h1:gH1MUaZCgtP7qQiI+F+A+OpeKF/HQWgtAddhTbhL2bs= +cloud.google.com/go/servicedirectory v1.5.0/go.mod h1:QMKFL0NUySbpZJ1UZs3oFAmdvVxhhxB6eJ/Vlp73dfg= +cloud.google.com/go/servicedirectory v1.6.0/go.mod h1:pUlbnWsLH9c13yGkxCmfumWEPjsRs1RlmJ4pqiNjVL4= +cloud.google.com/go/servicedirectory v1.7.0/go.mod h1:5p/U5oyvgYGYejufvxhgwjL8UVXjkuw7q5XcG10wx1U= +cloud.google.com/go/servicedirectory v1.8.0/go.mod h1:srXodfhY1GFIPvltunswqXpVxFPpZjf8nkKQT7XcXaY= +cloud.google.com/go/servicedirectory v1.9.0/go.mod h1:29je5JjiygNYlmsGz8k6o+OZ8vd4f//bQLtvzkPPT/s= +cloud.google.com/go/servicedirectory v1.10.1/go.mod h1:Xv0YVH8s4pVOwfM/1eMTl0XJ6bzIOSLDt8f8eLaGOxQ= +cloud.google.com/go/servicedirectory v1.11.0/go.mod h1:Xv0YVH8s4pVOwfM/1eMTl0XJ6bzIOSLDt8f8eLaGOxQ= +cloud.google.com/go/servicemanagement v1.4.0/go.mod h1:d8t8MDbezI7Z2R1O/wu8oTggo3BI2GKYbdG4y/SJTco= +cloud.google.com/go/servicemanagement v1.5.0/go.mod h1:XGaCRe57kfqu4+lRxaFEAuqmjzF0r+gWHjWqKqBvKFo= +cloud.google.com/go/servicemanagement v1.6.0/go.mod h1:aWns7EeeCOtGEX4OvZUWCCJONRZeFKiptqKf1D0l/Jc= +cloud.google.com/go/servicemanagement v1.8.0/go.mod h1:MSS2TDlIEQD/fzsSGfCdJItQveu9NXnUniTrq/L8LK4= +cloud.google.com/go/serviceusage v1.3.0/go.mod h1:Hya1cozXM4SeSKTAgGXgj97GlqUvF5JaoXacR1JTP/E= +cloud.google.com/go/serviceusage v1.4.0/go.mod h1:SB4yxXSaYVuUBYUml6qklyONXNLt83U0Rb+CXyhjEeU= +cloud.google.com/go/serviceusage v1.5.0/go.mod h1:w8U1JvqUqwJNPEOTQjrMHkw3IaIFLoLsPLvsE3xueec= +cloud.google.com/go/serviceusage v1.6.0/go.mod h1:R5wwQcbOWsyuOfbP9tGdAnCAc6B9DRwPG1xtWMDeuPA= +cloud.google.com/go/shell v1.3.0/go.mod h1:VZ9HmRjZBsjLGXusm7K5Q5lzzByZmJHf1d0IWHEN5X4= +cloud.google.com/go/shell v1.4.0/go.mod h1:HDxPzZf3GkDdhExzD/gs8Grqk+dmYcEjGShZgYa9URw= +cloud.google.com/go/shell v1.6.0/go.mod h1:oHO8QACS90luWgxP3N9iZVuEiSF84zNyLytb+qE2f9A= +cloud.google.com/go/shell v1.7.1/go.mod h1:u1RaM+huXFaTojTbW4g9P5emOrrmLE69KrxqQahKn4g= +cloud.google.com/go/spanner v1.41.0/go.mod h1:MLYDBJR/dY4Wt7ZaMIQ7rXOTLjYrmxLE/5ve9vFfWos= +cloud.google.com/go/spanner v1.44.0/go.mod h1:G8XIgYdOK+Fbcpbs7p2fiprDw4CaZX63whnSMLVBxjk= +cloud.google.com/go/spanner v1.45.0/go.mod h1:FIws5LowYz8YAE1J8fOS7DJup8ff7xJeetWEo5REA2M= +cloud.google.com/go/spanner v1.47.0/go.mod h1:IXsJwVW2j4UKs0eYDqodab6HgGuA1bViSqW4uH9lfUI= +cloud.google.com/go/spanner v1.49.0/go.mod h1:eGj9mQGK8+hkgSVbHNQ06pQ4oS+cyc4tXXd6Dif1KoM= +cloud.google.com/go/spanner v1.50.0/go.mod h1:eGj9mQGK8+hkgSVbHNQ06pQ4oS+cyc4tXXd6Dif1KoM= +cloud.google.com/go/speech v1.6.0/go.mod h1:79tcr4FHCimOp56lwC01xnt/WPJZc4v3gzyT7FoBkCM= +cloud.google.com/go/speech v1.7.0/go.mod h1:KptqL+BAQIhMsj1kOP2la5DSEEerPDuOP/2mmkhHhZQ= +cloud.google.com/go/speech v1.8.0/go.mod h1:9bYIl1/tjsAnMgKGHKmBZzXKEkGgtU+MpdDPTE9f7y0= +cloud.google.com/go/speech v1.9.0/go.mod h1:xQ0jTcmnRFFM2RfX/U+rk6FQNUF6DQlydUSyoooSpco= +cloud.google.com/go/speech v1.14.1/go.mod h1:gEosVRPJ9waG7zqqnsHpYTOoAS4KouMRLDFMekpJ0J0= +cloud.google.com/go/speech v1.15.0/go.mod h1:y6oH7GhqCaZANH7+Oe0BhgIogsNInLlz542tg3VqeYI= +cloud.google.com/go/speech v1.17.1/go.mod h1:8rVNzU43tQvxDaGvqOhpDqgkJTFowBpDvCJ14kGlJYo= +cloud.google.com/go/speech v1.19.0/go.mod h1:8rVNzU43tQvxDaGvqOhpDqgkJTFowBpDvCJ14kGlJYo= cloud.google.com/go/storage v1.0.0/go.mod h1:IhtSnM/ZTZV8YYJWCY8RULGVqBDmpoyjwiyrjsg+URw= cloud.google.com/go/storage v1.5.0/go.mod h1:tpKbwo567HUNpVclU5sGELwQWBDZ8gh0ZeosJ0Rtdos= cloud.google.com/go/storage v1.6.0/go.mod h1:N7U0C8pVQ/+NIKOBQyamJIeKQKkZ+mxpohlUTyfDhBk= cloud.google.com/go/storage v1.8.0/go.mod h1:Wv1Oy7z6Yz3DshWRJFhqM/UCfaWIRTdp0RXyy7KQOVs= cloud.google.com/go/storage v1.10.0/go.mod h1:FLPqc6j+Ki4BU591ie1oL6qBQGu2Bl/tZ9ullr3+Kg0= +cloud.google.com/go/storage v1.14.0/go.mod h1:GrKmX003DSIwi9o29oFT7YDnHYwZoctc3fOKtUw0Xmo= +cloud.google.com/go/storage v1.22.1/go.mod h1:S8N1cAStu7BOeFfE8KAQzmyyLkK8p/vmRq6kuBTW58Y= +cloud.google.com/go/storage v1.23.0/go.mod h1:vOEEDNFnciUMhBeT6hsJIn3ieU5cFRmzeLgDvXzfIXc= +cloud.google.com/go/storage v1.27.0/go.mod h1:x9DOL8TK/ygDUMieqwfhdpQryTeEkhGKMi80i/iqR2s= +cloud.google.com/go/storage v1.28.1/go.mod h1:Qnisd4CqDdo6BGs2AD5LLnEsmSQ80wQ5ogcBBKhU86Y= +cloud.google.com/go/storage v1.29.0/go.mod h1:4puEjyTKnku6gfKoTfNOU/W+a9JyuVNxjpS5GBrB8h4= +cloud.google.com/go/storage v1.30.1/go.mod h1:NfxhC0UJE1aXSx7CIIbCf7y9HKT7BiccwkR7+P7gN8E= cloud.google.com/go/storage v1.38.0 h1:Az68ZRGlnNTpIBbLjSMIV2BDcwwXYlRlQzis0llkpJg= cloud.google.com/go/storage v1.38.0/go.mod h1:tlUADB0mAb9BgYls9lq+8MGkfzOXuLrnHXlpHmvFJoY= +cloud.google.com/go/storagetransfer v1.5.0/go.mod h1:dxNzUopWy7RQevYFHewchb29POFv3/AaBgnhqzqiK0w= +cloud.google.com/go/storagetransfer v1.6.0/go.mod h1:y77xm4CQV/ZhFZH75PLEXY0ROiS7Gh6pSKrM8dJyg6I= +cloud.google.com/go/storagetransfer v1.7.0/go.mod h1:8Giuj1QNb1kfLAiWM1bN6dHzfdlDAVC9rv9abHot2W4= +cloud.google.com/go/storagetransfer v1.8.0/go.mod h1:JpegsHHU1eXg7lMHkvf+KE5XDJ7EQu0GwNJbbVGanEw= +cloud.google.com/go/storagetransfer v1.10.0/go.mod h1:DM4sTlSmGiNczmV6iZyceIh2dbs+7z2Ayg6YAiQlYfA= +cloud.google.com/go/talent v1.1.0/go.mod h1:Vl4pt9jiHKvOgF9KoZo6Kob9oV4lwd/ZD5Cto54zDRw= +cloud.google.com/go/talent v1.2.0/go.mod h1:MoNF9bhFQbiJ6eFD3uSsg0uBALw4n4gaCaEjBw9zo8g= +cloud.google.com/go/talent v1.3.0/go.mod h1:CmcxwJ/PKfRgd1pBjQgU6W3YBwiewmUzQYH5HHmSCmM= +cloud.google.com/go/talent v1.4.0/go.mod h1:ezFtAgVuRf8jRsvyE6EwmbTK5LKciD4KVnHuDEFmOOA= +cloud.google.com/go/talent v1.5.0/go.mod h1:G+ODMj9bsasAEJkQSzO2uHQWXHHXUomArjWQQYkqK6c= +cloud.google.com/go/talent v1.6.2/go.mod h1:CbGvmKCG61mkdjcqTcLOkb2ZN1SrQI8MDyma2l7VD24= +cloud.google.com/go/texttospeech v1.4.0/go.mod h1:FX8HQHA6sEpJ7rCMSfXuzBcysDAuWusNNNvN9FELDd8= +cloud.google.com/go/texttospeech v1.5.0/go.mod h1:oKPLhR4n4ZdQqWKURdwxMy0uiTS1xU161C8W57Wkea4= +cloud.google.com/go/texttospeech v1.6.0/go.mod h1:YmwmFT8pj1aBblQOI3TfKmwibnsfvhIBzPXcW4EBovc= +cloud.google.com/go/texttospeech v1.7.1/go.mod h1:m7QfG5IXxeneGqTapXNxv2ItxP/FS0hCZBwXYqucgSk= +cloud.google.com/go/tpu v1.3.0/go.mod h1:aJIManG0o20tfDQlRIej44FcwGGl/cD0oiRyMKG19IQ= +cloud.google.com/go/tpu v1.4.0/go.mod h1:mjZaX8p0VBgllCzF6wcU2ovUXN9TONFLd7iz227X2Xg= +cloud.google.com/go/tpu v1.5.0/go.mod h1:8zVo1rYDFuW2l4yZVY0R0fb/v44xLh3llq7RuV61fPM= +cloud.google.com/go/tpu v1.6.1/go.mod h1:sOdcHVIgDEEOKuqUoi6Fq53MKHJAtOwtz0GuKsWSH3E= +cloud.google.com/go/trace v1.3.0/go.mod h1:FFUE83d9Ca57C+K8rDl/Ih8LwOzWIV1krKgxg6N0G28= +cloud.google.com/go/trace v1.4.0/go.mod h1:UG0v8UBqzusp+z63o7FK74SdFE+AXpCLdFb1rshXG+Y= +cloud.google.com/go/trace v1.8.0/go.mod h1:zH7vcsbAhklH8hWFig58HvxcxyQbaIqMarMg9hn5ECA= +cloud.google.com/go/trace v1.9.0/go.mod h1:lOQqpE5IaWY0Ixg7/r2SjixMuc6lfTFeO4QGM4dQWOk= +cloud.google.com/go/trace v1.10.1/go.mod h1:gbtL94KE5AJLH3y+WVpfWILmqgc6dXcqgNXdOPAQTYk= +cloud.google.com/go/translate v1.3.0/go.mod h1:gzMUwRjvOqj5i69y/LYLd8RrNQk+hOmIXTi9+nb3Djs= +cloud.google.com/go/translate v1.4.0/go.mod h1:06Dn/ppvLD6WvA5Rhdp029IX2Mi3Mn7fpMRLPvXT5Wg= +cloud.google.com/go/translate v1.5.0/go.mod h1:29YDSYveqqpA1CQFD7NQuP49xymq17RXNaUDdc0mNu0= +cloud.google.com/go/translate v1.6.0/go.mod h1:lMGRudH1pu7I3n3PETiOB2507gf3HnfLV8qlkHZEyos= +cloud.google.com/go/translate v1.7.0/go.mod h1:lMGRudH1pu7I3n3PETiOB2507gf3HnfLV8qlkHZEyos= +cloud.google.com/go/translate v1.8.1/go.mod h1:d1ZH5aaOA0CNhWeXeC8ujd4tdCFw8XoNWRljklu5RHs= +cloud.google.com/go/translate v1.8.2/go.mod h1:d1ZH5aaOA0CNhWeXeC8ujd4tdCFw8XoNWRljklu5RHs= +cloud.google.com/go/translate v1.9.0/go.mod h1:d1ZH5aaOA0CNhWeXeC8ujd4tdCFw8XoNWRljklu5RHs= +cloud.google.com/go/video v1.8.0/go.mod h1:sTzKFc0bUSByE8Yoh8X0mn8bMymItVGPfTuUBUyRgxk= +cloud.google.com/go/video v1.9.0/go.mod h1:0RhNKFRF5v92f8dQt0yhaHrEuH95m068JYOvLZYnJSw= +cloud.google.com/go/video v1.12.0/go.mod h1:MLQew95eTuaNDEGriQdcYn0dTwf9oWiA4uYebxM5kdg= +cloud.google.com/go/video v1.13.0/go.mod h1:ulzkYlYgCp15N2AokzKjy7MQ9ejuynOJdf1tR5lGthk= +cloud.google.com/go/video v1.14.0/go.mod h1:SkgaXwT+lIIAKqWAJfktHT/RbgjSuY6DobxEp0C5yTQ= +cloud.google.com/go/video v1.15.0/go.mod h1:SkgaXwT+lIIAKqWAJfktHT/RbgjSuY6DobxEp0C5yTQ= +cloud.google.com/go/video v1.17.1/go.mod h1:9qmqPqw/Ib2tLqaeHgtakU+l5TcJxCJbhFXM7UJjVzU= +cloud.google.com/go/video v1.19.0/go.mod h1:9qmqPqw/Ib2tLqaeHgtakU+l5TcJxCJbhFXM7UJjVzU= +cloud.google.com/go/video v1.20.0/go.mod h1:U3G3FTnsvAGqglq9LxgqzOiBc/Nt8zis8S+850N2DUM= +cloud.google.com/go/videointelligence v1.6.0/go.mod h1:w0DIDlVRKtwPCn/C4iwZIJdvC69yInhW0cfi+p546uU= +cloud.google.com/go/videointelligence v1.7.0/go.mod h1:k8pI/1wAhjznARtVT9U1llUaFNPh7muw8QyOUpavru4= +cloud.google.com/go/videointelligence v1.8.0/go.mod h1:dIcCn4gVDdS7yte/w+koiXn5dWVplOZkE+xwG9FgK+M= +cloud.google.com/go/videointelligence v1.9.0/go.mod h1:29lVRMPDYHikk3v8EdPSaL8Ku+eMzDljjuvRs105XoU= +cloud.google.com/go/videointelligence v1.10.0/go.mod h1:LHZngX1liVtUhZvi2uNS0VQuOzNi2TkY1OakiuoUOjU= +cloud.google.com/go/videointelligence v1.11.1/go.mod h1:76xn/8InyQHarjTWsBR058SmlPCwQjgcvoW0aZykOvo= +cloud.google.com/go/vision v1.2.0/go.mod h1:SmNwgObm5DpFBme2xpyOyasvBc1aPdjvMk2bBk0tKD0= +cloud.google.com/go/vision/v2 v2.2.0/go.mod h1:uCdV4PpN1S0jyCyq8sIM42v2Y6zOLkZs+4R9LrGYwFo= +cloud.google.com/go/vision/v2 v2.3.0/go.mod h1:UO61abBx9QRMFkNBbf1D8B1LXdS2cGiiCRx0vSpZoUo= +cloud.google.com/go/vision/v2 v2.4.0/go.mod h1:VtI579ll9RpVTrdKdkMzckdnwMyX2JILb+MhPqRbPsY= +cloud.google.com/go/vision/v2 v2.5.0/go.mod h1:MmaezXOOE+IWa+cS7OhRRLK2cNv1ZL98zhqFFZaaH2E= +cloud.google.com/go/vision/v2 v2.6.0/go.mod h1:158Hes0MvOS9Z/bDMSFpjwsUrZ5fPrdwuyyvKSGAGMY= +cloud.google.com/go/vision/v2 v2.7.0/go.mod h1:H89VysHy21avemp6xcf9b9JvZHVehWbET0uT/bcuY/0= +cloud.google.com/go/vision/v2 v2.7.2/go.mod h1:jKa8oSYBWhYiXarHPvP4USxYANYUEdEsQrloLjrSwJU= +cloud.google.com/go/vmmigration v1.2.0/go.mod h1:IRf0o7myyWFSmVR1ItrBSFLFD/rJkfDCUTO4vLlJvsE= +cloud.google.com/go/vmmigration v1.3.0/go.mod h1:oGJ6ZgGPQOFdjHuocGcLqX4lc98YQ7Ygq8YQwHh9A7g= +cloud.google.com/go/vmmigration v1.5.0/go.mod h1:E4YQ8q7/4W9gobHjQg4JJSgXXSgY21nA5r8swQV+Xxc= +cloud.google.com/go/vmmigration v1.6.0/go.mod h1:bopQ/g4z+8qXzichC7GW1w2MjbErL54rk3/C843CjfY= +cloud.google.com/go/vmmigration v1.7.1/go.mod h1:WD+5z7a/IpZ5bKK//YmT9E047AD+rjycCAvyMxGJbro= +cloud.google.com/go/vmwareengine v0.1.0/go.mod h1:RsdNEf/8UDvKllXhMz5J40XxDrNJNN4sagiox+OI208= +cloud.google.com/go/vmwareengine v0.2.2/go.mod h1:sKdctNJxb3KLZkE/6Oui94iw/xs9PRNC2wnNLXsHvH8= +cloud.google.com/go/vmwareengine v0.3.0/go.mod h1:wvoyMvNWdIzxMYSpH/R7y2h5h3WFkx6d+1TIsP39WGY= +cloud.google.com/go/vmwareengine v0.4.1/go.mod h1:Px64x+BvjPZwWuc4HdmVhoygcXqEkGHXoa7uyfTgSI0= +cloud.google.com/go/vmwareengine v1.0.0/go.mod h1:Px64x+BvjPZwWuc4HdmVhoygcXqEkGHXoa7uyfTgSI0= +cloud.google.com/go/vpcaccess v1.4.0/go.mod h1:aQHVbTWDYUR1EbTApSVvMq1EnT57ppDmQzZ3imqIk4w= +cloud.google.com/go/vpcaccess v1.5.0/go.mod h1:drmg4HLk9NkZpGfCmZ3Tz0Bwnm2+DKqViEpeEpOq0m8= +cloud.google.com/go/vpcaccess v1.6.0/go.mod h1:wX2ILaNhe7TlVa4vC5xce1bCnqE3AeH27RV31lnmZes= +cloud.google.com/go/vpcaccess v1.7.1/go.mod h1:FogoD46/ZU+JUBX9D606X21EnxiszYi2tArQwLY4SXs= +cloud.google.com/go/webrisk v1.4.0/go.mod h1:Hn8X6Zr+ziE2aNd8SliSDWpEnSS1u4R9+xXZmFiHmGE= +cloud.google.com/go/webrisk v1.5.0/go.mod h1:iPG6fr52Tv7sGk0H6qUFzmL3HHZev1htXuWDEEsqMTg= +cloud.google.com/go/webrisk v1.6.0/go.mod h1:65sW9V9rOosnc9ZY7A7jsy1zoHS5W9IAXv6dGqhMQMc= +cloud.google.com/go/webrisk v1.7.0/go.mod h1:mVMHgEYH0r337nmt1JyLthzMr6YxwN1aAIEc2fTcq7A= +cloud.google.com/go/webrisk v1.8.0/go.mod h1:oJPDuamzHXgUc+b8SiHRcVInZQuybnvEW72PqTc7sSg= +cloud.google.com/go/webrisk v1.9.1/go.mod h1:4GCmXKcOa2BZcZPn6DCEvE7HypmEJcJkr4mtM+sqYPc= +cloud.google.com/go/websecurityscanner v1.3.0/go.mod h1:uImdKm2wyeXQevQJXeh8Uun/Ym1VqworNDlBXQevGMo= +cloud.google.com/go/websecurityscanner v1.4.0/go.mod h1:ebit/Fp0a+FWu5j4JOmJEV8S8CzdTkAS77oDsiSqYWQ= +cloud.google.com/go/websecurityscanner v1.5.0/go.mod h1:Y6xdCPy81yi0SQnDY1xdNTNpfY1oAgXUlcfN3B3eSng= +cloud.google.com/go/websecurityscanner v1.6.1/go.mod h1:Njgaw3rttgRHXzwCB8kgCYqv5/rGpFCsBOvPbYgszpg= +cloud.google.com/go/workflows v1.6.0/go.mod h1:6t9F5h/unJz41YqfBmqSASJSXccBLtD1Vwf+KmJENM0= +cloud.google.com/go/workflows v1.7.0/go.mod h1:JhSrZuVZWuiDfKEFxU0/F1PQjmpnpcoISEXH2bcHC3M= +cloud.google.com/go/workflows v1.8.0/go.mod h1:ysGhmEajwZxGn1OhGOGKsTXc5PyxOc0vfKf5Af+to4M= +cloud.google.com/go/workflows v1.9.0/go.mod h1:ZGkj1aFIOd9c8Gerkjjq7OW7I5+l6cSvT3ujaO/WwSA= +cloud.google.com/go/workflows v1.10.0/go.mod h1:fZ8LmRmZQWacon9UCX1r/g/DfAXx5VcPALq2CxzdePw= +cloud.google.com/go/workflows v1.11.1/go.mod h1:Z+t10G1wF7h8LgdY/EmRcQY8ptBD/nvofaL6FqlET6g= +cloud.google.com/go/workflows v1.12.0/go.mod h1:PYhSk2b6DhZ508tj8HXKaBh+OFe+xdl0dHF/tJdzPQM= dmitri.shuralyov.com/gpu/mtl v0.0.0-20190408044501-666a987793e9/go.mod h1:H6x//7gZCb22OMCxBHrMx7a5I7Hp++hsVxbQ4BYO7hU= filippo.io/edwards25519 v1.1.0 h1:FNf4tywRC1HmFuKW5xopWpigGjJKiJSV0Cqo0cJWDaA= filippo.io/edwards25519 v1.1.0/go.mod h1:BxyFTGdWcka3PhytdK4V28tE5sGfRvvvRV7EaN4VDT4= @@ -80,6 +856,7 @@ github.com/Masterminds/semver/v3 v3.3.0 h1:B8LGeaivUe71a5qox1ICM/JLl0NqZSW5CHyL+ github.com/Masterminds/semver/v3 v3.3.0/go.mod h1:4V+yj/TJE1HU9XfppCwVMZq3I84lprf4nC11bSS5beM= github.com/Masterminds/sprig/v3 v3.2.2 h1:17jRggJu518dr3QaafizSXOjKYp94wKfABxUmyxvxX8= github.com/Masterminds/sprig/v3 v3.2.2/go.mod h1:UoaO7Yp8KlPnJIYWTFkMaqPUYKTfGFPhxNuwnnxkKlk= +github.com/OneOfOne/xxhash v1.2.2/go.mod h1:HSdplMjZKSmBqAxg5vPj2TmRDmfkzw+cTzAElWljhcU= github.com/VividCortex/ewma v1.1.1/go.mod h1:2Tkkvm3sRDVXaiyucHiACn4cqf7DpdyLvmxzcbUokwA= github.com/VividCortex/ewma v1.2.0 h1:f58SaIzcDXrSy3kWaHNvuJgJ3Nmz59Zji6XoJR/q1ow= github.com/VividCortex/ewma v1.2.0/go.mod h1:nz4BbCtbLyFDeC9SUHbtcT5644juEuWfUAUnGx7j5l4= @@ -103,6 +880,9 @@ github.com/andybalholm/brotli v1.0.4/go.mod h1:fO7iG3H7G2nSZ7m0zPUDn85XEX2GTukHG github.com/andybalholm/brotli v1.0.5 h1:8uQZIdzKmjc/iuPu7O2ioW48L81FgatrcpfFmiq/cCs= github.com/andybalholm/brotli v1.0.5/go.mod h1:fO7iG3H7G2nSZ7m0zPUDn85XEX2GTukHGRSepvi9Eig= github.com/antihax/optional v1.0.0/go.mod h1:uupD/76wgC+ih3iEmQUL+0Ugr19nfwCT1kdvxnR2qWY= +github.com/apache/arrow/go/v10 v10.0.1/go.mod h1:YvhnlEePVnBS4+0z3fhPfUy7W1Ikj0Ih0vcRo/gZ1M0= +github.com/apache/arrow/go/v11 v11.0.0/go.mod h1:Eg5OsL5H+e299f7u5ssuXsuHQVEGC4xei5aX110hRiI= +github.com/apache/arrow/go/v12 v12.0.0/go.mod h1:d+tV/eHZZ7Dz7RPrFKtPK02tpr+c9/PEd/zm8mDS9Vg= github.com/apache/arrow/go/v12 v12.0.1 h1:JsR2+hzYYjgSUkBSaahpqCetqZMr76djX80fF/DiJbg= github.com/apache/arrow/go/v12 v12.0.1/go.mod h1:weuTY7JvTG/HDPtMQxEUp7pU73vkLWMLpY67QwZ/WWw= github.com/apache/skywalking-eyes v0.4.0 h1:O13kdRU6FCEZevfD01mdhTgCZLLfPZIQ0GXZrLl7FpQ= @@ -147,8 +927,12 @@ github.com/carlmjohnson/flagext v0.21.0/go.mod h1:Eenv0epIUAr4NuedNmkzI8WmBmjIxZ github.com/cenkalti/backoff/v4 v4.2.1 h1:y4OZtCnogmCPw98Zjyt5a6+QwPLGkiQsYW5oUqylYbM= github.com/cenkalti/backoff/v4 v4.2.1/go.mod h1:Y3VNntkOUPxTVeUxJ/G5vcM//AlwfmyYozVcomhLiZE= github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= +github.com/census-instrumentation/opencensus-proto v0.3.0/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= +github.com/census-instrumentation/opencensus-proto v0.4.1/go.mod h1:4T9NM4+4Vw91VeyqjLS6ao50K5bOcLKN6Q42XnYaRYw= +github.com/cespare/xxhash v1.1.0/go.mod h1:XrSqR1VqqWfGrhpAt58auRo0WTKS1nRRg3ghfAqPWnc= github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= github.com/cespare/xxhash/v2 v2.1.2/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= +github.com/cespare/xxhash/v2 v2.2.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= github.com/cespare/xxhash/v2 v2.3.0 h1:UL815xU9SqsFlibzuggzjXhog7bL6oX9BbNZnL2UFvs= github.com/cespare/xxhash/v2 v2.3.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= github.com/charithe/durationcheck v0.0.10 h1:wgw73BiocdBDQPik+zcEoBG/ob8uyBHf2iyoHGPf5w4= @@ -168,11 +952,20 @@ github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDk github.com/cloudfoundry/gosigar v1.3.6 h1:gIc08FbB3QPb+nAQhINIK/qhf5REKkY0FTGgRGXkcVc= github.com/cloudfoundry/gosigar v1.3.6/go.mod h1:lNWstu5g5gw59O09Y+wsMNFzBSnU8a0u+Sfx4dq360E= github.com/cncf/udpa/go v0.0.0-20191209042840-269d4d468f6f/go.mod h1:M8M6+tZqaGXZJjfX53e64911xZQV5JYwmTeXPW+k8Sc= +github.com/cncf/udpa/go v0.0.0-20200629203442-efcf912fb354/go.mod h1:WmhPx2Nbnhtbo57+VJT5O0JRkEi1Wbu0z5j0R8u5Hbk= github.com/cncf/udpa/go v0.0.0-20201120205902-5459f2c99403/go.mod h1:WmhPx2Nbnhtbo57+VJT5O0JRkEi1Wbu0z5j0R8u5Hbk= github.com/cncf/udpa/go v0.0.0-20210930031921-04548b0d99d4/go.mod h1:6pvJx4me5XPnfI9Z40ddWsdw2W/uZgQLFXToKeRcDiI= +github.com/cncf/udpa/go v0.0.0-20220112060539-c52dc94e7fbe/go.mod h1:6pvJx4me5XPnfI9Z40ddWsdw2W/uZgQLFXToKeRcDiI= +github.com/cncf/xds/go v0.0.0-20210312221358-fbca930ec8ed/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs= +github.com/cncf/xds/go v0.0.0-20210805033703-aa0b78936158/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs= github.com/cncf/xds/go v0.0.0-20210922020428-25de7278fc84/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs= github.com/cncf/xds/go v0.0.0-20211001041855-01bcc9b48dfe/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs= github.com/cncf/xds/go v0.0.0-20211011173535-cb28da3451f1/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs= +github.com/cncf/xds/go v0.0.0-20220314180256-7f1daf1720fc/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs= +github.com/cncf/xds/go v0.0.0-20230105202645-06c439db220b/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs= +github.com/cncf/xds/go v0.0.0-20230310173818-32f1caf87195/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs= +github.com/cncf/xds/go v0.0.0-20230428030218-4003588d1b74/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs= +github.com/cncf/xds/go v0.0.0-20230607035331-e9ce68804cb4/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs= github.com/cockroachdb/datadriven v1.0.3-0.20230413201302-be42291fc80f h1:otljaYPt5hWxV3MUfO5dFPFiOXg9CyG5/kCfayTqsJ4= github.com/cockroachdb/datadriven v1.0.3-0.20230413201302-be42291fc80f/go.mod h1:a9RdTaap04u637JoCzcUoIcDmvwSUtcUFtT/C3kJlTU= github.com/cockroachdb/errors v1.11.1 h1:xSEW75zKaKCWzR3OfxXUxgrk/NtT4G1MiOv5lWZazG8= @@ -238,9 +1031,23 @@ github.com/emirpasic/gods v1.18.1/go.mod h1:8tpGGwCnJ5H4r6BWwaV6OrWmMoPhUl5jm/FM github.com/envoyproxy/go-control-plane v0.9.0/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= github.com/envoyproxy/go-control-plane v0.9.4/go.mod h1:6rpuAdCZL397s3pYoYcLgu1mIlRU8Am5FuJP05cCM98= +github.com/envoyproxy/go-control-plane v0.9.7/go.mod h1:cwu0lG7PUMfa9snN8LXBig5ynNVH9qI8YYLbd1fK2po= github.com/envoyproxy/go-control-plane v0.9.9-0.20201210154907-fd9021fe5dad/go.mod h1:cXg6YxExXjJnVBQHBLXeUAgxn2UodCpnH306RInaBQk= +github.com/envoyproxy/go-control-plane v0.9.9-0.20210217033140-668b12f5399d/go.mod h1:cXg6YxExXjJnVBQHBLXeUAgxn2UodCpnH306RInaBQk= +github.com/envoyproxy/go-control-plane v0.9.9-0.20210512163311-63b5d3c536b0/go.mod h1:hliV/p42l8fGbc6Y9bQ70uLwIvmJyVE5k4iMKlh8wCQ= +github.com/envoyproxy/go-control-plane v0.9.10-0.20210907150352-cf90f659a021/go.mod h1:AFq3mo9L8Lqqiid3OhADV3RfLJnjiw63cSpi+fDTRC0= github.com/envoyproxy/go-control-plane v0.10.2-0.20220325020618-49ff273808a1/go.mod h1:KJwIaB5Mv44NWtYuAOFCVOjcI94vtpEz2JU/D2v6IjE= +github.com/envoyproxy/go-control-plane v0.10.3/go.mod h1:fJJn/j26vwOu972OllsvAgJJM//w9BV6Fxbg2LuVd34= +github.com/envoyproxy/go-control-plane v0.11.0/go.mod h1:VnHyVMpzcLvCFt9yUz1UnCwHLhwx1WguiVDV7pTG/tI= +github.com/envoyproxy/go-control-plane v0.11.1-0.20230524094728-9239064ad72f/go.mod h1:sfYdkwUW4BA3PbKjySwjJy+O4Pu0h62rlqCMHNk+K+Q= +github.com/envoyproxy/go-control-plane v0.11.1/go.mod h1:uhMcXKCQMEJHiAb0w+YGefQLaTEw+YhGluxZkrTmD0g= github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= +github.com/envoyproxy/protoc-gen-validate v0.6.7/go.mod h1:dyJXwwfPK2VSqiB9Klm1J6romD608Ba7Hij42vrOBCo= +github.com/envoyproxy/protoc-gen-validate v0.9.1/go.mod h1:OKNgG7TCp5pF4d6XftA0++PMirau2/yoOwVac3AbF2w= +github.com/envoyproxy/protoc-gen-validate v0.10.0/go.mod h1:DRjgyB0I43LtJapqN6NiRwroiAU2PaFuvk/vjgh61ss= +github.com/envoyproxy/protoc-gen-validate v0.10.1/go.mod h1:DRjgyB0I43LtJapqN6NiRwroiAU2PaFuvk/vjgh61ss= +github.com/envoyproxy/protoc-gen-validate v1.0.1/go.mod h1:0vj8bNkYbSTNS2PIyH87KZaeN4x9zpL9Qt8fQC7d+vs= +github.com/envoyproxy/protoc-gen-validate v1.0.2/go.mod h1:GpiZQP3dDbg4JouG/NNS7QWXpgx6x8QiMKdmN72jogE= github.com/fatih/color v1.10.0/go.mod h1:ELkj/draVOlAH/xkhN6mQ50Qd0MPOk5AAr3maGEBuJM= github.com/fatih/color v1.18.0 h1:S8gINlzdQ840/4pfAwic/ZE0djQEH3wM94VfqLTZcOM= github.com/fatih/color v1.18.0/go.mod h1:4FelSpRwEGDpQ12mAdzqdOukCy4u8WUtOY6lkT/6HfU= @@ -309,7 +1116,6 @@ github.com/goccy/go-json v0.10.2/go.mod h1:6MelG93GURQebXPDq3khkgXZkazVtN9CRI+MG github.com/goccy/go-reflect v1.2.0 h1:O0T8rZCuNmGXewnATuKYnkL0xm6o8UNOJZd/gOkb9ms= github.com/goccy/go-reflect v1.2.0/go.mod h1:n0oYZn8VcV2CkWTxi8B9QjkCoq6GTtCEdfmR66YhFtE= github.com/godbus/dbus/v5 v5.0.4/go.mod h1:xhWf0FNVPg57R7Z0UbKHbJfkEywrmjJnf7w5xrFpKfA= -github.com/gogo/protobuf v0.0.0-20180717141946-636bf0302bc9/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= github.com/gogo/protobuf v1.2.1/go.mod h1:hp+jE20tsWTFYpLwKvXlhS1hjn+gTNwPg2I6zVXpSg4= github.com/gogo/protobuf v1.3.2 h1:Ov1cvc58UF3b5XjBnZv7+opcTcQFZebYjWzi34vdm4Q= github.com/gogo/protobuf v1.3.2/go.mod h1:P1XiOD3dCwIKUDQYPy72D8LYyHL2YPYrpS2s69NZV8Q= @@ -320,6 +1126,9 @@ github.com/golang-jwt/jwt/v5 v5.2.1 h1:OuVbFODueb089Lh128TAcimifWaLhJwVflnrgM17w github.com/golang-jwt/jwt/v5 v5.2.1/go.mod h1:pqrtFR0X4osieyHYxtmOUWsAWrfe1Q5UVIyoH402zdk= github.com/golang/freetype v0.0.0-20170609003504-e2365dfdc4a0/go.mod h1:E/TSTwGwJL78qG/PmXZO1EjYhfJinVAhrmmHX6Z8B9k= github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= +github.com/golang/glog v1.0.0/go.mod h1:EWib/APOK0SL3dFbYqvxE3UYd8E6s1ouQ7iEp/0LWV4= +github.com/golang/glog v1.1.0/go.mod h1:pfYeQZ3JWZoXTV5sFc986z3HTpwQs9At6P4ImfuP3NQ= +github.com/golang/glog v1.1.2/go.mod h1:zR+okUeTbrL6EL3xHUDxZuEtGv04p5shwip1+mL/rLQ= github.com/golang/glog v1.2.0 h1:uCdmnmatrKCgMBlM4rMuJZWOkPDqdbZPnrMXDY4gI68= github.com/golang/glog v1.2.0/go.mod h1:6AhwSGph0fcJtXVM/PEHPqZlFeoLxhs7/t5UDAwmO+w= github.com/golang/groupcache v0.0.0-20190702054246-869f871628b6/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= @@ -335,7 +1144,7 @@ github.com/golang/mock v1.4.1/go.mod h1:UOMv5ysSaYNkG+OFQykRIcU/QvvxJf3p21QfJ2Bt github.com/golang/mock v1.4.3/go.mod h1:UOMv5ysSaYNkG+OFQykRIcU/QvvxJf3p21QfJ2Bt3cw= github.com/golang/mock v1.4.4/go.mod h1:l3mdAwkq5BuhzHwde/uurv3sEJeZMXNpwsxVWU71h+4= github.com/golang/mock v1.5.0/go.mod h1:CWnOUgYIOo4TcNZ0wHX3YZCqsaM1I1Jvs6v3mP3KVu8= -github.com/golang/protobuf v0.0.0-20180814211427-aa810b61a9c7/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= +github.com/golang/mock v1.6.0/go.mod h1:p6yTPP+5HYm5mzsMV8JkE6ZKdX+/wYM6Hr+LicevLPs= github.com/golang/protobuf v1.1.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.3.1/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= @@ -352,12 +1161,14 @@ github.com/golang/protobuf v1.4.1/go.mod h1:U8fpvMrcmy5pZrNK1lt4xCsGvpyWQ/VVv6QD github.com/golang/protobuf v1.4.2/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI= github.com/golang/protobuf v1.4.3/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI= github.com/golang/protobuf v1.5.0/go.mod h1:FsONVRAS9T7sI+LIUmWTfcYkHO4aIWwzhcaSAoJOfIk= +github.com/golang/protobuf v1.5.1/go.mod h1:DopwsBzvsk0Fs44TXzsVbJyPhcCPeIwnvohx4u74HPM= github.com/golang/protobuf v1.5.2/go.mod h1:XVQd3VNwM+JqD3oG2Ue2ip4fOMUkwXdXDdiuN0vRsmY= github.com/golang/protobuf v1.5.3/go.mod h1:XVQd3VNwM+JqD3oG2Ue2ip4fOMUkwXdXDdiuN0vRsmY= github.com/golang/protobuf v1.5.4 h1:i7eJL8qZTpSEXOPTxNKhASYpMn+8e5Q6AdndVa1dWek= github.com/golang/protobuf v1.5.4/go.mod h1:lnTiLA8Wa4RWRcIUkrtSVa5nRhsEGBg48fD6rSs7xps= github.com/golang/snappy v0.0.0-20180518054509-2e65f85255db/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= github.com/golang/snappy v0.0.1/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= +github.com/golang/snappy v0.0.3/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= github.com/golang/snappy v0.0.4 h1:yAGX7huGHXlcLOEtBnF4w7FQwA26wojNCwOYAEhLjQM= github.com/golang/snappy v0.0.4/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= github.com/golangci/gofmt v0.0.0-20240816233607-d8596aa466a9 h1:/1322Qns6BtQxUZDTAT4SdcoxknUki7IAoK4SAXr8ME= @@ -394,6 +1205,7 @@ github.com/google/go-cmp v0.5.9/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeN github.com/google/go-cmp v0.6.0 h1:ofyhxvXcZhMsU5ulbFiLKl/XBFqE1GSq7atu8tAmTRI= github.com/google/go-cmp v0.6.0/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= github.com/google/go-github/v33 v33.0.0/go.mod h1:GMdDnVZY/2TsWgp/lkYnpSAh6TrzhANBBwm6k6TTEXg= +github.com/google/go-pkcs11 v0.2.0/go.mod h1:6eQoGcuNJpa7jnd5pMGdkSaQpNDYvPlXWMcjXXThLlY= github.com/google/go-querystring v1.0.0/go.mod h1:odCYkC5MyYFN7vkCjXpyrEuKhc/BUO6wN/zVPAxq5ck= github.com/google/go-querystring v1.1.0/go.mod h1:Kcdr2DB4koayq7X8pmAG4sNG59So17icRSOU623lUBU= github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= @@ -404,6 +1216,8 @@ github.com/google/licensecheck v0.3.1/go.mod h1:ORkR35t/JjW+emNKtfJDII0zlciG9Jgb github.com/google/martian v2.1.0+incompatible h1:/CP5g8u/VJHijgedC/Legn3BAbAaWPgecwXBIDzw5no= github.com/google/martian v2.1.0+incompatible/go.mod h1:9I4somxYTbIHy5NJKHRl3wXiIaQGbYVAs8BPL6v8lEs= github.com/google/martian/v3 v3.0.0/go.mod h1:y5Zk1BBys9G+gd6Jrk0W3cC1+ELVxBWuIGO+w/tUAp0= +github.com/google/martian/v3 v3.1.0/go.mod h1:y5Zk1BBys9G+gd6Jrk0W3cC1+ELVxBWuIGO+w/tUAp0= +github.com/google/martian/v3 v3.2.1/go.mod h1:oBOf6HBosgwRXnUGWUB05QECsc6uvmMiJ3+6W4l/CUk= github.com/google/martian/v3 v3.3.2 h1:IqNFLAmvJOgVlpdEBiQbDc2EwKW77amAycfTuWKdfvw= github.com/google/martian/v3 v3.3.2/go.mod h1:oBOf6HBosgwRXnUGWUB05QECsc6uvmMiJ3+6W4l/CUk= github.com/google/pprof v0.0.0-20181206194817-3ea8567a2e57/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc= @@ -413,12 +1227,23 @@ github.com/google/pprof v0.0.0-20200212024743-f11f1df84d12/go.mod h1:ZgVRPoUq/hf github.com/google/pprof v0.0.0-20200229191704-1ebb73c60ed3/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= github.com/google/pprof v0.0.0-20200430221834-fc25d7d30c6d/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= github.com/google/pprof v0.0.0-20200708004538-1a94d8640e99/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= +github.com/google/pprof v0.0.0-20201023163331-3e6fc7fc9c4c/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= +github.com/google/pprof v0.0.0-20201203190320-1bf35d6f28c2/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= +github.com/google/pprof v0.0.0-20201218002935-b9804c9f04c2/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= +github.com/google/pprof v0.0.0-20210122040257-d980be63207e/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= +github.com/google/pprof v0.0.0-20210226084205-cbba55b83ad5/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= +github.com/google/pprof v0.0.0-20210601050228-01bbb1931b22/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= +github.com/google/pprof v0.0.0-20210609004039-a478d1d731e9/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= +github.com/google/pprof v0.0.0-20210720184732-4bb14d4b1be1/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= github.com/google/pprof v0.0.0-20211214055906-6f57359322fd/go.mod h1:KgnwoLYCZ8IQu3XUZ8Nc/bM9CCZFOyjUNOSygVozoDg= github.com/google/pprof v0.0.0-20241001023024-f4c0cfd0cf1d h1:Jaz2JzpQaQXyET0AjLBXShrthbpqMkhGiEfkcQAiAUs= github.com/google/pprof v0.0.0-20241001023024-f4c0cfd0cf1d/go.mod h1:vavhavw2zAxS5dIdcRluK6cSGGPlZynqzFM8NdvU144= github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm40UhjYkI= github.com/google/renameio/v2 v2.0.0 h1:UifI23ZTGY8Tt29JbYFiuyIU3eX+RNFtUwefq9qAhxg= github.com/google/renameio/v2 v2.0.0/go.mod h1:BtmJXm5YlszgC+TD4HOEEUFgkJP3nLxehU6hfe7jRt4= +github.com/google/s2a-go v0.1.0/go.mod h1:OJpEgntRZo8ugHpF9hkoLJbS5dSI20XZeXJ9JVywLlM= +github.com/google/s2a-go v0.1.3/go.mod h1:Ej+mSEMGRnqRzjc7VtF+jdBwYG5fuJfiZ8ELkjEwM0A= +github.com/google/s2a-go v0.1.4/go.mod h1:Ej+mSEMGRnqRzjc7VtF+jdBwYG5fuJfiZ8ELkjEwM0A= github.com/google/s2a-go v0.1.7 h1:60BLSyTrOV4/haCDW4zb1guZItoSq8foHCXrAnjBo/o= github.com/google/s2a-go v0.1.7/go.mod h1:50CgR4k1jNlWBu4UfS4AcfhVe1r6pdZPygJ3R8F0Qdw= github.com/google/skylark v0.0.0-20181101142754-a5f7082aabed h1:rZdD1GeRTHD1aG+VIvhQEYXurx6Wfg4QIT5YVl2tSC8= @@ -426,14 +1251,37 @@ github.com/google/skylark v0.0.0-20181101142754-a5f7082aabed/go.mod h1:CKSX6SxHW github.com/google/uuid v1.1.1/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/google/uuid v1.1.2/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/google/uuid v1.3.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= +github.com/google/uuid v1.3.1/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/google/uuid v1.6.0 h1:NIvaJDMOsjHA8n1jAhLSgzrAzy1Hgr+hNrb57e+94F0= github.com/google/uuid v1.6.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= +github.com/googleapis/enterprise-certificate-proxy v0.0.0-20220520183353-fd19c99a87aa/go.mod h1:17drOmN3MwGY7t0e+Ei9b45FFGA3fBs3x36SsCg1hq8= +github.com/googleapis/enterprise-certificate-proxy v0.1.0/go.mod h1:17drOmN3MwGY7t0e+Ei9b45FFGA3fBs3x36SsCg1hq8= +github.com/googleapis/enterprise-certificate-proxy v0.2.0/go.mod h1:8C0jb7/mgJe/9KK8Lm7X9ctZC2t60YyIpYEI16jx0Qg= +github.com/googleapis/enterprise-certificate-proxy v0.2.1/go.mod h1:AwSRAtLfXpU5Nm3pW+v7rGDHp09LsPtGY9MduiEsR9k= +github.com/googleapis/enterprise-certificate-proxy v0.2.3/go.mod h1:AwSRAtLfXpU5Nm3pW+v7rGDHp09LsPtGY9MduiEsR9k= +github.com/googleapis/enterprise-certificate-proxy v0.2.4/go.mod h1:AwSRAtLfXpU5Nm3pW+v7rGDHp09LsPtGY9MduiEsR9k= +github.com/googleapis/enterprise-certificate-proxy v0.2.5/go.mod h1:RxW0N9901Cko1VOCW3SXCpWP+mlIEkk2tP7jnHy9a3w= github.com/googleapis/enterprise-certificate-proxy v0.3.2 h1:Vie5ybvEvT75RniqhfFxPRy3Bf7vr3h0cechB90XaQs= github.com/googleapis/enterprise-certificate-proxy v0.3.2/go.mod h1:VLSiSSBs/ksPL8kq3OBOQ6WRI2QnaFynd1DCjZ62+V0= github.com/googleapis/gax-go/v2 v2.0.4/go.mod h1:0Wqv26UfaUD9n4G6kQubkQ+KchISgw+vpHVxEJEs9eg= github.com/googleapis/gax-go/v2 v2.0.5/go.mod h1:DWXyrwAJ9X0FpwwEdw+IPEYBICEFu5mhpdKc/us6bOk= +github.com/googleapis/gax-go/v2 v2.1.0/go.mod h1:Q3nei7sK6ybPYH7twZdmQpAd1MKb7pfu6SK+H1/DsU0= +github.com/googleapis/gax-go/v2 v2.1.1/go.mod h1:hddJymUZASv3XPyGkUpKj8pPO47Rmb0eJc8R6ouapiM= +github.com/googleapis/gax-go/v2 v2.2.0/go.mod h1:as02EH8zWkzwUoLbBaFeQ+arQaj/OthfcblKl4IGNaM= +github.com/googleapis/gax-go/v2 v2.3.0/go.mod h1:b8LNqSzNabLiUpXKkY7HAR5jr6bIT99EXz9pXxye9YM= +github.com/googleapis/gax-go/v2 v2.4.0/go.mod h1:XOTVJ59hdnfJLIP/dh8n5CGryZR2LxK9wbMD5+iXC6c= +github.com/googleapis/gax-go/v2 v2.5.1/go.mod h1:h6B0KMMFNtI2ddbGJn3T3ZbwkeT6yqEF02fYlzkUCyo= +github.com/googleapis/gax-go/v2 v2.6.0/go.mod h1:1mjbznJAPHFpesgE5ucqfYEscaz5kMdcIDwU/6+DDoY= +github.com/googleapis/gax-go/v2 v2.7.0/go.mod h1:TEop28CZZQ2y+c0VxMUmu1lV+fQx57QpBWsYpwqHJx8= +github.com/googleapis/gax-go/v2 v2.7.1/go.mod h1:4orTrqY6hXxxaUL4LHIPl6lGo8vAE38/qKbhSAKP6QI= +github.com/googleapis/gax-go/v2 v2.8.0/go.mod h1:4orTrqY6hXxxaUL4LHIPl6lGo8vAE38/qKbhSAKP6QI= +github.com/googleapis/gax-go/v2 v2.10.0/go.mod h1:4UOEnMCrxsSqQ940WnTiD6qJ63le2ev3xfyagutxiPw= +github.com/googleapis/gax-go/v2 v2.11.0/go.mod h1:DxmR61SGKkGLa2xigwuZIQpkCI2S5iydzRfb3peWZJI= +github.com/googleapis/gax-go/v2 v2.12.0/go.mod h1:y+aIqrI5eb1YGMVJfuV3185Ts/D7qKpsEkdD5+I6QGU= github.com/googleapis/gax-go/v2 v2.12.2 h1:mhN09QQW1jEWeMF74zGR81R30z4VJzjZsfkUhuHF+DA= github.com/googleapis/gax-go/v2 v2.12.2/go.mod h1:61M8vcyyXR2kqKFxKrfA22jaA8JGF7Dc8App1U3H6jc= +github.com/googleapis/go-type-adapters v1.0.0/go.mod h1:zHW75FOG2aur7gAO2B+MLby+cLsWGBF62rFAi7WjWO4= +github.com/googleapis/google-cloud-go-testing v0.0.0-20200911160855-bcd43fbb19e8/go.mod h1:dvDLG8qkwmyD9a/MJJN3XJcT3xFxOKAvTZGvuZmac9g= github.com/gordonklaus/ineffassign v0.1.0 h1:y2Gd/9I7MdY1oEIt+n+rowjBNDcLQq3RsH5hwJd0f9s= github.com/gordonklaus/ineffassign v0.1.0/go.mod h1:Qcp2HIAYhR7mNUVSIxZww3Guk4it82ghYcEXIAk+QT0= github.com/gorilla/handlers v1.5.1 h1:9lRY6j8DEeeBT10CvO9hGW0gmky0BprnvDI5vfhUHH4= @@ -460,6 +1308,8 @@ github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0 h1:Ovs26xHkKqVztRpIrF/92Bcuy github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0/go.mod h1:8NvIoxWQoOIhqOTXgfV/d3M/q6VIi02HzZEHgUlZvzk= github.com/grpc-ecosystem/grpc-gateway v1.16.0 h1:gmcG1KaJ57LophUzW0Hy8NmPhnMZb4M0+kPpLofRdBo= github.com/grpc-ecosystem/grpc-gateway v1.16.0/go.mod h1:BDjrQk3hbvj6Nolgz8mAMFbcEtjT1g+wF4CSlocrBnw= +github.com/grpc-ecosystem/grpc-gateway/v2 v2.7.0/go.mod h1:hgWBS7lorOAVIJEQMi4ZsPv9hVvWI6+ch50m39Pf2Ks= +github.com/grpc-ecosystem/grpc-gateway/v2 v2.11.3/go.mod h1:o//XUCC/F+yRGJoPO/VU0GSB0f8Nhgmxx0VIRUvaC0w= github.com/grpc-ecosystem/grpc-gateway/v2 v2.19.1 h1:/c3QmbOGMGTOumP2iT/rCwB7b0QDGLKzqOmktBjT+Is= github.com/grpc-ecosystem/grpc-gateway/v2 v2.19.1/go.mod h1:5SN9VR2LTsRFsrEC6FHgRbTWrTHu6tqPeKxEQv15giM= github.com/hashicorp/go-uuid v0.0.0-20180228145832-27454136f036/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro= @@ -473,7 +1323,9 @@ github.com/hexops/gotextdiff v1.0.3 h1:gitA9+qJrrTCsiCl7+kh75nPqQt1cx4ZkudSTLoUq github.com/hexops/gotextdiff v1.0.3/go.mod h1:pSWU5MAI3yDq+fZBTazCSJysOMbxWL1BSow5/V2vxeg= github.com/huandu/xstrings v1.3.1 h1:4jgBlKK6tLKFvO8u5pmYjG91cqytmDCDvGh7ECVFfFs= github.com/huandu/xstrings v1.3.1/go.mod h1:y5/lhBue+AyNmUVz9RLU9xbLR0o4KIIExikq4ovT0aE= +github.com/iancoleman/strcase v0.2.0/go.mod h1:iwCmte+B7n89clKwxIoIXy/HfoL7AsD47ZCWhYzw7ho= github.com/ianlancetaylor/demangle v0.0.0-20181102032728-5e5cf60278f6/go.mod h1:aSSvb/t6k1mPoxDqO4vJh6VOCGPwU4O0C2/Eqndh1Sc= +github.com/ianlancetaylor/demangle v0.0.0-20200824232613-28f6c0f3b639/go.mod h1:aSSvb/t6k1mPoxDqO4vJh6VOCGPwU4O0C2/Eqndh1Sc= github.com/ianlancetaylor/demangle v0.0.0-20210905161508-09a460cdf81d/go.mod h1:aYm2/VgdVmcIU8iMfdMvDMsRAQjcfZSKFby6HOFvi/w= github.com/imdario/mergo v0.3.11/go.mod h1:jmQim1M+e3UYxmgPu/WyfjB3N3VflVyUjjjwH0dnCYA= github.com/imdario/mergo v0.3.16 h1:wwQJbIsHYGMUyLSPrEq1CT16AhnhNJQ51+4fdHUnCl4= @@ -545,6 +1397,7 @@ github.com/klauspost/cpuid v1.3.1/go.mod h1:bYW4mA6ZgKPob1/Dlai2LviZJO7KGI3uoWLd github.com/klauspost/cpuid/v2 v2.0.9 h1:lgaqFMSdTdQYdZ04uHyN2d/eKdOMyi2YLSvlQIBFYa4= github.com/klauspost/cpuid/v2 v2.0.9/go.mod h1:FInQzS24/EEf25PyTYn52gqo7WaD8xa0213Md/qVLRg= github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= +github.com/kr/fs v0.1.0/go.mod h1:FFnZGqtBN9Gxj7eW1uZ42v5BccTP0vu6NEaFoC2HwRg= github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= github.com/kr/pretty v0.2.1/go.mod h1:ipq/a2n7PKx3OHsz4KJII5eveXtPO4qwEXGdVfWzfnI= github.com/kr/pretty v0.3.0/go.mod h1:640gp4NfQd8pI5XOwp5fnNeVWj67G7CFk/SaSQn7NBk= @@ -573,6 +1426,10 @@ github.com/lestrrat-go/option v1.0.1/go.mod h1:5ZHFbivi4xwXxhxY9XHDe2FHo6/Z7WWmt github.com/lufia/plan9stats v0.0.0-20211012122336-39d0f177ccd0/go.mod h1:zJYVVT2jmtg6P3p1VtQj7WsuWi/y4VnjVBn7F8KPB3I= github.com/lufia/plan9stats v0.0.0-20230326075908-cb1d2100619a h1:N9zuLhTvBSRt0gWSiJswwQ2HqDmtX/ZCDJURnKUt1Ik= github.com/lufia/plan9stats v0.0.0-20230326075908-cb1d2100619a/go.mod h1:JKx41uQRwqlTZabZc+kILPrO/3jlKnQ2Z8b7YiVw5cE= +github.com/lyft/protoc-gen-star v0.6.0/go.mod h1:TGAoBVkt8w7MPG72TrKIu85MIdXwDuzJYeZuUPFPNwA= +github.com/lyft/protoc-gen-star v0.6.1/go.mod h1:TGAoBVkt8w7MPG72TrKIu85MIdXwDuzJYeZuUPFPNwA= +github.com/lyft/protoc-gen-star/v2 v2.0.1/go.mod h1:RcCdONR2ScXaYnQC5tUzxzlpA3WVYF7/opLeUgcQs/o= +github.com/lyft/protoc-gen-star/v2 v2.0.3/go.mod h1:amey7yeodaJhXSbf/TlLvWiqQfLOSpEk//mLlc+axEk= github.com/magiconair/properties v1.8.0/go.mod h1:PppfXfuXeibc/6YijjN8zIbojt8czPbwD3XqdrwzmxQ= github.com/mattn/go-colorable v0.1.8/go.mod h1:u6P/XSegPjTcexA+o6vUJrdnUu04hMope9wVRipJSqc= github.com/mattn/go-colorable v0.1.13 h1:fFA4WZxdEF4tXPZVKMLwD8oUnCTTo08duU7wxecdEvA= @@ -587,6 +1444,7 @@ github.com/mattn/go-runewidth v0.0.12/go.mod h1:RAqKPSqVFrSLVXbA8x7dzmKdmGzieGRC github.com/mattn/go-runewidth v0.0.13/go.mod h1:Jdepj2loyihRzMpdS35Xk/zdY8IAYHsh153qUoGf23w= github.com/mattn/go-runewidth v0.0.16 h1:E5ScNMtiwvlvB5paMFdw9p4kSQzbXFikJ5SQO6TULQc= github.com/mattn/go-runewidth v0.0.16/go.mod h1:Jdepj2loyihRzMpdS35Xk/zdY8IAYHsh153qUoGf23w= +github.com/mattn/go-sqlite3 v1.14.14/go.mod h1:NyWgC/yNuGj7Q9rpYnZvas74GogHl5/Z4A/KQRfk6bU= github.com/mattn/go-sqlite3 v1.14.15/go.mod h1:2eHXhiwb8IkHr+BDWZGa96P6+rkvnG63S2DGjv9HUNg= github.com/matttproud/golang_protobuf_extensions v1.0.1/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0= github.com/mgechev/revive v1.5.0 h1:oaSmjA7rP8+HyoRuCgC531VHwnLH1AlJdjj+1AnQceQ= @@ -669,9 +1527,6 @@ github.com/pingcap/fn v1.0.0 h1:CyA6AxcOZkQh52wIqYlAmaVmF6EvrcqFywP463pjA8g= github.com/pingcap/fn v1.0.0/go.mod h1:u9WZ1ZiOD1RpNhcI42RucFh/lBuzTu6rw88a+oF2Z24= github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989 h1:surzm05a8C9dN8dIUmo4Be2+pMRb6f55i+UIYrluu2E= github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989/go.mod h1:O17XtbryoCJhkKGbT62+L2OlrniwqiGLSqrmdHCMzZw= -github.com/pingcap/kvproto v0.0.0-20191211054548-3c6b38ea5107/go.mod h1:WWLmULLO7l8IOcQG+t+ItJ3fEcrL5FxF0Wu+HrMy26w= -github.com/pingcap/kvproto v0.0.0-20241120022153-92b0414aeed8 h1:aNNifhc6xCjXKejjiNYtJJLFNMXnoDiXxkJIg1JErQE= -github.com/pingcap/kvproto v0.0.0-20241120022153-92b0414aeed8/go.mod h1:rXxWk2UnwfUhLXha1jxRWPADw9eMZGWEWCg92Tgmb/8= github.com/pingcap/log v0.0.0-20210625125904-98ed8e2eb1c7/go.mod h1:8AanEdAHATuRurdGxZXBz0At+9avep+ub7U1AGYLIMM= github.com/pingcap/log v1.1.0/go.mod h1:DWQW5jICDR7UJh4HtxXSM20Churx4CQL0fwL/SoOSA4= github.com/pingcap/log v1.1.1-0.20240314023424-862ccc32f18d h1:y3EueKVfVykdpTyfUnQGqft0ud+xVFuCdp1XkVL0X1E= @@ -687,6 +1542,8 @@ github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINE github.com/pkg/errors v0.9.1 h1:FEBLx1zS214owpjy7qsBeixbURkuhQAwrK5UwLGTwt4= github.com/pkg/errors v0.9.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/profile v1.2.1/go.mod h1:hJw3o1OdXxsrSjjVksARp5W95eeEaEfptyVZyv6JUPA= +github.com/pkg/sftp v1.10.1/go.mod h1:lYOWFsE0bwd1+KfKJaKeuokY15vzFx25BLbzYYoAxZI= +github.com/pkg/sftp v1.13.1/go.mod h1:3HaPG6Dq1ILlpPZRO0HVMrsydcdLt6HRDccSgb87qRg= github.com/pkg/xattr v0.4.9 h1:5883YPCtkSd8LFbs13nXplj9g9tlrwoJRjgpgMu1/fE= github.com/pkg/xattr v0.4.9/go.mod h1:di8WF84zAKk8jzR1UBTEWh9AUlIZZ7M/JNt8e9B6ktU= github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= @@ -702,6 +1559,9 @@ github.com/prometheus/client_golang v1.20.5 h1:cxppBPuYhUnsO6yo/aoRol4L7q7UFfdm+ github.com/prometheus/client_golang v1.20.5/go.mod h1:PIEt8X02hGcP8JWbeHyeZ53Y/jReSnHgO035n//V5WE= github.com/prometheus/client_model v0.0.0-20180712105110-5c3871d89910/go.mod h1:MbSGuTsp3dbXC40dX6PRTWyKYBIrTGTE9sqQNg2J8bo= github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= +github.com/prometheus/client_model v0.2.0/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= +github.com/prometheus/client_model v0.3.0/go.mod h1:LDGWKZIo7rky3hgvBe+caln+Dr3dPggB5dvjtD7w9+w= +github.com/prometheus/client_model v0.4.0/go.mod h1:oMQmHW1/JoDwqLtg57MGgP/Fb1CJEYF2imWWhWtMkYU= github.com/prometheus/client_model v0.6.1 h1:ZKSh/rekM+n3CeS952MLRAdFwIKqeY8b62p8ais2e9E= github.com/prometheus/client_model v0.6.1/go.mod h1:OrxVMOVHjw3lKMa8+x6HeMGkHMQyHDk9E3jmP2AmGiY= github.com/prometheus/common v0.0.0-20181020173914-7e9e6cabbd39/go.mod h1:daVV7qP5qjZbuso7PdcryaAu0sAZbrN9i7WWcTMWvro= @@ -773,9 +1633,13 @@ github.com/sourcegraph/appdash v0.0.0-20190731080439-ebfcffb1b5c0 h1:IJ3DuWHPTJr github.com/sourcegraph/appdash v0.0.0-20190731080439-ebfcffb1b5c0/go.mod h1:V952P4GGl1v/MMynLwxVdWEbSZJx+n0oOO3ljnez+WU= github.com/sourcegraph/appdash-data v0.0.0-20151005221446-73f23eafcf67 h1:8ZnTA26bBOoPkAbbitKPgNlpw0Bwt7ZlpYgZWHWJR/w= github.com/sourcegraph/appdash-data v0.0.0-20151005221446-73f23eafcf67/go.mod h1:tNZjgbYncKL5HxvDULAr/mWDmFz4B7H8yrXEDlnoIiw= +github.com/spaolacci/murmur3 v0.0.0-20180118202830-f09979ecbc72/go.mod h1:JwIasOWyU6f++ZhiEuf87xNszmSA2myDM2Kzu9HwQUA= github.com/spf13/afero v1.1.2/go.mod h1:j4pytiNVoe2o6bmDsKpLACNPDBIoEAkihy7loJ1B0CQ= github.com/spf13/afero v1.2.1/go.mod h1:9ZxEEn6pIJ8Rxe320qSDBk6AsU0r9pR7Q4OcevTdifk= github.com/spf13/afero v1.2.2/go.mod h1:9ZxEEn6pIJ8Rxe320qSDBk6AsU0r9pR7Q4OcevTdifk= +github.com/spf13/afero v1.3.3/go.mod h1:5KUK8ByomD5Ti5Artl0RtHeI5pTF7MIDuXL3yY520V4= +github.com/spf13/afero v1.6.0/go.mod h1:Ai8FlHk4v/PARR026UzYexafAt9roJ7LcLMAmO6Z93I= +github.com/spf13/afero v1.9.2/go.mod h1:iUV7ddyEEZPO5gA3zD4fJt6iStLlL+Lg4m2cihcDf8Y= github.com/spf13/cast v1.3.0/go.mod h1:Qx5cxh0v+4UWYiBimWS+eyWzqEqokIECu5etghLkUJE= github.com/spf13/cast v1.3.1/go.mod h1:Qx5cxh0v+4UWYiBimWS+eyWzqEqokIECu5etghLkUJE= github.com/spf13/cast v1.5.0 h1:rj3WzYc11XZaIZMPKmwP96zkFEnnAmV8s6XbB2aY32w= @@ -809,6 +1673,8 @@ github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/ github.com/stretchr/testify v1.7.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/stretchr/testify v1.8.0/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO+kdMU+MU= github.com/stretchr/testify v1.8.1/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4= +github.com/stretchr/testify v1.8.2/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4= +github.com/stretchr/testify v1.8.3/go.mod h1:sz/lmYIOXD/1dqDmKjjqLyZ2RngseejIcXlSw2iwfAo= github.com/stretchr/testify v1.8.4/go.mod h1:sz/lmYIOXD/1dqDmKjjqLyZ2RngseejIcXlSw2iwfAo= github.com/stretchr/testify v1.9.0 h1:HtqpIVDClZ4nwg75+f6Lvsy/wHu+3BoSGCbBAcpTsTg= github.com/stretchr/testify v1.9.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf8C91i36aY= @@ -865,6 +1731,8 @@ github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9dec github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= github.com/yuin/goldmark v1.4.1/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= github.com/yuin/goldmark v1.4.13/go.mod h1:6yULJ656Px+3vBD8DxQVa3kxgyrAnzto9xy5taEt/CY= +github.com/yujuncen/kvproto v0.0.0-20241125070101-be2e12233a90 h1:EyTrFe4KEIMHfvngHkyz0yb2EzuANIvFUSpVjAqkhu4= +github.com/yujuncen/kvproto v0.0.0-20241125070101-be2e12233a90/go.mod h1:rXxWk2UnwfUhLXha1jxRWPADw9eMZGWEWCg92Tgmb/8= github.com/yusufpapurcu/wmi v1.2.2/go.mod h1:SBZ9tNy3G9/m5Oi98Zks0QjeHVDvuK0qfxQmPyzfmi0= github.com/yusufpapurcu/wmi v1.2.4 h1:zFUKzehAFReQwLys1b/iSMl+JQGSCSjtVqQn9bBrPo0= github.com/yusufpapurcu/wmi v1.2.4/go.mod h1:SBZ9tNy3G9/m5Oi98Zks0QjeHVDvuK0qfxQmPyzfmi0= @@ -898,6 +1766,8 @@ go.opencensus.io v0.22.0/go.mod h1:+kGneAE2xo2IficOXnaByMWTGM9T73dGwxeWcUqIpI8= go.opencensus.io v0.22.2/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= go.opencensus.io v0.22.3/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= go.opencensus.io v0.22.4/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= +go.opencensus.io v0.22.5/go.mod h1:5pWMHQbX5EPX2/62yrJeAkowc+lfs/XD7Uxpq3pI6kk= +go.opencensus.io v0.23.0/go.mod h1:XItmlyltB5F7CS4xOC1DcqMoFqwtC6OG2xF7mCv7P7E= go.opencensus.io v0.24.0 h1:y73uSU6J157QMP2kn2r30vwW1A2W2WFwSCGnAVxeaD0= go.opencensus.io v0.24.0/go.mod h1:vNK8G9p7aAivkbmorf4v+7Hgx+Zs0yY+0fOtgBfjQKo= go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.49.0 h1:4Pp6oUg3+e/6M4C0A/3kJ2VYa++dsWVTtGgLVj5xtHg= @@ -917,6 +1787,8 @@ go.opentelemetry.io/otel/sdk v1.24.0/go.mod h1:KVrIYw6tEubO9E96HQpcmpTKDVn9gdv35 go.opentelemetry.io/otel/trace v1.24.0 h1:CsKnnL4dUAr/0llH9FKuc698G04IrpWV0MQA/Y1YELI= go.opentelemetry.io/otel/trace v1.24.0/go.mod h1:HPc3Xr/cOApsBI154IU0OI0HJexz+aw5uPdbs3UCjNU= go.opentelemetry.io/proto/otlp v0.7.0/go.mod h1:PqfVotwruBrMGOCsRd/89rSnXhoiJIqeYNgFYFoEGnI= +go.opentelemetry.io/proto/otlp v0.15.0/go.mod h1:H7XAot3MsfNsj7EXtrA2q5xSNQ10UqI405h3+duxN4U= +go.opentelemetry.io/proto/otlp v0.19.0/go.mod h1:H7XAot3MsfNsj7EXtrA2q5xSNQ10UqI405h3+duxN4U= go.opentelemetry.io/proto/otlp v1.1.0 h1:2Di21piLrCqJ3U3eXGCTPHE9R8Nh+0uglSnOyxikMeI= go.opentelemetry.io/proto/otlp v1.1.0/go.mod h1:GpBHCBWiqvVLDqmHZsoMM3C5ySeKTC7ej/RNTae6MdY= go.starlark.net v0.0.0-20210223155950-e043a3d3c984/go.mod h1:t3mmBBPzAVvK0L0n1drDmrQsJ8FoIx4INCqVMTr/Zo0= @@ -956,12 +1828,21 @@ golang.org/x/crypto v0.0.0-20181203042331-505ab145d0a9/go.mod h1:6SG95UA2DQfeDnf golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= golang.org/x/crypto v0.0.0-20190510104115-cbcb75029529/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20190605123033-f99c8df09eb5/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= +golang.org/x/crypto v0.0.0-20190820162420-60c769a6c586/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20200414173820-0848c9571904/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= +golang.org/x/crypto v0.0.0-20210421170649-83a5a9bb288b/go.mod h1:T9bdIzuCu7OtxOm1hfPfRQxPLYneinmdGuTeoZ9dtd4= golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= +golang.org/x/crypto v0.0.0-20211108221036-ceb1ce70b4fa/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= +golang.org/x/crypto v0.0.0-20220314234659-1baeb1ce4c0b/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= golang.org/x/crypto v0.0.0-20220518034528-6f7dac969898/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= +golang.org/x/crypto v0.1.0/go.mod h1:RecgLatLF4+eUMCP1PoPZQb+cVrJcOPbHkTkbkB9sbw= golang.org/x/crypto v0.7.0/go.mod h1:pYwdfH91IfpZVANVyUOhSIPZaFoJGxTFbZhFTx+dXZU= +golang.org/x/crypto v0.9.0/go.mod h1:yrmDGqONDYtNj3tH8X9dzUun2m2lzPa9ngI6/RUPGR0= +golang.org/x/crypto v0.10.0/go.mod h1:o4eNf7Ede1fv+hwOwZsTHl9EsPFO6q6ZvYR8vYfY45I= +golang.org/x/crypto v0.11.0/go.mod h1:xgJhtzW8F9jGdVFWZESrid1U1bjeNy4zgy5cRr/CIio= +golang.org/x/crypto v0.12.0/go.mod h1:NF0Gs7EO5K4qLn+Ylc+fih8BSTeIjAP05siRnAh98yw= golang.org/x/crypto v0.13.0/go.mod h1:y6Z2r+Rw4iayiXXAIxJIDAJ1zMW4yaTpebo8fPOliYc= golang.org/x/crypto v0.14.0/go.mod h1:MVFd36DqK4CsrnJYDkBA3VC4m2GkXAM0PvzMCn4JQf4= golang.org/x/crypto v0.29.0 h1:L5SG1JTTXupVV3n6sUqMTeWbjAyfPwoda2DLX8J8FrQ= @@ -1008,6 +1889,8 @@ golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHl golang.org/x/lint v0.0.0-20191125180803-fdd1cda4f05f/go.mod h1:5qLYkcX4OjUUV8bRuDixDT3tpyyb+LUpUlRWLxfhWrs= golang.org/x/lint v0.0.0-20200130185559-910be7a94367/go.mod h1:3xt1FjdF8hUf6vQPIChWIBhFzV8gjjsPE/fR3IyQdNY= golang.org/x/lint v0.0.0-20200302205851-738671d3881b/go.mod h1:3xt1FjdF8hUf6vQPIChWIBhFzV8gjjsPE/fR3IyQdNY= +golang.org/x/lint v0.0.0-20201208152925-83fdc39ff7b5/go.mod h1:3xt1FjdF8hUf6vQPIChWIBhFzV8gjjsPE/fR3IyQdNY= +golang.org/x/lint v0.0.0-20210508222113-6edffad5e616/go.mod h1:3xt1FjdF8hUf6vQPIChWIBhFzV8gjjsPE/fR3IyQdNY= golang.org/x/mobile v0.0.0-20190312151609-d3739f865fa6/go.mod h1:z+o9i4GpDbdi3rU15maQ/Ox0txvL9dWGYEHz965HBQE= golang.org/x/mobile v0.0.0-20190719004257-d2bd2a29d028/go.mod h1:E/iHnbuqvinMTCcRqshq8CkpyQDoeVncDDYHnLhea+o= golang.org/x/mod v0.0.0-20190513183733-4bf6d317e70e/go.mod h1:mXi4GBBbnImb6dmsKGUJ2LatrhH/nqhxcFungHvyanc= @@ -1016,20 +1899,24 @@ golang.org/x/mod v0.1.1-0.20191105210325-c90efee705ee/go.mod h1:QqPTAvyqsEbceGzB golang.org/x/mod v0.1.1-0.20191107180719-034126e5016b/go.mod h1:QqPTAvyqsEbceGzBzNggFXnrqF1CaUcvgkdR5Ot7KZg= golang.org/x/mod v0.2.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.3.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= +golang.org/x/mod v0.4.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.4.1/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.4.2/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= +golang.org/x/mod v0.5.0/go.mod h1:5OXOZSfqPIIbmVBIIKWRFfZjPR0E5r58TLhUjH0a2Ro= golang.org/x/mod v0.5.1/go.mod h1:5OXOZSfqPIIbmVBIIKWRFfZjPR0E5r58TLhUjH0a2Ro= golang.org/x/mod v0.6.0-dev.0.20220106191415-9b9b3d81d5e3/go.mod h1:3p9vT2HGsQu2K1YbXdKPJLVgG5VJdoTa1poYQBtP1AY= golang.org/x/mod v0.6.0-dev.0.20220419223038-86c51ed26bb4/go.mod h1:jJ57K6gSWd91VN4djpZkiMVwK6gcyfeH4XE8wZrZaV4= +golang.org/x/mod v0.7.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= golang.org/x/mod v0.8.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= +golang.org/x/mod v0.9.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= golang.org/x/mod v0.10.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= +golang.org/x/mod v0.11.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= golang.org/x/mod v0.12.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= golang.org/x/mod v0.13.0/go.mod h1:hTbmBsO62+eylJbnUtE2MGJUyE7QWk4xUqPFrRgJ+7c= golang.org/x/mod v0.22.0 h1:D4nJWe9zXqHOmWqj4VMOJhvzj7bEZg4wEYa759z1pH4= golang.org/x/mod v0.22.0/go.mod h1:6SkKJ3Xj0I0BrPOZoBy3bdMptDDU9oJrpohJ3eWZ1fY= golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20181005035420-146acd28ed58/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20190108225652-1e06a53dbb7e/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20190213061140-3a22650c66bd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= @@ -1055,22 +1942,45 @@ golang.org/x/net v0.0.0-20200625001655-4c5254603344/go.mod h1:/O7V0waA8r7cgGh81R golang.org/x/net v0.0.0-20200707034311-ab3426394381/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= golang.org/x/net v0.0.0-20200822124328-c89045814202/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= +golang.org/x/net v0.0.0-20201031054903-ff519b6c9102/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= golang.org/x/net v0.0.0-20201110031124-69a78807bb2b/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= golang.org/x/net v0.0.0-20201202161906-c7110b5ffcbb/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= +golang.org/x/net v0.0.0-20201209123823-ac852fbbde11/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= +golang.org/x/net v0.0.0-20201224014010-6772e930b67b/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= +golang.org/x/net v0.0.0-20210119194325-5f4716e94777/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= golang.org/x/net v0.0.0-20210226172049-e18ecbb05110/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= +golang.org/x/net v0.0.0-20210316092652-d523dce5a7f4/go.mod h1:RBQZq4jEuRlivfhVLdyRGr576XBO4/greRjx4P4O3yc= golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= +golang.org/x/net v0.0.0-20210503060351-7fd8e65b6420/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= +golang.org/x/net v0.0.0-20210813160813-60bc85c4be6d/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/net v0.0.0-20211015210444-4f30a5c0130f/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/net v0.0.0-20211112202133-69e39bad7dc2/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/net v0.0.0-20211123203042-d83791d6bcd9/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/net v0.0.0-20220127200216-cd36cc0744dd/go.mod h1:CfG3xpIq0wQ8r1q4Su4UZFWDARRcnwPjda9FqA0JpMk= +golang.org/x/net v0.0.0-20220225172249-27dd8689420f/go.mod h1:CfG3xpIq0wQ8r1q4Su4UZFWDARRcnwPjda9FqA0JpMk= +golang.org/x/net v0.0.0-20220325170049-de3da57026de/go.mod h1:CfG3xpIq0wQ8r1q4Su4UZFWDARRcnwPjda9FqA0JpMk= +golang.org/x/net v0.0.0-20220412020605-290c469a71a5/go.mod h1:CfG3xpIq0wQ8r1q4Su4UZFWDARRcnwPjda9FqA0JpMk= +golang.org/x/net v0.0.0-20220425223048-2871e0cb64e4/go.mod h1:CfG3xpIq0wQ8r1q4Su4UZFWDARRcnwPjda9FqA0JpMk= golang.org/x/net v0.0.0-20220517181318-183a9ca12b87/go.mod h1:CfG3xpIq0wQ8r1q4Su4UZFWDARRcnwPjda9FqA0JpMk= +golang.org/x/net v0.0.0-20220607020251-c690dde0001d/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c= +golang.org/x/net v0.0.0-20220617184016-355a448f1bc9/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c= +golang.org/x/net v0.0.0-20220624214902-1bab6f366d9e/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c= golang.org/x/net v0.0.0-20220722155237-a158d28d115b/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c= +golang.org/x/net v0.0.0-20220909164309-bea034e7d591/go.mod h1:YDH+HFinaLZZlnHAfSS6ZXJJ9M9t4Dl22yv3iI2vPwk= +golang.org/x/net v0.0.0-20221012135044-0b7e1fb9d458/go.mod h1:YDH+HFinaLZZlnHAfSS6ZXJJ9M9t4Dl22yv3iI2vPwk= +golang.org/x/net v0.0.0-20221014081412-f15817d10f9b/go.mod h1:YDH+HFinaLZZlnHAfSS6ZXJJ9M9t4Dl22yv3iI2vPwk= golang.org/x/net v0.1.0/go.mod h1:Cx3nUiGt4eDBEyega/BKRp+/AlGL8hYe7U9odMt2Cco= +golang.org/x/net v0.2.0/go.mod h1:KqCZLdyyvdV855qA2rE3GC2aiw5xGR5TEjj8smXukLY= +golang.org/x/net v0.4.0/go.mod h1:MBQ8lrhLObU/6UmLb4fmbmk5OcyYmqtbGd/9yIeKjEE= +golang.org/x/net v0.5.0/go.mod h1:DivGGAXEgPSlEBzxGzZI+ZLohi+xUj054jfeKui00ws= golang.org/x/net v0.6.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= golang.org/x/net v0.7.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= golang.org/x/net v0.8.0/go.mod h1:QVkue5JL9kW//ek3r6jTKnTFis1tRmNAW2P1shuFdJc= golang.org/x/net v0.9.0/go.mod h1:d48xBJpPfHeWQsugry2m+kC02ZBRGRgulfHnEXEuWns= golang.org/x/net v0.10.0/go.mod h1:0qNGK6F8kojg2nk9dLZ2mShWaEBan6FAoqfSigmmuDg= +golang.org/x/net v0.11.0/go.mod h1:2L/ixqYpgIVXmeoSA/4Lu7BzTG4KIyPIryS4IsOd1oQ= +golang.org/x/net v0.12.0/go.mod h1:zEVYFnQC7m/vmpQFELhcD1EWkZlX69l4oqgmer6hfKA= +golang.org/x/net v0.14.0/go.mod h1:PpSgVXXLK0OxS0F31C1/tv6XNguvCrnXIDrFMspZIUI= golang.org/x/net v0.15.0/go.mod h1:idbUs1IY1+zTqbi8yxTbhexhEEk5ur9LInksu6HrEpk= golang.org/x/net v0.16.0/go.mod h1:NxSsAGuq816PNPmqtQdLE42eU2Fs7NoRIZrHJAlaCOE= golang.org/x/net v0.17.0/go.mod h1:NxSsAGuq816PNPmqtQdLE42eU2Fs7NoRIZrHJAlaCOE= @@ -1081,7 +1991,33 @@ golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4Iltr golang.org/x/oauth2 v0.0.0-20190604053449-0f29369cfe45/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/oauth2 v0.0.0-20191202225959-858c2ad4c8b6/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/oauth2 v0.0.0-20200107190931-bf48bf16ab8d/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= +golang.org/x/oauth2 v0.0.0-20200902213428-5d25da1a8d43/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= +golang.org/x/oauth2 v0.0.0-20201109201403-9fd604954f58/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= +golang.org/x/oauth2 v0.0.0-20201208152858-08078c50e5b5/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= +golang.org/x/oauth2 v0.0.0-20210218202405-ba52d332ba99/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= +golang.org/x/oauth2 v0.0.0-20210220000619-9bb904979d93/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= +golang.org/x/oauth2 v0.0.0-20210313182246-cd4f82c27b84/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= +golang.org/x/oauth2 v0.0.0-20210514164344-f6687ab2804c/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= +golang.org/x/oauth2 v0.0.0-20210628180205-a41e5a781914/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= +golang.org/x/oauth2 v0.0.0-20210805134026-6f1e6394065a/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= +golang.org/x/oauth2 v0.0.0-20210819190943-2bc19b11175f/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= +golang.org/x/oauth2 v0.0.0-20211104180415-d3ed0bb246c8/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= +golang.org/x/oauth2 v0.0.0-20220223155221-ee480838109b/go.mod h1:DAh4E804XQdzx2j+YRIaUnCqCV2RuMz24cGBJ5QYIrc= +golang.org/x/oauth2 v0.0.0-20220309155454-6242fa91716a/go.mod h1:DAh4E804XQdzx2j+YRIaUnCqCV2RuMz24cGBJ5QYIrc= golang.org/x/oauth2 v0.0.0-20220411215720-9780585627b5/go.mod h1:DAh4E804XQdzx2j+YRIaUnCqCV2RuMz24cGBJ5QYIrc= +golang.org/x/oauth2 v0.0.0-20220608161450-d0670ef3b1eb/go.mod h1:jaDAt6Dkxork7LmZnYtzbRWj0W47D86a3TGe0YHBvmE= +golang.org/x/oauth2 v0.0.0-20220622183110-fd043fe589d2/go.mod h1:jaDAt6Dkxork7LmZnYtzbRWj0W47D86a3TGe0YHBvmE= +golang.org/x/oauth2 v0.0.0-20220822191816-0ebed06d0094/go.mod h1:h4gKUeWbJ4rQPri7E0u6Gs4e9Ri2zaLxzw5DI5XGrYg= +golang.org/x/oauth2 v0.0.0-20220909003341-f21342109be1/go.mod h1:h4gKUeWbJ4rQPri7E0u6Gs4e9Ri2zaLxzw5DI5XGrYg= +golang.org/x/oauth2 v0.0.0-20221006150949-b44042a4b9c1/go.mod h1:h4gKUeWbJ4rQPri7E0u6Gs4e9Ri2zaLxzw5DI5XGrYg= +golang.org/x/oauth2 v0.0.0-20221014153046-6fdb5e3db783/go.mod h1:h4gKUeWbJ4rQPri7E0u6Gs4e9Ri2zaLxzw5DI5XGrYg= +golang.org/x/oauth2 v0.4.0/go.mod h1:RznEsdpjGAINPTOF0UH/t+xJ75L18YO3Ho6Pyn+uRec= +golang.org/x/oauth2 v0.5.0/go.mod h1:9/XBHVqLaWO3/BRHs5jbpYCnOZVjj5V0ndyaAM7KB4I= +golang.org/x/oauth2 v0.6.0/go.mod h1:ycmewcwgD4Rpr3eZJLSB4Kyyljb3qDh40vJ8STE5HKw= +golang.org/x/oauth2 v0.7.0/go.mod h1:hPLQkd9LyjfXTiRohC/41GhcFqxisoUQ99sCUOHO9x4= +golang.org/x/oauth2 v0.8.0/go.mod h1:yr7u4HXZRm1R1kBWqr/xKNqewf0plRYoB7sla+BCIXE= +golang.org/x/oauth2 v0.10.0/go.mod h1:kTpgurOux7LqtuxjuyZa4Gj2gdezIt/jQtGnNFfypQI= +golang.org/x/oauth2 v0.11.0/go.mod h1:LdF7O/8bLR/qWK9DrpXmbHLTouvRHK0SgJl0GmDBchk= golang.org/x/oauth2 v0.23.0 h1:PbgcYx2W7i4LvjJWEbf0ngHV6qJYr86PkAV3bXdLEbs= golang.org/x/oauth2 v0.23.0/go.mod h1:XYTD2NtWslqkgxebSiOHnXEap4TF09sJSc7H1sXbhtI= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -1093,9 +2029,14 @@ golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20200317015054-43a5402ce75a/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20200625203802-6e8e738ad208/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20201207232520-09787c993a3a/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20220601150217-0de741cfad7f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20220819030929-7fc1605a5dde/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20220929204114-8fcdb60fdcc0/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.1.0/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.2.0/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.3.0/go.mod h1:FU7BRWz2tNW+3quACPkgCx/L+uEAv1htQ0V83Z9Rj+Y= golang.org/x/sync v0.4.0/go.mod h1:FU7BRWz2tNW+3quACPkgCx/L+uEAv1htQ0V83Z9Rj+Y= golang.org/x/sync v0.9.0 h1:fEo0HyrW1GIgZdpbhCRO0PkJajUS5H9IFUztCgEo2jQ= @@ -1131,36 +2072,72 @@ golang.org/x/sys v0.0.0-20200511232937-7e40ca221e25/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20200515095857-1151b9dac4a9/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200523222454-059865788121/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200803210538-64077c9b5642/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200905004654-be1d3432aa8f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20201201145000-ef89a241ccb3/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20201204225414-ed752295db88/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20210104204734-6f8348627aad/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210119212857-b64e53b001e4/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210124154548-22da62e12c0c/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20210220050731-9a76102bfb43/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20210225134936-a50acf3fe073/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210304124612-50617c2ba197/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20210305230114-8fe3ee5dd75b/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20210315160823-c6e025ad8005/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20210320140829-1e4c9ba3b0c4/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210330210617-4fbd30eecc44/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210403161142-5e06dd20ab57/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210423082822-04245dca01da/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20210423185535-09eb48e85fd7/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210510120138-977fb7262007/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20210514084401-e8d321eab015/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20210603125802-9665404d3644/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20210616094352-59db8d763f22/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20210630005230-0f9fa26af87c/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20210806184541-e5e7981a1069/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210816074244-15123e1e1f71/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20210816183151-1e6c022a8912/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20210823070655-63515b42dcdf/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20210908233432-aa78b53d3365/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20211007075335-d3039528d8ac/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20211013075003-97ac67df715c/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20211019181941-9d821ace8654/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20211025201205-69cdffdb9359/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20211124211545-fe61309f8881/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20211210111614-af8b64212486/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20211216021012-1d35b9e2eb4e/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20220128215802-99c3d69c2c27/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20220209214540-3681064d5158/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20220227234510-4e6760a101f9/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20220328115105-d36c6a25d886/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220408201424-a24fb2fb8a0f/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20220412211240-33da011f77ad/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20220502124256-b6088ccd6cba/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20220503163025-988cb79eb6c6/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220517195934-5e4e11fc645e/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220520151302-bc2c85ada10a/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20220610221304-9f5ed59c137d/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20220615213510-4f61da869c0c/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20220624220833-87e55d714810/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220715151400-c0bba94af5f8/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220722155257-8c9f86f7a55f/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20220728004956-3c1f35247d10/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220811171246-fbc7d0a398ab/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20220829200755-d48e67d00261/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220909162455-aba9fc2a8ff2/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20221010170243-090e33056c14/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.1.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.2.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.3.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.4.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.5.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.6.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.7.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.8.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.9.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.10.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.11.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.12.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.13.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= @@ -1169,10 +2146,16 @@ golang.org/x/sys v0.27.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= golang.org/x/term v0.1.0/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= +golang.org/x/term v0.2.0/go.mod h1:TVmDHMZPmdnySmBfhjOoOdhjzdE1h4u1VwSiw2l1Nuc= +golang.org/x/term v0.3.0/go.mod h1:q750SLmJuPmVoN1blW3UFBPREJfb1KmY3vwxfr+nFDA= +golang.org/x/term v0.4.0/go.mod h1:9P2UbLfCdcvo3p/nzKvsmas4TnlujnuoV9hGgYzW1lQ= golang.org/x/term v0.5.0/go.mod h1:jMB1sMXY+tzblOD4FWmEbocvup2/aLOaQEp7JmGp78k= golang.org/x/term v0.6.0/go.mod h1:m6U89DPEgQRMq3DNkDClhWw02AUbt2daBVO4cn4Hv9U= golang.org/x/term v0.7.0/go.mod h1:P32HKFT3hSsZrRxla30E9HqToFYAQPCMs/zFMBUFqPY= golang.org/x/term v0.8.0/go.mod h1:xPskH00ivmX89bAKVGSKKtLOWNx2+17Eiy94tnKShWo= +golang.org/x/term v0.9.0/go.mod h1:M6DEAAIenWoTxdKrOltXcmDY3rSplQUkrvaDU5FcQyo= +golang.org/x/term v0.10.0/go.mod h1:lpqdcUyK/oCiQxvxVrppt5ggO2KCZ5QblwqPnfZ6d5o= +golang.org/x/term v0.11.0/go.mod h1:zC9APTIj3jG3FdV/Ons+XE1riIZXG4aZ4GTHiPZJPIU= golang.org/x/term v0.12.0/go.mod h1:owVbMEjm3cBLCHdkQu9b1opXd4ETQWc3BhuQGKgXgvU= golang.org/x/term v0.13.0/go.mod h1:LTmsnFJwVN6bCy1rVCoS+qHT1HhALEFxKncY3WNNh4U= golang.org/x/term v0.26.0 h1:WEQa6V3Gja/BhNxg540hBip/kkaYtRg3cxg4oXSw4AU= @@ -1182,19 +2165,28 @@ golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.1-0.20180807135948-17ff2d5776d2/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk= golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= +golang.org/x/text v0.3.4/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.5/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.6/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.7/go.mod h1:u+2+/6zg+i71rQMx5EYifcz6MCKuco9NR6JIITiCfzQ= +golang.org/x/text v0.3.8/go.mod h1:E6s5w1FMmriuDzIBO73fBruAKo1PCIq6d2Q6DHfQ8WQ= golang.org/x/text v0.4.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8= +golang.org/x/text v0.5.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8= +golang.org/x/text v0.6.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8= golang.org/x/text v0.7.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8= golang.org/x/text v0.8.0/go.mod h1:e1OnstbJyHTd6l/uOt8jFFHp6TRDWZR/bV3emEE/zU8= golang.org/x/text v0.9.0/go.mod h1:e1OnstbJyHTd6l/uOt8jFFHp6TRDWZR/bV3emEE/zU8= +golang.org/x/text v0.10.0/go.mod h1:TvPlkZtksWOMsz7fbANvkp4WM8x/WCo/om8BMLbz+aE= +golang.org/x/text v0.11.0/go.mod h1:TvPlkZtksWOMsz7fbANvkp4WM8x/WCo/om8BMLbz+aE= +golang.org/x/text v0.12.0/go.mod h1:TvPlkZtksWOMsz7fbANvkp4WM8x/WCo/om8BMLbz+aE= golang.org/x/text v0.13.0/go.mod h1:TvPlkZtksWOMsz7fbANvkp4WM8x/WCo/om8BMLbz+aE= golang.org/x/text v0.20.0 h1:gK/Kv2otX8gz+wn7Rmb3vT96ZwuoxnQlY+HlJVj7Qug= golang.org/x/text v0.20.0/go.mod h1:D4IsuqiFMhST5bX19pQ9ikHC2GsaKyk/oF+pn3ducp4= golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20190308202827-9d24e82272b4/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20191024005414-555d28b269f0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= +golang.org/x/time v0.0.0-20220922220347-f3bd1da661af/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= +golang.org/x/time v0.1.0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.3.0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.7.0 h1:ntUhktv3OPE6TgYxXWv9vKvUSJyIFJlyohwbkEwPrKQ= golang.org/x/time v0.7.0/go.mod h1:3BpzKBy/shNhVucY/MWOyx10tF3SFh9QdLuxbVysPQM= @@ -1248,18 +2240,31 @@ golang.org/x/tools v0.0.0-20200619180055-7c47624df98f/go.mod h1:EkVYQZoAsY45+roY golang.org/x/tools v0.0.0-20200729194436-6467de6f59a7/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA= golang.org/x/tools v0.0.0-20200804011535-6c149bb5ef0d/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA= golang.org/x/tools v0.0.0-20200825202427-b303f430e36d/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA= +golang.org/x/tools v0.0.0-20200904185747-39188db58858/go.mod h1:Cj7w3i3Rnn0Xh82ur9kSqwfTHTeVxaDqrfMjpcNT6bE= +golang.org/x/tools v0.0.0-20201110124207-079ba7bd75cd/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.0.0-20201124115921-2c860bdd6e78/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= +golang.org/x/tools v0.0.0-20201201161351-ac6f37ff4c2a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= +golang.org/x/tools v0.0.0-20201208233053-a543418bbed2/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= +golang.org/x/tools v0.0.0-20210105154028-b0ab187a4818/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.0.0-20210106214847-113979e3529a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= +golang.org/x/tools v0.0.0-20210108195828-e2f9c7f1fc8e/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.1.0/go.mod h1:xkSsbof2nBLbhDlRMhhhyNLN/zl3eTqcnHD5viDpcZ0= golang.org/x/tools v0.1.1-0.20210205202024-ef80cdb6ec6d/go.mod h1:9bzcO0MWcOuT0tm1iBGzDVPshzfwoVvREIui8C+MHqU= golang.org/x/tools v0.1.1-0.20210302220138-2ac05c832e1a/go.mod h1:9bzcO0MWcOuT0tm1iBGzDVPshzfwoVvREIui8C+MHqU= golang.org/x/tools v0.1.1/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= +golang.org/x/tools v0.1.2/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= +golang.org/x/tools v0.1.3/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= +golang.org/x/tools v0.1.4/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= golang.org/x/tools v0.1.5/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= golang.org/x/tools v0.1.9/go.mod h1:nABZi5QlRsZVlzPpHl034qft6wpY4eDcsTt5AaioBiU= golang.org/x/tools v0.1.10/go.mod h1:Uh6Zz+xoGYZom868N8YTex3t7RhtHDBrE8Gzo9bV56E= golang.org/x/tools v0.1.12/go.mod h1:hNGJHUnrk76NpqgfD5Aqm5Crs+Hm0VOH/i9J2+nxYbc= +golang.org/x/tools v0.3.0/go.mod h1:/rWhSS2+zyEVwoJf8YAX6L2f0ntZ7Kn/mGgAWcipA5k= golang.org/x/tools v0.6.0/go.mod h1:Xwgl3UAJ/d3gWutnCtw505GrjyAbvKui8lOU390QaIU= +golang.org/x/tools v0.7.0/go.mod h1:4pg6aUX35JBAogB10C9AtvVL+qowtN4pT3CGSQex14s= golang.org/x/tools v0.8.0/go.mod h1:JxBZ99ISMI5ViVkT1tr6tdNmXeTrcpVSD3vZ1RsRdN4= +golang.org/x/tools v0.9.1/go.mod h1:owI94Op576fPu3cIGQeHs3joujW/2Oc6MtlxbF5dfNc= +golang.org/x/tools v0.10.0/go.mod h1:UJwyiVBsOA2uwvK/e5OY3GTpDUJriEd+/YlqAwLPmyM= golang.org/x/tools v0.13.0/go.mod h1:HvlwmtVNQAhOuCjW7xxvovg8wbNq7LwfXh/k7wXUl58= golang.org/x/tools v0.14.0/go.mod h1:uYBEerGOWcJyEORxN+Ek8+TT266gXkNlHdJBwexUsBg= golang.org/x/tools v0.27.0 h1:qEKojBykQkQ4EynWy4S8Weg69NumxKdn40Fce3uc/8o= @@ -1268,8 +2273,10 @@ golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8T golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +golang.org/x/xerrors v0.0.0-20220411194840-2f41105eb62f/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20220517211312-f3a8303e98df/go.mod h1:K8+ghG5WaK9qNqU5K3HdILfMLy1f3aNYFI/wnl100a8= golang.org/x/xerrors v0.0.0-20220609144429-65e65417b02f/go.mod h1:K8+ghG5WaK9qNqU5K3HdILfMLy1f3aNYFI/wnl100a8= +golang.org/x/xerrors v0.0.0-20220907171357-04be3eba64a2/go.mod h1:K8+ghG5WaK9qNqU5K3HdILfMLy1f3aNYFI/wnl100a8= golang.org/x/xerrors v0.0.0-20231012003039-104605ab7028 h1:+cNy6SZtPcJQH3LJVLOSmiC7MMxXNOb3PU/VUEz+EhU= golang.org/x/xerrors v0.0.0-20231012003039-104605ab7028/go.mod h1:NDW/Ps6MPRej6fsCIbMTohpP40sJ/P/vI1MoTEGwX90= gonum.org/v1/gonum v0.0.0-20180816165407-929014505bf4/go.mod h1:Y+Yx5eoAFn32cQvJDxZx5Dpnq+c3wtXuadVZAcxbbBo= @@ -1299,6 +2306,54 @@ google.golang.org/api v0.24.0/go.mod h1:lIXQywCXRcnZPGlsd8NbLnOjtAoL6em04bJ9+z0M google.golang.org/api v0.28.0/go.mod h1:lIXQywCXRcnZPGlsd8NbLnOjtAoL6em04bJ9+z0MncE= google.golang.org/api v0.29.0/go.mod h1:Lcubydp8VUV7KeIHD9z2Bys/sm/vGKnG1UHuDBSrHWM= google.golang.org/api v0.30.0/go.mod h1:QGmEvQ87FHZNiUVJkT14jQNYJ4ZJjdRF23ZXz5138Fc= +google.golang.org/api v0.35.0/go.mod h1:/XrVsuzM0rZmrsbjJutiuftIzeuTQcEeaYcSk/mQ1dg= +google.golang.org/api v0.36.0/go.mod h1:+z5ficQTmoYpPn8LCUNVpK5I7hwkpjbcgqA7I34qYtE= +google.golang.org/api v0.40.0/go.mod h1:fYKFpnQN0DsDSKRVRcQSDQNtqWPfM9i+zNPxepjRCQ8= +google.golang.org/api v0.41.0/go.mod h1:RkxM5lITDfTzmyKFPt+wGrCJbVfniCr2ool8kTBzRTU= +google.golang.org/api v0.43.0/go.mod h1:nQsDGjRXMo4lvh5hP0TKqF244gqhGcr/YSIykhUk/94= +google.golang.org/api v0.47.0/go.mod h1:Wbvgpq1HddcWVtzsVLyfLp8lDg6AA241LmgIL59tHXo= +google.golang.org/api v0.48.0/go.mod h1:71Pr1vy+TAZRPkPs/xlCf5SsU8WjuAWv1Pfjbtukyy4= +google.golang.org/api v0.50.0/go.mod h1:4bNT5pAuq5ji4SRZm+5QIkjny9JAyVD/3gaSihNefaw= +google.golang.org/api v0.51.0/go.mod h1:t4HdrdoNgyN5cbEfm7Lum0lcLDLiise1F8qDKX00sOU= +google.golang.org/api v0.54.0/go.mod h1:7C4bFFOvVDGXjfDTAsgGwDgAxRDeQ4X8NvUedIt6z3k= +google.golang.org/api v0.55.0/go.mod h1:38yMfeP1kfjsl8isn0tliTjIb1rJXcQi4UXlbqivdVE= +google.golang.org/api v0.56.0/go.mod h1:38yMfeP1kfjsl8isn0tliTjIb1rJXcQi4UXlbqivdVE= +google.golang.org/api v0.57.0/go.mod h1:dVPlbZyBo2/OjBpmvNdpn2GRm6rPy75jyU7bmhdrMgI= +google.golang.org/api v0.61.0/go.mod h1:xQRti5UdCmoCEqFxcz93fTl338AVqDgyaDRuOZ3hg9I= +google.golang.org/api v0.63.0/go.mod h1:gs4ij2ffTRXwuzzgJl/56BdwJaA194ijkfn++9tDuPo= +google.golang.org/api v0.67.0/go.mod h1:ShHKP8E60yPsKNw/w8w+VYaj9H6buA5UqDp8dhbQZ6g= +google.golang.org/api v0.70.0/go.mod h1:Bs4ZM2HGifEvXwd50TtW70ovgJffJYw2oRCOFU/SkfA= +google.golang.org/api v0.71.0/go.mod h1:4PyU6e6JogV1f9eA4voyrTY2batOLdgZ5qZ5HOCc4j8= +google.golang.org/api v0.74.0/go.mod h1:ZpfMZOVRMywNyvJFeqL9HRWBgAuRfSjJFpe9QtRRyDs= +google.golang.org/api v0.75.0/go.mod h1:pU9QmyHLnzlpar1Mjt4IbapUCy8J+6HD6GeELN69ljA= +google.golang.org/api v0.77.0/go.mod h1:pU9QmyHLnzlpar1Mjt4IbapUCy8J+6HD6GeELN69ljA= +google.golang.org/api v0.78.0/go.mod h1:1Sg78yoMLOhlQTeF+ARBoytAcH1NNyyl390YMy6rKmw= +google.golang.org/api v0.80.0/go.mod h1:xY3nI94gbvBrE0J6NHXhxOmW97HG7Khjkku6AFB3Hyg= +google.golang.org/api v0.84.0/go.mod h1:NTsGnUFJMYROtiquksZHBWtHfeMC7iYthki7Eq3pa8o= +google.golang.org/api v0.85.0/go.mod h1:AqZf8Ep9uZ2pyTvgL+x0D3Zt0eoT9b5E8fmzfu6FO2g= +google.golang.org/api v0.90.0/go.mod h1:+Sem1dnrKlrXMR/X0bPnMWyluQe4RsNoYfmNLhOIkzw= +google.golang.org/api v0.93.0/go.mod h1:+Sem1dnrKlrXMR/X0bPnMWyluQe4RsNoYfmNLhOIkzw= +google.golang.org/api v0.95.0/go.mod h1:eADj+UBuxkh5zlrSntJghuNeg8HwQ1w5lTKkuqaETEI= +google.golang.org/api v0.96.0/go.mod h1:w7wJQLTM+wvQpNf5JyEcBoxK0RH7EDrh/L4qfsuJ13s= +google.golang.org/api v0.97.0/go.mod h1:w7wJQLTM+wvQpNf5JyEcBoxK0RH7EDrh/L4qfsuJ13s= +google.golang.org/api v0.98.0/go.mod h1:w7wJQLTM+wvQpNf5JyEcBoxK0RH7EDrh/L4qfsuJ13s= +google.golang.org/api v0.99.0/go.mod h1:1YOf74vkVndF7pG6hIHuINsM7eWwpVTAfNMNiL91A08= +google.golang.org/api v0.100.0/go.mod h1:ZE3Z2+ZOr87Rx7dqFsdRQkRBk36kDtp/h+QpHbB7a70= +google.golang.org/api v0.102.0/go.mod h1:3VFl6/fzoA+qNuS1N1/VfXY4LjoXN/wzeIp7TweWwGo= +google.golang.org/api v0.103.0/go.mod h1:hGtW6nK1AC+d9si/UBhw8Xli+QMOf6xyNAyJw4qU9w0= +google.golang.org/api v0.106.0/go.mod h1:2Ts0XTHNVWxypznxWOYUeI4g3WdP9Pk2Qk58+a/O9MY= +google.golang.org/api v0.107.0/go.mod h1:2Ts0XTHNVWxypznxWOYUeI4g3WdP9Pk2Qk58+a/O9MY= +google.golang.org/api v0.108.0/go.mod h1:2Ts0XTHNVWxypznxWOYUeI4g3WdP9Pk2Qk58+a/O9MY= +google.golang.org/api v0.110.0/go.mod h1:7FC4Vvx1Mooxh8C5HWjzZHcavuS2f6pmJpZx60ca7iI= +google.golang.org/api v0.111.0/go.mod h1:qtFHvU9mhgTJegR31csQ+rwxyUTHOKFqCKWp1J0fdw0= +google.golang.org/api v0.114.0/go.mod h1:ifYI2ZsFK6/uGddGfAD5BMxlnkBqCmqHSDUVi45N5Yg= +google.golang.org/api v0.118.0/go.mod h1:76TtD3vkgmZ66zZzp72bUUklpmQmKlhh6sYtIjYK+5E= +google.golang.org/api v0.122.0/go.mod h1:gcitW0lvnyWjSp9nKxAbdHKIZ6vF4aajGueeslZOyms= +google.golang.org/api v0.124.0/go.mod h1:xu2HQurE5gi/3t1aFCvhPD781p0a3p11sdunTJ2BlP4= +google.golang.org/api v0.125.0/go.mod h1:mBwVAtz+87bEN6CbA1GtZPDOqY2R5ONPqJeIlvyo4Aw= +google.golang.org/api v0.126.0/go.mod h1:mBwVAtz+87bEN6CbA1GtZPDOqY2R5ONPqJeIlvyo4Aw= +google.golang.org/api v0.128.0/go.mod h1:Y611qgqaE92On/7g65MQgxYul3c0rEB894kniWLY750= +google.golang.org/api v0.139.0/go.mod h1:CVagp6Eekz9CjGZ718Z+sloknzkDJE7Vc1Ckj9+viBk= google.golang.org/api v0.169.0 h1:QwWPy71FgMWqJN/l6jVlFHUa29a7dcUy02I8o799nPY= google.golang.org/api v0.169.0/go.mod h1:gpNOiMA2tZ4mf5R9Iwf4rK/Dcz0fbdIgWYWVoxmsyLg= google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= @@ -1309,7 +2364,6 @@ google.golang.org/appengine v1.6.5/go.mod h1:8WjMMxjGQR8xUklV/ARdw2HLXBOI7O7uCID google.golang.org/appengine v1.6.6/go.mod h1:8WjMMxjGQR8xUklV/ARdw2HLXBOI7O7uCIDZVag1xfc= google.golang.org/appengine v1.6.7/go.mod h1:8WjMMxjGQR8xUklV/ARdw2HLXBOI7O7uCIDZVag1xfc= google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= -google.golang.org/genproto v0.0.0-20181004005441-af9cb2a35e7f/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20190307195333-5fe7a883aa19/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= google.golang.org/genproto v0.0.0-20190418145605-e7d98fc518a7/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= google.golang.org/genproto v0.0.0-20190425155659-357c62f0e4bb/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= @@ -1340,13 +2394,157 @@ google.golang.org/genproto v0.0.0-20200618031413-b414f8b61790/go.mod h1:jDfRM7Fc google.golang.org/genproto v0.0.0-20200729003335-053ba62fc06f/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= google.golang.org/genproto v0.0.0-20200804131852-c06518451d9c/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= google.golang.org/genproto v0.0.0-20200825200019-8632dd797987/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= +google.golang.org/genproto v0.0.0-20200904004341-0bd0a958aa1d/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= +google.golang.org/genproto v0.0.0-20201109203340-2640f1f9cdfb/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= +google.golang.org/genproto v0.0.0-20201201144952-b05cb90ed32e/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= +google.golang.org/genproto v0.0.0-20201210142538-e3217bee35cc/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= +google.golang.org/genproto v0.0.0-20201214200347-8c77b98c765d/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= +google.golang.org/genproto v0.0.0-20210108203827-ffc7fda8c3d7/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= +google.golang.org/genproto v0.0.0-20210222152913-aa3ee6e6a81c/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= +google.golang.org/genproto v0.0.0-20210226172003-ab064af71705/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= +google.golang.org/genproto v0.0.0-20210303154014-9728d6b83eeb/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= +google.golang.org/genproto v0.0.0-20210310155132-4ce2db91004e/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= +google.golang.org/genproto v0.0.0-20210319143718-93e7006c17a6/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= +google.golang.org/genproto v0.0.0-20210329143202-679c6ae281ee/go.mod h1:9lPAdzaEmUacj36I+k7YKbEc5CXzPIeORRgDAUOu28A= +google.golang.org/genproto v0.0.0-20210402141018-6c239bbf2bb1/go.mod h1:9lPAdzaEmUacj36I+k7YKbEc5CXzPIeORRgDAUOu28A= +google.golang.org/genproto v0.0.0-20210513213006-bf773b8c8384/go.mod h1:P3QM42oQyzQSnHPnZ/vqoCdDmzH28fzWByN9asMeM8A= +google.golang.org/genproto v0.0.0-20210602131652-f16073e35f0c/go.mod h1:UODoCrxHCcBojKKwX1terBiRUaqAsFqJiF615XL43r0= +google.golang.org/genproto v0.0.0-20210604141403-392c879c8b08/go.mod h1:UODoCrxHCcBojKKwX1terBiRUaqAsFqJiF615XL43r0= +google.golang.org/genproto v0.0.0-20210608205507-b6d2f5bf0d7d/go.mod h1:UODoCrxHCcBojKKwX1terBiRUaqAsFqJiF615XL43r0= +google.golang.org/genproto v0.0.0-20210624195500-8bfb893ecb84/go.mod h1:SzzZ/N+nwJDaO1kznhnlzqS8ocJICar6hYhVyhi++24= +google.golang.org/genproto v0.0.0-20210713002101-d411969a0d9a/go.mod h1:AxrInvYm1dci+enl5hChSFPOmmUF1+uAa/UsgNRWd7k= +google.golang.org/genproto v0.0.0-20210716133855-ce7ef5c701ea/go.mod h1:AxrInvYm1dci+enl5hChSFPOmmUF1+uAa/UsgNRWd7k= +google.golang.org/genproto v0.0.0-20210728212813-7823e685a01f/go.mod h1:ob2IJxKrgPT52GcgX759i1sleT07tiKowYBGbczaW48= +google.golang.org/genproto v0.0.0-20210805201207-89edb61ffb67/go.mod h1:ob2IJxKrgPT52GcgX759i1sleT07tiKowYBGbczaW48= +google.golang.org/genproto v0.0.0-20210813162853-db860fec028c/go.mod h1:cFeNkxwySK631ADgubI+/XFU/xp8FD5KIVV4rj8UC5w= +google.golang.org/genproto v0.0.0-20210821163610-241b8fcbd6c8/go.mod h1:eFjDcFEctNawg4eG61bRv87N7iHBWyVhJu7u1kqDUXY= +google.golang.org/genproto v0.0.0-20210828152312-66f60bf46e71/go.mod h1:eFjDcFEctNawg4eG61bRv87N7iHBWyVhJu7u1kqDUXY= +google.golang.org/genproto v0.0.0-20210831024726-fe130286e0e2/go.mod h1:eFjDcFEctNawg4eG61bRv87N7iHBWyVhJu7u1kqDUXY= +google.golang.org/genproto v0.0.0-20210903162649-d08c68adba83/go.mod h1:eFjDcFEctNawg4eG61bRv87N7iHBWyVhJu7u1kqDUXY= +google.golang.org/genproto v0.0.0-20210909211513-a8c4777a87af/go.mod h1:eFjDcFEctNawg4eG61bRv87N7iHBWyVhJu7u1kqDUXY= +google.golang.org/genproto v0.0.0-20210924002016-3dee208752a0/go.mod h1:5CzLGKJ67TSI2B9POpiiyGha0AjJvZIUgRMt1dSmuhc= +google.golang.org/genproto v0.0.0-20211118181313-81c1377c94b1/go.mod h1:5CzLGKJ67TSI2B9POpiiyGha0AjJvZIUgRMt1dSmuhc= +google.golang.org/genproto v0.0.0-20211206160659-862468c7d6e0/go.mod h1:5CzLGKJ67TSI2B9POpiiyGha0AjJvZIUgRMt1dSmuhc= +google.golang.org/genproto v0.0.0-20211208223120-3a66f561d7aa/go.mod h1:5CzLGKJ67TSI2B9POpiiyGha0AjJvZIUgRMt1dSmuhc= +google.golang.org/genproto v0.0.0-20211221195035-429b39de9b1c/go.mod h1:5CzLGKJ67TSI2B9POpiiyGha0AjJvZIUgRMt1dSmuhc= +google.golang.org/genproto v0.0.0-20220126215142-9970aeb2e350/go.mod h1:5CzLGKJ67TSI2B9POpiiyGha0AjJvZIUgRMt1dSmuhc= +google.golang.org/genproto v0.0.0-20220207164111-0872dc986b00/go.mod h1:5CzLGKJ67TSI2B9POpiiyGha0AjJvZIUgRMt1dSmuhc= +google.golang.org/genproto v0.0.0-20220218161850-94dd64e39d7c/go.mod h1:kGP+zUP2Ddo0ayMi4YuN7C3WZyJvGLZRh8Z5wnAqvEI= +google.golang.org/genproto v0.0.0-20220222213610-43724f9ea8cf/go.mod h1:kGP+zUP2Ddo0ayMi4YuN7C3WZyJvGLZRh8Z5wnAqvEI= +google.golang.org/genproto v0.0.0-20220304144024-325a89244dc8/go.mod h1:kGP+zUP2Ddo0ayMi4YuN7C3WZyJvGLZRh8Z5wnAqvEI= +google.golang.org/genproto v0.0.0-20220310185008-1973136f34c6/go.mod h1:kGP+zUP2Ddo0ayMi4YuN7C3WZyJvGLZRh8Z5wnAqvEI= +google.golang.org/genproto v0.0.0-20220324131243-acbaeb5b85eb/go.mod h1:hAL49I2IFola2sVEjAn7MEwsja0xp51I0tlGAf9hz4E= +google.golang.org/genproto v0.0.0-20220329172620-7be39ac1afc7/go.mod h1:8w6bsBMX6yCPbAVTeqQHvzxW0EIFigd5lZyahWgyfDo= +google.golang.org/genproto v0.0.0-20220407144326-9054f6ed7bac/go.mod h1:8w6bsBMX6yCPbAVTeqQHvzxW0EIFigd5lZyahWgyfDo= +google.golang.org/genproto v0.0.0-20220413183235-5e96e2839df9/go.mod h1:8w6bsBMX6yCPbAVTeqQHvzxW0EIFigd5lZyahWgyfDo= +google.golang.org/genproto v0.0.0-20220414192740-2d67ff6cf2b4/go.mod h1:8w6bsBMX6yCPbAVTeqQHvzxW0EIFigd5lZyahWgyfDo= +google.golang.org/genproto v0.0.0-20220421151946-72621c1f0bd3/go.mod h1:8w6bsBMX6yCPbAVTeqQHvzxW0EIFigd5lZyahWgyfDo= +google.golang.org/genproto v0.0.0-20220429170224-98d788798c3e/go.mod h1:8w6bsBMX6yCPbAVTeqQHvzxW0EIFigd5lZyahWgyfDo= +google.golang.org/genproto v0.0.0-20220502173005-c8bf987b8c21/go.mod h1:RAyBrSAP7Fh3Nc84ghnVLDPuV51xc9agzmm4Ph6i0Q4= +google.golang.org/genproto v0.0.0-20220505152158-f39f71e6c8f3/go.mod h1:RAyBrSAP7Fh3Nc84ghnVLDPuV51xc9agzmm4Ph6i0Q4= +google.golang.org/genproto v0.0.0-20220518221133-4f43b3371335/go.mod h1:RAyBrSAP7Fh3Nc84ghnVLDPuV51xc9agzmm4Ph6i0Q4= +google.golang.org/genproto v0.0.0-20220523171625-347a074981d8/go.mod h1:RAyBrSAP7Fh3Nc84ghnVLDPuV51xc9agzmm4Ph6i0Q4= +google.golang.org/genproto v0.0.0-20220608133413-ed9918b62aac/go.mod h1:KEWEmljWE5zPzLBa/oHl6DaEt9LmfH6WtH1OHIvleBA= +google.golang.org/genproto v0.0.0-20220616135557-88e70c0c3a90/go.mod h1:KEWEmljWE5zPzLBa/oHl6DaEt9LmfH6WtH1OHIvleBA= +google.golang.org/genproto v0.0.0-20220617124728-180714bec0ad/go.mod h1:KEWEmljWE5zPzLBa/oHl6DaEt9LmfH6WtH1OHIvleBA= +google.golang.org/genproto v0.0.0-20220624142145-8cd45d7dbd1f/go.mod h1:KEWEmljWE5zPzLBa/oHl6DaEt9LmfH6WtH1OHIvleBA= +google.golang.org/genproto v0.0.0-20220628213854-d9e0b6570c03/go.mod h1:KEWEmljWE5zPzLBa/oHl6DaEt9LmfH6WtH1OHIvleBA= +google.golang.org/genproto v0.0.0-20220722212130-b98a9ff5e252/go.mod h1:GkXuJDJ6aQ7lnJcRF+SJVgFdQhypqgl3LB1C9vabdRE= +google.golang.org/genproto v0.0.0-20220801145646-83ce21fca29f/go.mod h1:iHe1svFLAZg9VWz891+QbRMwUv9O/1Ww+/mngYeThbc= +google.golang.org/genproto v0.0.0-20220815135757-37a418bb8959/go.mod h1:dbqgFATTzChvnt+ujMdZwITVAJHFtfyN1qUhDqEiIlk= +google.golang.org/genproto v0.0.0-20220817144833-d7fd3f11b9b1/go.mod h1:dbqgFATTzChvnt+ujMdZwITVAJHFtfyN1qUhDqEiIlk= +google.golang.org/genproto v0.0.0-20220822174746-9e6da59bd2fc/go.mod h1:dbqgFATTzChvnt+ujMdZwITVAJHFtfyN1qUhDqEiIlk= +google.golang.org/genproto v0.0.0-20220829144015-23454907ede3/go.mod h1:dbqgFATTzChvnt+ujMdZwITVAJHFtfyN1qUhDqEiIlk= +google.golang.org/genproto v0.0.0-20220829175752-36a9c930ecbf/go.mod h1:dbqgFATTzChvnt+ujMdZwITVAJHFtfyN1qUhDqEiIlk= +google.golang.org/genproto v0.0.0-20220913154956-18f8339a66a5/go.mod h1:0Nb8Qy+Sk5eDzHnzlStwW3itdNaWoZA5XeSG+R3JHSo= +google.golang.org/genproto v0.0.0-20220914142337-ca0e39ece12f/go.mod h1:0Nb8Qy+Sk5eDzHnzlStwW3itdNaWoZA5XeSG+R3JHSo= +google.golang.org/genproto v0.0.0-20220915135415-7fd63a7952de/go.mod h1:0Nb8Qy+Sk5eDzHnzlStwW3itdNaWoZA5XeSG+R3JHSo= +google.golang.org/genproto v0.0.0-20220916172020-2692e8806bfa/go.mod h1:0Nb8Qy+Sk5eDzHnzlStwW3itdNaWoZA5XeSG+R3JHSo= +google.golang.org/genproto v0.0.0-20220919141832-68c03719ef51/go.mod h1:0Nb8Qy+Sk5eDzHnzlStwW3itdNaWoZA5XeSG+R3JHSo= +google.golang.org/genproto v0.0.0-20220920201722-2b89144ce006/go.mod h1:ht8XFiar2npT/g4vkk7O0WYS1sHOHbdujxbEp7CJWbw= +google.golang.org/genproto v0.0.0-20220926165614-551eb538f295/go.mod h1:woMGP53BroOrRY3xTxlbr8Y3eB/nzAvvFM83q7kG2OI= +google.golang.org/genproto v0.0.0-20220926220553-6981cbe3cfce/go.mod h1:woMGP53BroOrRY3xTxlbr8Y3eB/nzAvvFM83q7kG2OI= +google.golang.org/genproto v0.0.0-20221010155953-15ba04fc1c0e/go.mod h1:3526vdqwhZAwq4wsRUaVG555sVgsNmIjRtO7t/JH29U= +google.golang.org/genproto v0.0.0-20221014173430-6e2ab493f96b/go.mod h1:1vXfmgAz9N9Jx0QA82PqRVauvCz1SGSz739p0f183jM= +google.golang.org/genproto v0.0.0-20221014213838-99cd37c6964a/go.mod h1:1vXfmgAz9N9Jx0QA82PqRVauvCz1SGSz739p0f183jM= +google.golang.org/genproto v0.0.0-20221024153911-1573dae28c9c/go.mod h1:9qHF0xnpdSfF6knlcsnpzUu5y+rpwgbvsyGAZPBMg4s= +google.golang.org/genproto v0.0.0-20221024183307-1bc688fe9f3e/go.mod h1:9qHF0xnpdSfF6knlcsnpzUu5y+rpwgbvsyGAZPBMg4s= +google.golang.org/genproto v0.0.0-20221027153422-115e99e71e1c/go.mod h1:CGI5F/G+E5bKwmfYo09AXuVN4dD894kIKUFmVbP2/Fo= +google.golang.org/genproto v0.0.0-20221109142239-94d6d90a7d66/go.mod h1:rZS5c/ZVYMaOGBfO68GWtjOw/eLaZM1X6iVtgjZ+EWg= +google.golang.org/genproto v0.0.0-20221114212237-e4508ebdbee1/go.mod h1:rZS5c/ZVYMaOGBfO68GWtjOw/eLaZM1X6iVtgjZ+EWg= +google.golang.org/genproto v0.0.0-20221117204609-8f9c96812029/go.mod h1:rZS5c/ZVYMaOGBfO68GWtjOw/eLaZM1X6iVtgjZ+EWg= +google.golang.org/genproto v0.0.0-20221118155620-16455021b5e6/go.mod h1:rZS5c/ZVYMaOGBfO68GWtjOw/eLaZM1X6iVtgjZ+EWg= +google.golang.org/genproto v0.0.0-20221201164419-0e50fba7f41c/go.mod h1:rZS5c/ZVYMaOGBfO68GWtjOw/eLaZM1X6iVtgjZ+EWg= +google.golang.org/genproto v0.0.0-20221201204527-e3fa12d562f3/go.mod h1:rZS5c/ZVYMaOGBfO68GWtjOw/eLaZM1X6iVtgjZ+EWg= +google.golang.org/genproto v0.0.0-20221202195650-67e5cbc046fd/go.mod h1:cTsE614GARnxrLsqKREzmNYJACSWWpAWdNMwnD7c2BE= +google.golang.org/genproto v0.0.0-20221227171554-f9683d7f8bef/go.mod h1:RGgjbofJ8xD9Sq1VVhDM1Vok1vRONV+rg+CjzG4SZKM= +google.golang.org/genproto v0.0.0-20230110181048-76db0878b65f/go.mod h1:RGgjbofJ8xD9Sq1VVhDM1Vok1vRONV+rg+CjzG4SZKM= +google.golang.org/genproto v0.0.0-20230112194545-e10362b5ecf9/go.mod h1:RGgjbofJ8xD9Sq1VVhDM1Vok1vRONV+rg+CjzG4SZKM= +google.golang.org/genproto v0.0.0-20230113154510-dbe35b8444a5/go.mod h1:RGgjbofJ8xD9Sq1VVhDM1Vok1vRONV+rg+CjzG4SZKM= +google.golang.org/genproto v0.0.0-20230123190316-2c411cf9d197/go.mod h1:RGgjbofJ8xD9Sq1VVhDM1Vok1vRONV+rg+CjzG4SZKM= +google.golang.org/genproto v0.0.0-20230124163310-31e0e69b6fc2/go.mod h1:RGgjbofJ8xD9Sq1VVhDM1Vok1vRONV+rg+CjzG4SZKM= +google.golang.org/genproto v0.0.0-20230125152338-dcaf20b6aeaa/go.mod h1:RGgjbofJ8xD9Sq1VVhDM1Vok1vRONV+rg+CjzG4SZKM= +google.golang.org/genproto v0.0.0-20230127162408-596548ed4efa/go.mod h1:RGgjbofJ8xD9Sq1VVhDM1Vok1vRONV+rg+CjzG4SZKM= +google.golang.org/genproto v0.0.0-20230209215440-0dfe4f8abfcc/go.mod h1:RGgjbofJ8xD9Sq1VVhDM1Vok1vRONV+rg+CjzG4SZKM= +google.golang.org/genproto v0.0.0-20230216225411-c8e22ba71e44/go.mod h1:8B0gmkoRebU8ukX6HP+4wrVQUY1+6PkQ44BSyIlflHA= +google.golang.org/genproto v0.0.0-20230222225845-10f96fb3dbec/go.mod h1:3Dl5ZL0q0isWJt+FVcfpQyirqemEuLAK/iFvg1UP1Hw= +google.golang.org/genproto v0.0.0-20230223222841-637eb2293923/go.mod h1:3Dl5ZL0q0isWJt+FVcfpQyirqemEuLAK/iFvg1UP1Hw= +google.golang.org/genproto v0.0.0-20230303212802-e74f57abe488/go.mod h1:TvhZT5f700eVlTNwND1xoEZQeWTB2RY/65kplwl/bFA= +google.golang.org/genproto v0.0.0-20230306155012-7f2fa6fef1f4/go.mod h1:NWraEVixdDnqcqQ30jipen1STv2r/n24Wb7twVTGR4s= +google.golang.org/genproto v0.0.0-20230320184635-7606e756e683/go.mod h1:NWraEVixdDnqcqQ30jipen1STv2r/n24Wb7twVTGR4s= +google.golang.org/genproto v0.0.0-20230323212658-478b75c54725/go.mod h1:UUQDJDOlWu4KYeJZffbWgBkS1YFobzKbLVfK69pe0Ak= +google.golang.org/genproto v0.0.0-20230330154414-c0448cd141ea/go.mod h1:UUQDJDOlWu4KYeJZffbWgBkS1YFobzKbLVfK69pe0Ak= +google.golang.org/genproto v0.0.0-20230331144136-dcfb400f0633/go.mod h1:UUQDJDOlWu4KYeJZffbWgBkS1YFobzKbLVfK69pe0Ak= +google.golang.org/genproto v0.0.0-20230403163135-c38d8f061ccd/go.mod h1:UUQDJDOlWu4KYeJZffbWgBkS1YFobzKbLVfK69pe0Ak= +google.golang.org/genproto v0.0.0-20230410155749-daa745c078e1/go.mod h1:nKE/iIaLqn2bQwXBg8f1g2Ylh6r5MN5CmZvuzZCgsCU= +google.golang.org/genproto v0.0.0-20230525234025-438c736192d0/go.mod h1:9ExIQyXL5hZrHzQceCwuSYwZZ5QZBazOcprJ5rgs3lY= +google.golang.org/genproto v0.0.0-20230526161137-0005af68ea54/go.mod h1:zqTuNwFlFRsw5zIts5VnzLQxSRqh+CGOTVMlYbY0Eyk= +google.golang.org/genproto v0.0.0-20230526203410-71b5a4ffd15e/go.mod h1:zqTuNwFlFRsw5zIts5VnzLQxSRqh+CGOTVMlYbY0Eyk= +google.golang.org/genproto v0.0.0-20230530153820-e85fd2cbaebc/go.mod h1:xZnkP7mREFX5MORlOPEzLMr+90PPZQ2QWzrVTWfAq64= +google.golang.org/genproto v0.0.0-20230629202037-9506855d4529/go.mod h1:xZnkP7mREFX5MORlOPEzLMr+90PPZQ2QWzrVTWfAq64= +google.golang.org/genproto v0.0.0-20230706204954-ccb25ca9f130/go.mod h1:O9kGHb51iE/nOGvQaDUuadVYqovW56s5emA88lQnj6Y= +google.golang.org/genproto v0.0.0-20230711160842-782d3b101e98/go.mod h1:S7mY02OqCJTD0E1OiQy1F72PWFB4bZJ87cAtLPYgDR0= +google.golang.org/genproto v0.0.0-20230726155614-23370e0ffb3e/go.mod h1:0ggbjUrZYpy1q+ANUS30SEoGZ53cdfwtbuG7Ptgy108= +google.golang.org/genproto v0.0.0-20230803162519-f966b187b2e5/go.mod h1:oH/ZOT02u4kWEp7oYBGYFFkCdKS/uYR9Z7+0/xuuFp8= +google.golang.org/genproto v0.0.0-20230821184602-ccc8af3d0e93/go.mod h1:yZTlhN0tQnXo3h00fuXNCxJdLdIdnVFVBaRJ5LWBbw4= +google.golang.org/genproto v0.0.0-20230822172742-b8732ec3820d/go.mod h1:yZTlhN0tQnXo3h00fuXNCxJdLdIdnVFVBaRJ5LWBbw4= +google.golang.org/genproto v0.0.0-20230913181813-007df8e322eb/go.mod h1:yZTlhN0tQnXo3h00fuXNCxJdLdIdnVFVBaRJ5LWBbw4= +google.golang.org/genproto v0.0.0-20230920204549-e6e6cdab5c13/go.mod h1:CCviP9RmpZ1mxVr8MUjCnSiY09IbAXZxhLE6EhHIdPU= +google.golang.org/genproto v0.0.0-20231012201019-e917dd12ba7a/go.mod h1:EMfReVxb80Dq1hhioy0sOsY9jCE46YDgHlJ7fWVUWRE= google.golang.org/genproto v0.0.0-20240227224415-6ceb2ff114de h1:F6qOa9AZTYJXOUEr4jDysRDLrm4PHePlge4v4TGAlxY= google.golang.org/genproto v0.0.0-20240227224415-6ceb2ff114de/go.mod h1:VUhTRKeHn9wwcdrk73nvdC9gF178Tzhmt/qyaFcPLSo= +google.golang.org/genproto/googleapis/api v0.0.0-20230525234020-1aefcd67740a/go.mod h1:ts19tUU+Z0ZShN1y3aPyq2+O3d5FUNNgT6FtOzmrNn8= +google.golang.org/genproto/googleapis/api v0.0.0-20230525234035-dd9d682886f9/go.mod h1:vHYtlOoi6TsQ3Uk2yxR7NI5z8uoV+3pZtR4jmHIkRig= +google.golang.org/genproto/googleapis/api v0.0.0-20230526203410-71b5a4ffd15e/go.mod h1:vHYtlOoi6TsQ3Uk2yxR7NI5z8uoV+3pZtR4jmHIkRig= +google.golang.org/genproto/googleapis/api v0.0.0-20230530153820-e85fd2cbaebc/go.mod h1:vHYtlOoi6TsQ3Uk2yxR7NI5z8uoV+3pZtR4jmHIkRig= +google.golang.org/genproto/googleapis/api v0.0.0-20230629202037-9506855d4529/go.mod h1:vHYtlOoi6TsQ3Uk2yxR7NI5z8uoV+3pZtR4jmHIkRig= +google.golang.org/genproto/googleapis/api v0.0.0-20230706204954-ccb25ca9f130/go.mod h1:mPBs5jNgx2GuQGvFwUvVKqtn6HsUw9nP64BedgvqEsQ= +google.golang.org/genproto/googleapis/api v0.0.0-20230711160842-782d3b101e98/go.mod h1:rsr7RhLuwsDKL7RmgDDCUc6yaGr1iqceVb5Wv6f6YvQ= +google.golang.org/genproto/googleapis/api v0.0.0-20230726155614-23370e0ffb3e/go.mod h1:rsr7RhLuwsDKL7RmgDDCUc6yaGr1iqceVb5Wv6f6YvQ= +google.golang.org/genproto/googleapis/api v0.0.0-20230803162519-f966b187b2e5/go.mod h1:5DZzOUPCLYL3mNkQ0ms0F3EuUNZ7py1Bqeq6sxzI7/Q= +google.golang.org/genproto/googleapis/api v0.0.0-20230822172742-b8732ec3820d/go.mod h1:KjSP20unUpOx5kyQUFa7k4OJg0qeJ7DEZflGDu2p6Bk= +google.golang.org/genproto/googleapis/api v0.0.0-20230913181813-007df8e322eb/go.mod h1:KjSP20unUpOx5kyQUFa7k4OJg0qeJ7DEZflGDu2p6Bk= +google.golang.org/genproto/googleapis/api v0.0.0-20231002182017-d307bd883b97/go.mod h1:iargEX0SFPm3xcfMI0d1domjg0ZF4Aa0p2awqyxhvF0= google.golang.org/genproto/googleapis/api v0.0.0-20240318140521-94a12d6c2237 h1:RFiFrvy37/mpSpdySBDrUdipW/dHwsRwh3J3+A9VgT4= google.golang.org/genproto/googleapis/api v0.0.0-20240318140521-94a12d6c2237/go.mod h1:Z5Iiy3jtmioajWHDGFk7CeugTyHtPvMHA4UTmUkyalE= +google.golang.org/genproto/googleapis/bytestream v0.0.0-20230530153820-e85fd2cbaebc/go.mod h1:ylj+BE99M198VPbBh6A8d9n3w8fChvyLK3wwBOjXBFA= +google.golang.org/genproto/googleapis/bytestream v0.0.0-20230807174057-1744710a1577/go.mod h1:NjCQG/D8JandXxM57PZbAJL1DCNL6EypA0vPPwfsc7c= +google.golang.org/genproto/googleapis/rpc v0.0.0-20230525234015-3fc162c6f38a/go.mod h1:xURIpW9ES5+/GZhnV6beoEtxQrnkRGIfP5VQG2tCBLc= +google.golang.org/genproto/googleapis/rpc v0.0.0-20230525234030-28d5490b6b19/go.mod h1:66JfowdXAEgad5O9NnYcsNPLCPZJD++2L9X0PCMODrA= +google.golang.org/genproto/googleapis/rpc v0.0.0-20230526203410-71b5a4ffd15e/go.mod h1:66JfowdXAEgad5O9NnYcsNPLCPZJD++2L9X0PCMODrA= +google.golang.org/genproto/googleapis/rpc v0.0.0-20230530153820-e85fd2cbaebc/go.mod h1:66JfowdXAEgad5O9NnYcsNPLCPZJD++2L9X0PCMODrA= +google.golang.org/genproto/googleapis/rpc v0.0.0-20230629202037-9506855d4529/go.mod h1:66JfowdXAEgad5O9NnYcsNPLCPZJD++2L9X0PCMODrA= +google.golang.org/genproto/googleapis/rpc v0.0.0-20230706204954-ccb25ca9f130/go.mod h1:8mL13HKkDa+IuJ8yruA3ci0q+0vsUz4m//+ottjwS5o= +google.golang.org/genproto/googleapis/rpc v0.0.0-20230711160842-782d3b101e98/go.mod h1:TUfxEVdsvPg18p6AslUXFoLdpED4oBnGwyqk3dV1XzM= +google.golang.org/genproto/googleapis/rpc v0.0.0-20230731190214-cbb8c96f2d6d/go.mod h1:TUfxEVdsvPg18p6AslUXFoLdpED4oBnGwyqk3dV1XzM= +google.golang.org/genproto/googleapis/rpc v0.0.0-20230803162519-f966b187b2e5/go.mod h1:zBEcrKX2ZOcEkHWxBPAIvYUWOKKMIhYcmNiUIu2ji3I= +google.golang.org/genproto/googleapis/rpc v0.0.0-20230822172742-b8732ec3820d/go.mod h1:+Bk1OCOj40wS2hwAMA+aCW9ypzm63QTBBHp6lQ3p+9M= +google.golang.org/genproto/googleapis/rpc v0.0.0-20230920183334-c177e329c48b/go.mod h1:+Bk1OCOj40wS2hwAMA+aCW9ypzm63QTBBHp6lQ3p+9M= +google.golang.org/genproto/googleapis/rpc v0.0.0-20230920204549-e6e6cdab5c13/go.mod h1:KSqppvjFjtoCI+KGd4PELB0qLNxdJHRGqRI09mB6pQA= +google.golang.org/genproto/googleapis/rpc v0.0.0-20231002182017-d307bd883b97/go.mod h1:v7nGkzlmW8P3n/bKmWBn2WpBjpOEx8Q6gMueudAmKfY= +google.golang.org/genproto/googleapis/rpc v0.0.0-20231016165738-49dd2c1f3d0b/go.mod h1:swOH3j0KzcDDgGUWr+SNpyTen5YrXjS3eyPzFYKc6lc= google.golang.org/genproto/googleapis/rpc v0.0.0-20240515191416-fc5f0ca64291 h1:AgADTJarZTBqgjiUzRgfaBchgYB3/WFTC80GPwsMcRI= google.golang.org/genproto/googleapis/rpc v0.0.0-20240515191416-fc5f0ca64291/go.mod h1:EfXuqaE1J41VCDicxHzUDm+8rk+7ZdXzHV0IhO/I6s0= -google.golang.org/grpc v0.0.0-20180607172857-7a6a684ca69e/go.mod h1:yo6s7OP7yaDglbqo1J04qKzAhqBH6lvTonzMVmEdcZw= google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= google.golang.org/grpc v1.20.1/go.mod h1:10oTOabMzJvdu6/UiuZezV6QK5dSlG84ov/aaiqXj38= google.golang.org/grpc v1.21.1/go.mod h1:oYelfM1adQP15Ek0mdvEgi9Df8B9CZIaU1084ijfRaM= @@ -1360,12 +2558,44 @@ google.golang.org/grpc v1.28.0/go.mod h1:rpkK4SK4GF4Ach/+MFLZUBavHOvF2JJB5uozKKa google.golang.org/grpc v1.29.1/go.mod h1:itym6AZVZYACWQqET3MqgPpjcuV5QH3BxFS3IjizoKk= google.golang.org/grpc v1.30.0/go.mod h1:N36X2cJ7JwdamYAgDz+s+rVMFjt3numwzf/HckM8pak= google.golang.org/grpc v1.31.0/go.mod h1:N36X2cJ7JwdamYAgDz+s+rVMFjt3numwzf/HckM8pak= +google.golang.org/grpc v1.31.1/go.mod h1:N36X2cJ7JwdamYAgDz+s+rVMFjt3numwzf/HckM8pak= google.golang.org/grpc v1.33.1/go.mod h1:fr5YgcSWrqhRRxogOsw7RzIpsmvOZ6IcH4kBYTpR3n0= google.golang.org/grpc v1.33.2/go.mod h1:JMHMWHQWaTccqQQlmk3MJZS+GWXOdAesneDmEnv2fbc= +google.golang.org/grpc v1.34.0/go.mod h1:WotjhfgOW/POjDeRt8vscBtXq+2VjORFy659qA51WJ8= +google.golang.org/grpc v1.35.0/go.mod h1:qjiiYl8FncCW8feJPdyg3v6XW24KsRHe+dy9BAGRRjU= google.golang.org/grpc v1.36.0/go.mod h1:qjiiYl8FncCW8feJPdyg3v6XW24KsRHe+dy9BAGRRjU= +google.golang.org/grpc v1.36.1/go.mod h1:qjiiYl8FncCW8feJPdyg3v6XW24KsRHe+dy9BAGRRjU= +google.golang.org/grpc v1.37.0/go.mod h1:NREThFqKR1f3iQ6oBuvc5LadQuXVGo9rkm5ZGrQdJfM= +google.golang.org/grpc v1.37.1/go.mod h1:NREThFqKR1f3iQ6oBuvc5LadQuXVGo9rkm5ZGrQdJfM= +google.golang.org/grpc v1.38.0/go.mod h1:NREThFqKR1f3iQ6oBuvc5LadQuXVGo9rkm5ZGrQdJfM= +google.golang.org/grpc v1.39.0/go.mod h1:PImNr+rS9TWYb2O4/emRugxiyHZ5JyHW5F+RPnDzfrE= +google.golang.org/grpc v1.39.1/go.mod h1:PImNr+rS9TWYb2O4/emRugxiyHZ5JyHW5F+RPnDzfrE= +google.golang.org/grpc v1.40.0/go.mod h1:ogyxbiOoUXAkP+4+xa6PZSE9DZgIHtSpzjDTB9KAK34= +google.golang.org/grpc v1.40.1/go.mod h1:ogyxbiOoUXAkP+4+xa6PZSE9DZgIHtSpzjDTB9KAK34= +google.golang.org/grpc v1.42.0/go.mod h1:k+4IHHFw41K8+bbowsex27ge2rCb65oeWqe4jJ590SU= +google.golang.org/grpc v1.44.0/go.mod h1:k+4IHHFw41K8+bbowsex27ge2rCb65oeWqe4jJ590SU= +google.golang.org/grpc v1.45.0/go.mod h1:lN7owxKUQEqMfSyQikvvk5tf/6zMPsrK+ONuO11+0rQ= +google.golang.org/grpc v1.46.0/go.mod h1:vN9eftEi1UMyUsIF80+uQXhHjbXYbm0uXoFCACuMGWk= +google.golang.org/grpc v1.46.2/go.mod h1:vN9eftEi1UMyUsIF80+uQXhHjbXYbm0uXoFCACuMGWk= +google.golang.org/grpc v1.47.0/go.mod h1:vN9eftEi1UMyUsIF80+uQXhHjbXYbm0uXoFCACuMGWk= +google.golang.org/grpc v1.48.0/go.mod h1:vN9eftEi1UMyUsIF80+uQXhHjbXYbm0uXoFCACuMGWk= google.golang.org/grpc v1.49.0/go.mod h1:ZgQEeidpAuNRZ8iRrlBKXZQP1ghovWIVhdJRyCDK+GI= +google.golang.org/grpc v1.50.0/go.mod h1:ZgQEeidpAuNRZ8iRrlBKXZQP1ghovWIVhdJRyCDK+GI= +google.golang.org/grpc v1.50.1/go.mod h1:ZgQEeidpAuNRZ8iRrlBKXZQP1ghovWIVhdJRyCDK+GI= +google.golang.org/grpc v1.51.0/go.mod h1:wgNDFcnuBGmxLKI/qn4T+m5BtEBYXJPvibbUPsAIPww= +google.golang.org/grpc v1.52.0/go.mod h1:pu6fVzoFb+NBYNAvQL08ic+lvB2IojljRYuun5vorUY= +google.golang.org/grpc v1.52.3/go.mod h1:pu6fVzoFb+NBYNAvQL08ic+lvB2IojljRYuun5vorUY= +google.golang.org/grpc v1.53.0/go.mod h1:OnIrk0ipVdj4N5d9IUoFUx72/VlD7+jUsHwZgwSMQpw= +google.golang.org/grpc v1.54.0/go.mod h1:PUSEXI6iWghWaB6lXM4knEgpJNu2qUcKfDtNci3EC2g= +google.golang.org/grpc v1.55.0/go.mod h1:iYEXKGkEBhg1PjZQvoYEVPTDkHo1/bjTnfwTeGONTY8= +google.golang.org/grpc v1.56.1/go.mod h1:I9bI3vqKfayGqPUAwGdOSu7kt6oIJLixfffKrpXqQ9s= +google.golang.org/grpc v1.56.2/go.mod h1:I9bI3vqKfayGqPUAwGdOSu7kt6oIJLixfffKrpXqQ9s= +google.golang.org/grpc v1.57.0/go.mod h1:Sd+9RMTACXwmub0zcNY2c4arhtrbBYD1AUHI/dt16Mo= +google.golang.org/grpc v1.58.2/go.mod h1:tgX3ZQDlNJGU96V6yHh1T/JeoBQ2TXdr43YbYSsCJk0= +google.golang.org/grpc v1.59.0/go.mod h1:aUPDwccQo6OTjy7Hct4AfBPD1GptF4fyUjIkQ9YtF98= google.golang.org/grpc v1.63.2 h1:MUeiw1B2maTVZthpU5xvASfTh3LDbxHd6IJ6QQVU+xM= google.golang.org/grpc v1.63.2/go.mod h1:WAX/8DgncnokcFUldAxq7GeB5DXHDbMF+lLvDomNkRA= +google.golang.org/grpc/cmd/protoc-gen-go-grpc v1.1.0/go.mod h1:6Kw0yEErY5E/yWrBtf03jp27GLLJujG4z/JK95pnjjw= google.golang.org/grpc/examples v0.0.0-20231221225426-4f03f3ff32c9 h1:ATnmU8nL2NfIyTSiBvJVDIDIr3qBmeW+c7z7XU21eWs= google.golang.org/grpc/examples v0.0.0-20231221225426-4f03f3ff32c9/go.mod h1:j5uROIAAgi3YmtiETMt1LW0d/lHqQ7wwrIY4uGRXLQ4= google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8= @@ -1383,6 +2613,9 @@ google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQ google.golang.org/protobuf v1.27.1/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= google.golang.org/protobuf v1.28.0/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqwMG9pJV4I= google.golang.org/protobuf v1.28.1/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqwMG9pJV4I= +google.golang.org/protobuf v1.29.1/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqwMG9pJV4I= +google.golang.org/protobuf v1.30.0/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqwMG9pJV4I= +google.golang.org/protobuf v1.31.0/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqwMG9pJV4I= google.golang.org/protobuf v1.34.2 h1:6xV6lTsCfpGD21XK49h7MhtcApnLqkfYgPcdHftf6hg= google.golang.org/protobuf v1.34.2/go.mod h1:qYOHts0dSfpeUzUFpOMr/WGzszTmLH+DiWniOlNbLDw= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= @@ -1447,6 +2680,7 @@ lukechampine.com/uint128 v1.1.1/go.mod h1:c4eWIwlEGaxC/+H1VguhU4PHXNWDCDMUlWdIWl lukechampine.com/uint128 v1.2.0/go.mod h1:c4eWIwlEGaxC/+H1VguhU4PHXNWDCDMUlWdIWl2j1gk= modernc.org/cc/v3 v3.36.0/go.mod h1:NFUHyPn4ekoC/JHeZFfZurN6ixxawE1BnVonP/oahEI= modernc.org/cc/v3 v3.36.2/go.mod h1:NFUHyPn4ekoC/JHeZFfZurN6ixxawE1BnVonP/oahEI= +modernc.org/cc/v3 v3.36.3/go.mod h1:NFUHyPn4ekoC/JHeZFfZurN6ixxawE1BnVonP/oahEI= modernc.org/cc/v3 v3.37.0/go.mod h1:vtL+3mdHx/wcj3iEGz84rQa8vEqR6XM84v5Lcvfph20= modernc.org/cc/v3 v3.40.0/go.mod h1:/bTg4dnWkSXowUO6ssQKnOV0yMVxDYNIsIrzqTFDGH0= modernc.org/ccgo/v3 v3.0.0-20220428102840-41399a37e894/go.mod h1:eI31LL8EwEBKPpNpA4bU1/i+sKOwOrQy8D87zWUcRZc= @@ -1466,6 +2700,7 @@ modernc.org/libc v1.16.1/go.mod h1:JjJE0eu4yeK7tab2n4S1w8tlWd9MxXLRzheaRnAKymU= modernc.org/libc v1.16.17/go.mod h1:hYIV5VZczAmGZAnG15Vdngn5HSF5cSkbvfz2B7GRuVU= modernc.org/libc v1.16.19/go.mod h1:p7Mg4+koNjc8jkqwcoFBJx7tXkpj00G77X7A72jXPXA= modernc.org/libc v1.17.0/go.mod h1:XsgLldpP4aWlPlsjqKRdHPqCxCjISdHfM/yeWC5GyW0= +modernc.org/libc v1.17.1/go.mod h1:FZ23b+8LjxZs7XtFMbSzL/EhPxNbfZbErxEHc7cbD9s= modernc.org/libc v1.17.4/go.mod h1:WNg2ZH56rDEwdropAJeZPQkXmDwh+JCA1s/htl6r2fA= modernc.org/libc v1.18.0/go.mod h1:vj6zehR5bfc98ipowQOM2nIDUZnVew/wNC/2tOGS+q0= modernc.org/libc v1.20.3/go.mod h1:ZRfIaEkgrYgZDl6pa4W39HgN5G/yDW+NRmNKZBDFrk0= @@ -1476,14 +2711,17 @@ modernc.org/mathutil v1.4.1/go.mod h1:mZW8CKdRPY1v87qxC/wUdX5O1qDzXMP5TH3wjfpga6 modernc.org/mathutil v1.5.0/go.mod h1:mZW8CKdRPY1v87qxC/wUdX5O1qDzXMP5TH3wjfpga6E= modernc.org/memory v1.1.1/go.mod h1:/0wo5ibyrQiaoUoH7f9D8dnglAmILJ5/cxZlRECf+Nw= modernc.org/memory v1.2.0/go.mod h1:/0wo5ibyrQiaoUoH7f9D8dnglAmILJ5/cxZlRECf+Nw= +modernc.org/memory v1.2.1/go.mod h1:PkUhL0Mugw21sHPeskwZW4D6VscE/GQJOnIpCnW6pSU= modernc.org/memory v1.3.0/go.mod h1:PkUhL0Mugw21sHPeskwZW4D6VscE/GQJOnIpCnW6pSU= modernc.org/memory v1.4.0/go.mod h1:PkUhL0Mugw21sHPeskwZW4D6VscE/GQJOnIpCnW6pSU= modernc.org/memory v1.5.0/go.mod h1:PkUhL0Mugw21sHPeskwZW4D6VscE/GQJOnIpCnW6pSU= modernc.org/opt v0.1.1/go.mod h1:WdSiB5evDcignE70guQKxYUl14mgWtbClRi5wmkkTX0= modernc.org/opt v0.1.3/go.mod h1:WdSiB5evDcignE70guQKxYUl14mgWtbClRi5wmkkTX0= +modernc.org/sqlite v1.18.1/go.mod h1:6ho+Gow7oX5V+OiOQ6Tr4xeqbx13UZ6t+Fw9IRUG4d4= modernc.org/sqlite v1.18.2/go.mod h1:kvrTLEWgxUcHa2GfHBQtanR1H9ht3hTJNtKpzH9k1u0= modernc.org/strutil v1.1.1/go.mod h1:DE+MQQ/hjKBZS2zNInV5hhcipt5rLPWkmpbGeW5mmdw= modernc.org/strutil v1.1.3/go.mod h1:MEHNA7PdEnEwLvspRMtWTNnp2nnyvMfkimT1NKNAGbw= +modernc.org/tcl v1.13.1/go.mod h1:XOLfOwzhkljL4itZkK6T72ckMgvj0BDsnKNdZVUOecw= modernc.org/tcl v1.13.2/go.mod h1:7CLiGIPo1M8Rv1Mitpv5akc2+8fxUd2y2UzC/MfMzy0= modernc.org/token v1.0.0/go.mod h1:UGzOrNV1mAFSEB63lOFHIpNRUVMvYTc6yu1SMY/XTDM= modernc.org/token v1.0.1/go.mod h1:UGzOrNV1mAFSEB63lOFHIpNRUVMvYTc6yu1SMY/XTDM= diff --git a/pkg/util/util.go b/pkg/util/util.go index ded6218542d59..a29ca22a17e5d 100644 --- a/pkg/util/util.go +++ b/pkg/util/util.go @@ -30,6 +30,8 @@ import ( "github.com/pingcap/tidb/pkg/parser" "go.uber.org/atomic" "go.uber.org/zap" + "google.golang.org/protobuf/proto" + "google.golang.org/protobuf/protoadapt" ) // SliceToMap converts slice to map @@ -289,3 +291,8 @@ func GetRecoverError(r any) error { } return errors.Errorf("%v", r) } + +// ProtoV1Clone clones a V1 proto message. +func ProtoV1Clone[T protoadapt.MessageV1](p T) T { + return protoadapt.MessageV1Of(proto.Clone(protoadapt.MessageV2Of(p))).(T) +} diff --git a/pkg/util/util_test.go b/pkg/util/util_test.go index bb7a467a5e203..3955130549165 100644 --- a/pkg/util/util_test.go +++ b/pkg/util/util_test.go @@ -22,6 +22,7 @@ import ( "time" "github.com/pingcap/errors" + pb "github.com/pingcap/kvproto/pkg/brpb" "github.com/pingcap/tidb/pkg/sessionctx/stmtctx" "github.com/pingcap/tidb/pkg/util/memory" "github.com/stretchr/testify/assert" @@ -120,3 +121,18 @@ func TestIsInCorrectIdentifierName(t *testing.T) { require.Equalf(t, tc.correct, got, "IsInCorrectIdentifierName(%v) != %v", tc.name, tc.correct) } } + +func TestDupProto(t *testing.T) { + p := &pb.StorageBackend{ + Backend: &pb.StorageBackend_S3{ + S3: &pb.S3{ + Endpoint: "127.0.0.1", + }, + }, + } + + p2 := ProtoV1Clone(p) + require.Equal(t, p2.Backend.(*pb.StorageBackend_S3).S3.Endpoint, "127.0.0.1") + p2.Backend.(*pb.StorageBackend_S3).S3.Endpoint = "127.0.0.2" + require.Equal(t, p.Backend.(*pb.StorageBackend_S3).S3.Endpoint, "127.0.0.1") +} From f70831fcc03fd24873ff78d1d10237a3b6306302 Mon Sep 17 00:00:00 2001 From: hillium Date: Wed, 13 Nov 2024 18:08:05 +0800 Subject: [PATCH 02/36] added basic implementation Signed-off-by: hillium --- br/pkg/restore/log_client/client.go | 18 +++--- .../log_client/compacted_file_strategy.go | 31 +++++----- br/pkg/restore/log_client/log_file_manager.go | 23 ++++++- br/pkg/restore/log_client/migration.go | 11 ++++ br/pkg/restore/log_client/ssts.go | 60 +++++++++++++++++++ br/pkg/restore/snap_client/import.go | 2 +- br/pkg/stream/stream_metas.go | 3 + br/pkg/task/stream.go | 5 +- 8 files changed, 125 insertions(+), 28 deletions(-) create mode 100644 br/pkg/restore/log_client/ssts.go diff --git a/br/pkg/restore/log_client/client.go b/br/pkg/restore/log_client/client.go index 8c3621eed6d2e..b5f23ccede077 100644 --- a/br/pkg/restore/log_client/client.go +++ b/br/pkg/restore/log_client/client.go @@ -254,7 +254,7 @@ func (rc *LogClient) Close(ctx context.Context) { func (rc *LogClient) RestoreCompactedSstFiles( ctx context.Context, - compactionsIter iter.TryNextor[*backuppb.LogFileSubcompaction], + compactionsIter iter.TryNextor[SSTs], rules map[int64]*restoreutils.RewriteRules, importModeSwitcher *restore.ImportModeSwitcher, onProgress func(int64), @@ -268,14 +268,14 @@ func (rc *LogClient) RestoreCompactedSstFiles( return r.Err } i := r.Item - rewriteRules, ok := rules[i.Meta.TableId] + rewriteRules, ok := rules[i.TableID()] if !ok { - log.Warn("[Compacted SST Restore] Skipping excluded table during restore.", zap.Int64("table_id", i.Meta.TableId)) + log.Warn("[Compacted SST Restore] Skipping excluded table during restore.", zap.Int64("table_id", i.TableID())) continue } set := restore.BackupFileSet{ - TableID: i.Meta.TableId, - SSTFiles: i.SstOutputs, + TableID: i.TableID(), + SSTFiles: i.GetSSTs(), RewriteRules: rewriteRules, } backupFileSets = append(backupFileSets, set) @@ -421,7 +421,7 @@ func (rc *LogClient) InitClients( opt := snapclient.NewSnapFileImporterOptions( rc.cipher, metaClient, importCli, backend, - snapclient.RewriteModeKeyspace, stores, rc.concurrencyPerStore, createCallBacks, closeCallBacks, + snapclient.RewriteModeKeyspace, stores, 8, createCallBacks, closeCallBacks, ) snapFileImporter, err := snapclient.NewSnapFileImporter( ctx, rc.dom.Store().GetCodec().GetAPIVersion(), snapclient.TiDBCompcated, opt) @@ -1448,15 +1448,15 @@ func (rc *LogClient) UpdateSchemaVersion(ctx context.Context) error { // It uses a region splitter to handle the splitting logic based on the provided rules and checkpoint sets. func (rc *LogClient) WrapCompactedFilesIterWithSplitHelper( ctx context.Context, - compactedIter iter.TryNextor[*backuppb.LogFileSubcompaction], + compactedIter iter.TryNextor[SSTs], rules map[int64]*restoreutils.RewriteRules, checkpointSets map[string]struct{}, updateStatsFn func(uint64, uint64), splitSize uint64, splitKeys int64, -) (iter.TryNextor[*backuppb.LogFileSubcompaction], error) { +) (iter.TryNextor[SSTs], error) { client := split.NewClient(rc.pdClient, rc.pdHTTPClient, rc.tlsConf, maxSplitKeysOnce, 3) - wrapper := restore.PipelineRestorerWrapper[*backuppb.LogFileSubcompaction]{ + wrapper := restore.PipelineRestorerWrapper[SSTs]{ PipelineRegionsSplitter: split.NewPipelineRegionsSplitter(client, splitSize, splitKeys), } strategy := NewCompactedFileSplitStrategy(rules, checkpointSets, updateStatsFn) diff --git a/br/pkg/restore/log_client/compacted_file_strategy.go b/br/pkg/restore/log_client/compacted_file_strategy.go index 9637cf2e529b6..2db62323ee238 100644 --- a/br/pkg/restore/log_client/compacted_file_strategy.go +++ b/br/pkg/restore/log_client/compacted_file_strategy.go @@ -23,7 +23,7 @@ type CompactedFileSplitStrategy struct { checkpointFileProgressFn func(uint64, uint64) } -var _ split.SplitStrategy[*backuppb.LogFileSubcompaction] = &CompactedFileSplitStrategy{} +var _ split.SplitStrategy[SSTs] = &CompactedFileSplitStrategy{} func NewCompactedFileSplitStrategy( rules map[int64]*restoreutils.RewriteRules, @@ -37,14 +37,14 @@ func NewCompactedFileSplitStrategy( } } -func (cs *CompactedFileSplitStrategy) Accumulate(subCompaction *backuppb.LogFileSubcompaction) { - splitHelper, exist := cs.TableSplitter[subCompaction.Meta.TableId] +func (cs *CompactedFileSplitStrategy) Accumulate(ssts SSTs) { + splitHelper, exist := cs.TableSplitter[ssts.TableID()] if !exist { splitHelper = split.NewSplitHelper() - cs.TableSplitter[subCompaction.Meta.TableId] = splitHelper + cs.TableSplitter[ssts.TableID()] = splitHelper } - for _, f := range subCompaction.SstOutputs { + for _, f := range ssts.GetSSTs() { startKey := codec.EncodeBytes(nil, f.StartKey) endKey := codec.EncodeBytes(nil, f.EndKey) cs.AccumulateCount += 1 @@ -82,14 +82,14 @@ func (cs *CompactedFileSplitStrategy) ShouldSplit() bool { return cs.AccumulateCount > (4096 / impactFactor) } -func (cs *CompactedFileSplitStrategy) ShouldSkip(subCompaction *backuppb.LogFileSubcompaction) bool { - _, exist := cs.Rules[subCompaction.Meta.TableId] +func (cs *CompactedFileSplitStrategy) ShouldSkip(subCompaction SSTs) bool { + _, exist := cs.Rules[subCompaction.TableID()] if !exist { - log.Info("skip for no rule files", zap.Int64("tableID", subCompaction.Meta.TableId)) + log.Info("skip for no rule files", zap.Int64("tableID", subCompaction.TableID())) return true } - sstOutputs := make([]*backuppb.File, 0, len(subCompaction.SstOutputs)) - for _, sst := range subCompaction.SstOutputs { + sstOutputs := make([]*backuppb.File, 0, len(subCompaction.GetSSTs())) + for _, sst := range subCompaction.GetSSTs() { if _, ok := cs.checkpointSets[sst.Name]; !ok { sstOutputs = append(sstOutputs, sst) } else { @@ -103,10 +103,11 @@ func (cs *CompactedFileSplitStrategy) ShouldSkip(subCompaction *backuppb.LogFile log.Info("all files in sub compaction skipped") return true } - if len(sstOutputs) != len(subCompaction.SstOutputs) { - log.Info("partial files in sub compaction skipped due to checkpoint") - subCompaction.SstOutputs = sstOutputs - return false - } + // TODO: Add a method for removing it. + // if len(sstOutputs) != len(subCompaction.SstOutputs) { + // log.Info("partial files in sub compaction skipped due to checkpoint") + // subCompaction.SstOutputs = sstOutputs + // return false + // } return false } diff --git a/br/pkg/restore/log_client/log_file_manager.go b/br/pkg/restore/log_client/log_file_manager.go index 4c2992467a2ab..2844e14be6c49 100644 --- a/br/pkg/restore/log_client/log_file_manager.go +++ b/br/pkg/restore/log_client/log_file_manager.go @@ -12,6 +12,7 @@ import ( "time" "github.com/pingcap/errors" + backup "github.com/pingcap/kvproto/pkg/brpb" backuppb "github.com/pingcap/kvproto/pkg/brpb" "github.com/pingcap/kvproto/pkg/encryptionpb" "github.com/pingcap/log" @@ -347,8 +348,26 @@ 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[*backuppb.LogFileSubcompaction] { - return rc.withMigrations.Compactions(ctx, rc.storage) +func (rc *LogFileManager) GetCompactionIter(ctx context.Context) iter.TryNextor[SSTs] { + return iter.Map(rc.withMigrations.Compactions(ctx, rc.storage), func(c *backup.LogFileSubcompaction) SSTs { + return &CompactedSSTs{c} + }) +} + +func (rc *LogFileManager) GetExtraFullBackupSSTs(ctx context.Context) iter.TryNextor[SSTs] { + return iter.FlatMap(rc.withMigrations.ExtraFullBackups(ctx, rc.storage), func(c *backup.ExtraFullBackup) iter.TryNextor[SSTs] { + remap := map[int64]int64{} + for _, r := range c.RewrittenTables { + remap[r.UpstreamOfUpstream] = r.Upstream + } + return iter.TryMap(iter.FromSlice(c.Files), func(f *backup.File) (SSTs, error) { + sst := &AddedSSTs{File: f} + if id, ok := remap[sst.TableID()]; ok && id != sst.TableID() { + return nil, errors.Annotatef(berrors.ErrInvalidArgument, "table id %d is rewritten to %d in upstream", sst.TableID(), id) + } + return sst, nil + }) + }) } // the kv entry with ts, the ts is decoded from entry. diff --git a/br/pkg/restore/log_client/migration.go b/br/pkg/restore/log_client/migration.go index a7b4307e0f568..af2493195b179 100644 --- a/br/pkg/restore/log_client/migration.go +++ b/br/pkg/restore/log_client/migration.go @@ -144,6 +144,7 @@ func (builder *WithMigrationsBuilder) coarseGrainedFilter(mig *backuppb.Migratio func (builder *WithMigrationsBuilder) Build(migs []*backuppb.Migration) WithMigrations { skipmap := make(metaSkipMap) compactionDirs := make([]string, 0, 8) + fullBackups := make([]*backuppb.ExtraFullBackup, 0, 8) for _, mig := range migs { // TODO: deal with TruncatedTo and DestructPrefix @@ -155,10 +156,15 @@ func (builder *WithMigrationsBuilder) Build(migs []*backuppb.Migration) WithMigr for _, c := range mig.Compactions { compactionDirs = append(compactionDirs, c.Artifacts) } + + for _, fullBackup := range mig.ExtraFullBackups { + fullBackups = append(fullBackups, fullBackup) + } } withMigrations := WithMigrations{ skipmap: skipmap, compactionDirs: compactionDirs, + fullBackups: fullBackups, } return withMigrations } @@ -210,6 +216,7 @@ func (mwm *MetaWithMigrations) Physicals(groupIndexIter GroupIndexIter) Physical type WithMigrations struct { skipmap metaSkipMap compactionDirs []string + fullBackups []*backuppb.ExtraFullBackup } func (wm *WithMigrations) Metas(metaNameIter MetaNameIter) MetaMigrationsIter { @@ -238,3 +245,7 @@ func (wm *WithMigrations) Compactions(ctx context.Context, s storage.ExternalSto return Subcompactions(ctx, name, s) }) } + +func (wm *WithMigrations) ExtraFullBackups(ctx context.Context, s storage.ExternalStorage) iter.TryNextor[*backuppb.ExtraFullBackup] { + return iter.FromSlice(wm.fullBackups) +} diff --git a/br/pkg/restore/log_client/ssts.go b/br/pkg/restore/log_client/ssts.go new file mode 100644 index 0000000000000..30c810a716e66 --- /dev/null +++ b/br/pkg/restore/log_client/ssts.go @@ -0,0 +1,60 @@ +// Copyright 2024 PingCAP, Inc. Licensed under Apache-2.0. + +package logclient + +import ( + "encoding/hex" + "fmt" + "sync/atomic" + + backuppb "github.com/pingcap/kvproto/pkg/brpb" + "github.com/pingcap/tidb/pkg/tablecodec" +) + +type SSTs interface { + TableID() int64 + GetSSTs() []*backuppb.File +} + +type CompactedSSTs struct { + *backuppb.LogFileSubcompaction +} + +func (s *CompactedSSTs) TableID() int64 { + return s.Meta.TableId +} + +func (s *CompactedSSTs) GetSSTs() []*backuppb.File { + return s.SstOutputs +} + +type AddedSSTs struct { + File *backuppb.File + + cachedTableID atomic.Int64 +} + +func (s *AddedSSTs) TableID() int64 { + cached := s.cachedTableID.Load() + if cached == 0 { + id := tablecodec.DecodeTableID(s.File.StartKey) + id2 := tablecodec.DecodeTableID(s.File.EndKey) + if id != id2 { + panic(fmt.Sprintf( + "yet restoring a SST with two adjacent tables not supported, they are %d and %d (start key = %s; end key = %s)", + id, + id2, + hex.EncodeToString(s.File.StartKey), + hex.EncodeToString(s.File.EndKey), + )) + } + s.cachedTableID.Store(id) + return id + } + + return cached +} + +func (s *AddedSSTs) GetSSTs() []*backuppb.File { + return []*backuppb.File{s.File} +} diff --git a/br/pkg/restore/snap_client/import.go b/br/pkg/restore/snap_client/import.go index 4e71c6fbe0cc4..1238754c394bf 100644 --- a/br/pkg/restore/snap_client/import.go +++ b/br/pkg/restore/snap_client/import.go @@ -453,7 +453,7 @@ func getSSTMetaFromFile( } // Get the column family of the file by the file name. - var cfName string + cfName := file.GetCf() if strings.Contains(file.GetName(), restoreutils.DefaultCFName) { cfName = restoreutils.DefaultCFName } else if strings.Contains(file.GetName(), restoreutils.WriteCFName) { diff --git a/br/pkg/stream/stream_metas.go b/br/pkg/stream/stream_metas.go index ec5514fd88a76..1909d300f242e 100644 --- a/br/pkg/stream/stream_metas.go +++ b/br/pkg/stream/stream_metas.go @@ -374,6 +374,9 @@ func AddMigrationToTable(m *pb.Migration, table *glue.Table) { for i, c := range m.DestructPrefix { table.Add(fmt.Sprintf("destruct-prefix[%02d]", i), rd(c)) } + for i, c := range m.ExtraFullBackups { + table.Add(fmt.Sprintf("extra_full_backups[%02d]", i), rd(c.FilesPrefixHint)) + } table.Add("truncate-to", rd(m.TruncatedTo)) } diff --git a/br/pkg/task/stream.go b/br/pkg/task/stream.go index 678fda6cbd621..a8263f80e7878 100644 --- a/br/pkg/task/stream.go +++ b/br/pkg/task/stream.go @@ -54,6 +54,7 @@ import ( "github.com/pingcap/tidb/br/pkg/streamhelper/daemon" "github.com/pingcap/tidb/br/pkg/summary" "github.com/pingcap/tidb/br/pkg/utils" + "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/util/cdcutil" @@ -1437,7 +1438,9 @@ func restoreStream( return errors.Trace(err) } + addedSSTsIter := client.LogFileManager.GetExtraFullBackupSSTs(ctx) compactionIter := client.LogFileManager.GetCompactionIter(ctx) + sstsIter := iter.ConcatAll(addedSSTsIter, compactionIter) se, err := g.CreateSession(mgr.GetStorage()) if err != nil { @@ -1460,7 +1463,7 @@ func restoreStream( p.IncBy(int64(kvCount)) } compactedSplitIter, err := client.WrapCompactedFilesIterWithSplitHelper( - ctx, compactionIter, rewriteRules, sstCheckpointSets, + ctx, sstsIter, rewriteRules, sstCheckpointSets, updateStatsWithCheckpoint, splitSize, splitKeys, ) if err != nil { From addb8f7d0f390b04c9e8615cd6cdc47888c6574a Mon Sep 17 00:00:00 2001 From: hillium Date: Tue, 26 Nov 2024 17:49:07 +0800 Subject: [PATCH 03/36] initial commit Signed-off-by: hillium --- br/pkg/restore/log_client/client.go | 25 +++++++++++++++- .../log_client/compacted_file_strategy.go | 27 ++++++++++++----- br/pkg/restore/log_client/log_file_manager.go | 5 +++- br/pkg/restore/log_client/migration.go | 4 +-- br/pkg/restore/log_client/ssts.go | 29 +++++++++++++++++- br/pkg/restore/snap_client/pitr_collector.go | 2 +- br/pkg/restore/utils/rewrite_rule.go | 30 +++++++++++++++++++ br/pkg/storage/local.go | 11 ++++--- br/pkg/storage/storage.go | 1 + 9 files changed, 116 insertions(+), 18 deletions(-) diff --git a/br/pkg/restore/log_client/client.go b/br/pkg/restore/log_client/client.go index b5f23ccede077..801acb1b68700 100644 --- a/br/pkg/restore/log_client/client.go +++ b/br/pkg/restore/log_client/client.go @@ -40,6 +40,7 @@ import ( "github.com/pingcap/tidb/br/pkg/conn" "github.com/pingcap/tidb/br/pkg/conn/util" "github.com/pingcap/tidb/br/pkg/encryption" + berrors "github.com/pingcap/tidb/br/pkg/errors" "github.com/pingcap/tidb/br/pkg/glue" "github.com/pingcap/tidb/br/pkg/logutil" "github.com/pingcap/tidb/br/pkg/metautil" @@ -268,11 +269,33 @@ func (rc *LogClient) RestoreCompactedSstFiles( return r.Err } i := r.Item - rewriteRules, ok := rules[i.TableID()] + + tid := i.TableID() + if r, ok := i.(RewrittenSST); ok && r.RewrittenTo() > 0 { + tid = r.RewrittenTo() + } + rewriteRules, ok := rules[tid] if !ok { log.Warn("[Compacted SST Restore] Skipping excluded table during restore.", zap.Int64("table_id", i.TableID())) continue } + + if r, ok := i.(RewrittenSST); ok { + rewritten := r.RewrittenTo() + if rewritten > 0 && rewritten != i.TableID() { + rewriteRules = rewriteRules.Clone() + if !rewriteRules.RewriteSourceTableID(rewritten, i.TableID()) { + return errors.Annotatef( + berrors.ErrUnknown, + "table rewritten from a table id (%d) to (%d) which doesn't exist in the stream", + rewritten, + i.TableID(), + ) + } + log.Info("Rewritten rewrite rules.", zap.Stringer("rules", rewriteRules), zap.Int64("table_id", i.TableID()), zap.Int64("rewritten_to", rewritten)) + } + } + set := restore.BackupFileSet{ TableID: i.TableID(), SSTFiles: i.GetSSTs(), diff --git a/br/pkg/restore/log_client/compacted_file_strategy.go b/br/pkg/restore/log_client/compacted_file_strategy.go index 2db62323ee238..0c9a8a7a81272 100644 --- a/br/pkg/restore/log_client/compacted_file_strategy.go +++ b/br/pkg/restore/log_client/compacted_file_strategy.go @@ -49,7 +49,7 @@ func (cs *CompactedFileSplitStrategy) Accumulate(ssts SSTs) { endKey := codec.EncodeBytes(nil, f.EndKey) cs.AccumulateCount += 1 if f.TotalKvs == 0 || f.Size_ == 0 { - log.Error("No key-value pairs in subcompaction", zap.String("name", f.Name)) + log.Warn("No key-value pairs in subcompaction", zap.String("name", f.Name)) continue } // The number of MVCC entries in the compacted SST files can be excessive. @@ -82,14 +82,27 @@ func (cs *CompactedFileSplitStrategy) ShouldSplit() bool { return cs.AccumulateCount > (4096 / impactFactor) } -func (cs *CompactedFileSplitStrategy) ShouldSkip(subCompaction SSTs) bool { - _, exist := cs.Rules[subCompaction.TableID()] - if !exist { - log.Info("skip for no rule files", zap.Int64("tableID", subCompaction.TableID())) +func hasARule[T any](ssts SSTs, rules map[int64]T) bool { + if _, exist := rules[ssts.TableID()]; exist { + return true + } + + if r, ok := ssts.(RewrittenSST); ok { + if _, exist := rules[r.RewrittenTo()]; exist { + return true + } + } + + return false +} + +func (cs *CompactedFileSplitStrategy) ShouldSkip(ssts SSTs) bool { + if !hasARule(ssts, cs.Rules) { + log.Warn("skip for no rule files", zap.Int64("tableID", ssts.TableID()), zap.Any("ssts", ssts)) return true } - sstOutputs := make([]*backuppb.File, 0, len(subCompaction.GetSSTs())) - for _, sst := range subCompaction.GetSSTs() { + sstOutputs := make([]*backuppb.File, 0, len(ssts.GetSSTs())) + for _, sst := range ssts.GetSSTs() { if _, ok := cs.checkpointSets[sst.Name]; !ok { sstOutputs = append(sstOutputs, sst) } else { diff --git a/br/pkg/restore/log_client/log_file_manager.go b/br/pkg/restore/log_client/log_file_manager.go index 2844e14be6c49..351fbbbe22e1d 100644 --- a/br/pkg/restore/log_client/log_file_manager.go +++ b/br/pkg/restore/log_client/log_file_manager.go @@ -363,7 +363,10 @@ func (rc *LogFileManager) GetExtraFullBackupSSTs(ctx context.Context) iter.TryNe return iter.TryMap(iter.FromSlice(c.Files), func(f *backup.File) (SSTs, error) { sst := &AddedSSTs{File: f} if id, ok := remap[sst.TableID()]; ok && id != sst.TableID() { - return nil, errors.Annotatef(berrors.ErrInvalidArgument, "table id %d is rewritten to %d in upstream", sst.TableID(), id) + sst.Rewritten = backuppb.RewrittenTableID{ + UpstreamOfUpstream: sst.TableID(), + Upstream: id, + } } return sst, nil }) diff --git a/br/pkg/restore/log_client/migration.go b/br/pkg/restore/log_client/migration.go index af2493195b179..059115d134286 100644 --- a/br/pkg/restore/log_client/migration.go +++ b/br/pkg/restore/log_client/migration.go @@ -157,9 +157,7 @@ func (builder *WithMigrationsBuilder) Build(migs []*backuppb.Migration) WithMigr compactionDirs = append(compactionDirs, c.Artifacts) } - for _, fullBackup := range mig.ExtraFullBackups { - fullBackups = append(fullBackups, fullBackup) - } + fullBackups = append(fullBackups, mig.ExtraFullBackups...) } withMigrations := WithMigrations{ skipmap: skipmap, diff --git a/br/pkg/restore/log_client/ssts.go b/br/pkg/restore/log_client/ssts.go index 30c810a716e66..fc8aa8f26d81a 100644 --- a/br/pkg/restore/log_client/ssts.go +++ b/br/pkg/restore/log_client/ssts.go @@ -11,8 +11,30 @@ import ( "github.com/pingcap/tidb/pkg/tablecodec" ) +var ( + _ RewrittenSST = &AddedSSTs{} +) + +// RewrittenSST is an extension to the `SSTs` that needs extra key rewriting. +// This allows a SST being restored "as if" it in another table. +// +// The name "rewritten" means that the SST has already been rewritten somewhere else -- +// before importing it, we need "replay" the rewrite on it. +// +// For example, if a SST contains content of table `1`. And `RewrittenTo` returns `10`, +// the downstream wants to rewrite table `10` to `100`: +// - When searching for rewrite rules for the SSTs, we will use the table ID `10`(`RewrittenTo()`). +// - When importing the SST, we will use the rewrite rule `1`(`TableID()`) -> `100`(RewriteRule). +type RewrittenSST interface { + // RewrittenTo returns the table ID that the SST should be treated as. + RewrittenTo() int64 +} + +// SSTs is an interface that represents a collection of SST files. type SSTs interface { + // TableID returns the ID of the table associated with the SST files. TableID() int64 + // GetSSTs returns a slice of pointers to backuppb.File, representing the SST files. GetSSTs() []*backuppb.File } @@ -29,7 +51,8 @@ func (s *CompactedSSTs) GetSSTs() []*backuppb.File { } type AddedSSTs struct { - File *backuppb.File + File *backuppb.File + Rewritten backuppb.RewrittenTableID cachedTableID atomic.Int64 } @@ -58,3 +81,7 @@ func (s *AddedSSTs) TableID() int64 { func (s *AddedSSTs) GetSSTs() []*backuppb.File { return []*backuppb.File{s.File} } + +func (s *AddedSSTs) RewrittenTo() int64 { + return s.Rewritten.Upstream +} diff --git a/br/pkg/restore/snap_client/pitr_collector.go b/br/pkg/restore/snap_client/pitr_collector.go index 8410f5111618e..9284606e65aef 100644 --- a/br/pkg/restore/snap_client/pitr_collector.go +++ b/br/pkg/restore/snap_client/pitr_collector.go @@ -67,7 +67,7 @@ func (p pitrCollectorRestorer) GoRestore(onProgress func(int64), batchFileSets . // WaitUntilFinish blocks until all pending restore files have completed processing. func (p pitrCollectorRestorer) WaitUntilFinish() error { - return errors.Annotate(p.wg.Wait(), "failed to wait on wait pitrCollector") + return errors.Annotate(p.wg.Wait(), "failed to wait on pitrCollector") } // Close releases any resources associated with the restoration process. diff --git a/br/pkg/restore/utils/rewrite_rule.go b/br/pkg/restore/utils/rewrite_rule.go index 9afa639b8ca8b..2a1480f2e1d96 100644 --- a/br/pkg/restore/utils/rewrite_rule.go +++ b/br/pkg/restore/utils/rewrite_rule.go @@ -27,6 +27,7 @@ import ( "github.com/pingcap/tidb/br/pkg/rtree" "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/tablecodec" + "github.com/pingcap/tidb/pkg/util" "github.com/pingcap/tidb/pkg/util/codec" "github.com/pingcap/tidb/pkg/util/redact" "go.uber.org/zap" @@ -52,6 +53,35 @@ type RewriteRules struct { TableIDRemapHint []TableIDRemap } +func (r *RewriteRules) RewriteSourceTableID(from, to int64) (rewritten bool) { + toPrefix := tablecodec.EncodeTablePrefix(to) + fromPrefix := tablecodec.EncodeTablePrefix(from) + for _, rule := range r.Data { + if bytes.HasPrefix(rule.OldKeyPrefix, fromPrefix) { + rule.OldKeyPrefix = append(toPrefix, rule.OldKeyPrefix[len(toPrefix):]...) + rewritten = true + } + } + return +} + +func (r *RewriteRules) Clone() *RewriteRules { + data := make([]*import_sstpb.RewriteRule, len(r.Data)) + for i, rule := range r.Data { + data[i] = util.ProtoV1Clone(rule) + } + remap := make([]TableIDRemap, len(r.TableIDRemapHint)) + copy(remap, r.TableIDRemapHint) + + return &RewriteRules{ + Data: data, + TableIDRemapHint: remap, + OldKeyspace: r.OldKeyspace, + NewKeyspace: r.NewKeyspace, + NewTableID: r.NewTableID, + } +} + // TableIDRemap presents a remapping of table id during rewriting. type TableIDRemap struct { Origin int64 diff --git a/br/pkg/storage/local.go b/br/pkg/storage/local.go index 35f6a94cd8cbe..180410c97be12 100644 --- a/br/pkg/storage/local.go +++ b/br/pkg/storage/local.go @@ -14,6 +14,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/log" + berrors "github.com/pingcap/tidb/br/pkg/errors" "go.uber.org/zap" ) @@ -259,11 +260,13 @@ func (l *LocalStorage) Rename(_ context.Context, oldFileName, newFileName string func (*LocalStorage) Close() {} func (l *LocalStorage) CopyFrom(ctx context.Context, e ExternalStorage, spec CopySpec) error { - bs, err := e.ReadFile(ctx, spec.From) - if err != nil { - return errors.Trace(err) + sl, ok := e.(*LocalStorage) + if !ok { + return errors.Annotatef(berrors.ErrInvalidArgument, "expect source to be LocalStorage, got %T", e) } - return l.WriteFile(ctx, spec.To, bs) + from := filepath.Join(sl.base, spec.From) + to := filepath.Join(l.base, spec.To) + return os.Link(from, to) } func pathExists(_path string) (bool, error) { diff --git a/br/pkg/storage/storage.go b/br/pkg/storage/storage.go index 82857a98bdcd8..e6d9212f3065e 100644 --- a/br/pkg/storage/storage.go +++ b/br/pkg/storage/storage.go @@ -102,6 +102,7 @@ type ReaderOption struct { } type Copier interface { + // CopyFrom copies a object to the current external storage by the specification. CopyFrom(ctx context.Context, e ExternalStorage, spec CopySpec) error } From e00d5c1744d4c4dbbc7da1fe28ba36be9eaf2414 Mon Sep 17 00:00:00 2001 From: hillium Date: Wed, 27 Nov 2024 14:41:56 +0800 Subject: [PATCH 04/36] added basic test toolkit Signed-off-by: hillium --- br/pkg/glue/glue.go | 20 ++++ br/pkg/gluetidb/glue.go | 4 + br/pkg/gluetidb/glue_test.go | 4 +- br/pkg/restore/log_client/client.go | 34 ++++++- br/pkg/task/stream.go | 41 +++++--- tests/realtikvtest/brietest/pitr_test.go | 121 +++++++++++++++++++++++ 6 files changed, 209 insertions(+), 15 deletions(-) create mode 100644 tests/realtikvtest/brietest/pitr_test.go diff --git a/br/pkg/glue/glue.go b/br/pkg/glue/glue.go index d691d638169ab..0a7b360039c2d 100644 --- a/br/pkg/glue/glue.go +++ b/br/pkg/glue/glue.go @@ -4,6 +4,7 @@ package glue import ( "context" + "sync/atomic" "github.com/pingcap/tidb/pkg/ddl" "github.com/pingcap/tidb/pkg/domain" @@ -82,3 +83,22 @@ type Progress interface { // called. Close() } + +type CounterProgress struct { + Counter atomic.Int64 +} + +func (c *CounterProgress) Inc() { + c.Counter.Add(1) +} + +func (c *CounterProgress) IncBy(cnt int64) { + c.Counter.Add(cnt) +} + +func (c *CounterProgress) GetCurrent() int64 { + return c.Counter.Load() +} + +func (c *CounterProgress) Close() { +} diff --git a/br/pkg/gluetidb/glue.go b/br/pkg/gluetidb/glue.go index 9514ae4f5f7a1..622e82d9a52ac 100644 --- a/br/pkg/gluetidb/glue.go +++ b/br/pkg/gluetidb/glue.go @@ -55,6 +55,10 @@ type Glue struct { startDomainMu *sync.Mutex } +func WrapSession(se sessiontypes.Session) glue.Session { + return &tidbSession{se: se} +} + type tidbSession struct { se sessiontypes.Session } diff --git a/br/pkg/gluetidb/glue_test.go b/br/pkg/gluetidb/glue_test.go index ceb43566f503b..f9d3287998dd7 100644 --- a/br/pkg/gluetidb/glue_test.go +++ b/br/pkg/gluetidb/glue_test.go @@ -92,9 +92,9 @@ func TestTheSessionIsoation(t *testing.T) { }, } for _, pinfo := range polices { - before := glueSe.(*tidbSession).se.GetInfoSchema().SchemaMetaVersion() + before := glueSe.(*TiDBSession).Se.GetInfoSchema().SchemaMetaVersion() req.NoError(glueSe.CreatePlacementPolicy(ctx, pinfo)) - after := glueSe.(*tidbSession).se.GetInfoSchema().SchemaMetaVersion() + after := glueSe.(*TiDBSession).Se.GetInfoSchema().SchemaMetaVersion() req.Greater(after, before) } req.NoError(glueSe.(glue.BatchCreateTableSession).CreateTables(ctx, map[string][]*model.TableInfo{ diff --git a/br/pkg/restore/log_client/client.go b/br/pkg/restore/log_client/client.go index 801acb1b68700..3c2ace1b02c70 100644 --- a/br/pkg/restore/log_client/client.go +++ b/br/pkg/restore/log_client/client.go @@ -25,8 +25,10 @@ import ( "strconv" "strings" "sync" + "sync/atomic" "time" + "github.com/docker/go-units" "github.com/fatih/color" "github.com/gogo/protobuf/proto" "github.com/opentracing/opentracing-go" @@ -212,6 +214,11 @@ type LogClient struct { // checkpoint information for log restore useCheckpoint bool + + restoreSSTKVSize atomic.Uint64 + restoreSSTKVCount atomic.Uint64 + restoreSSTPhySize atomic.Uint64 + restoreSSTTakes atomic.Uint64 } // NewRestoreClient returns a new RestoreClient. @@ -260,6 +267,7 @@ func (rc *LogClient) RestoreCompactedSstFiles( importModeSwitcher *restore.ImportModeSwitcher, onProgress func(int64), ) error { + begin := time.Now() backupFileSets := make([]restore.BackupFileSet, 0, 8) // Collect all items from the iterator in advance to avoid blocking during restoration. // This approach ensures that we have all necessary data ready for processing, @@ -327,7 +335,31 @@ func (rc *LogClient) RestoreCompactedSstFiles( return errors.Trace(err) } } - return rc.sstRestoreManager.restorer.WaitUntilFinish() + err := rc.sstRestoreManager.restorer.WaitUntilFinish() + + for _, files := range backupFileSets { + for _, f := range files.SSTFiles { + log.Info("Collected file.", zap.Uint64("total_kv", f.TotalKvs), zap.Uint64("total_bytes", f.TotalBytes), zap.Uint64("size", f.Size_)) + rc.restoreSSTKVCount.Add(f.TotalKvs) + rc.restoreSSTKVSize.Add(f.TotalBytes) + rc.restoreSSTPhySize.Add(f.Size_) + } + } + rc.restoreSSTTakes.Add(uint64(time.Since(begin))) + + return err +} + +func (rc *LogClient) RestoreSSTStatisticFields(pushTo *[]zapcore.Field) { + takes := time.Duration(rc.restoreSSTTakes.Load()) + fields := []zapcore.Field{ + zap.Uint64("restore-sst-kv-count", rc.restoreSSTKVCount.Load()), + zap.Uint64("restore-sst-kv-size", rc.restoreSSTKVSize.Load()), + zap.Uint64("restore-sst-physical-size (after compression)", rc.restoreSSTPhySize.Load()), + zap.Duration("restore-sst-total-take", takes), + zap.String("average-speed (sst)", units.HumanSize(float64(rc.restoreSSTKVSize.Load())/takes.Seconds())+"/s"), + } + *pushTo = append(*pushTo, fields...) } func (rc *LogClient) SetRawKVBatchClient( diff --git a/br/pkg/task/stream.go b/br/pkg/task/stream.go index a8263f80e7878..0e8dd76aa9085 100644 --- a/br/pkg/task/stream.go +++ b/br/pkg/task/stream.go @@ -62,6 +62,7 @@ import ( "github.com/tikv/client-go/v2/oracle" clientv3 "go.etcd.io/etcd/client/v3" "go.uber.org/zap" + "go.uber.org/zap/zapcore" ) const ( @@ -131,6 +132,18 @@ type StreamConfig struct { AdvancerCfg advancercfg.Config `json:"advancer-config" toml:"advancer-config"` } +func DefaultStreamConfig(flagsDef func(*pflag.FlagSet)) StreamConfig { + fs := pflag.NewFlagSet("dummy", pflag.ContinueOnError) + flagsDef(fs) + DefineCommonFlags(fs) + cfg := StreamConfig{} + err := cfg.ParseFromFlags(fs) + if err != nil { + log.Panic("failed to parse backup flags to config", zap.Error(err)) + } + return cfg +} + func (cfg *StreamConfig) makeStorage(ctx context.Context) (storage.ExternalStorage, error) { u, err := storage.ParseBackend(cfg.Storage, &cfg.BackendOptions) if err != nil { @@ -1256,6 +1269,7 @@ func restoreStream( checkpointTotalKVCount uint64 checkpointTotalSize uint64 currentTS uint64 + extraFields []zapcore.Field mu sync.Mutex startTime = time.Now() ) @@ -1264,18 +1278,20 @@ func restoreStream( summary.Log("restore log failed summary", zap.Error(err)) } else { totalDureTime := time.Since(startTime) - summary.Log("restore log success summary", zap.Duration("total-take", totalDureTime), - 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.Uint64("total-kv-count", totalKVCount), - zap.Uint64("skipped-kv-count-by-checkpoint", checkpointTotalKVCount), - zap.String("total-size", units.HumanSize(float64(totalSize))), - zap.String("skipped-size-by-checkpoint", units.HumanSize(float64(checkpointTotalSize))), - zap.String("average-speed", units.HumanSize(float64(totalSize)/totalDureTime.Seconds())+"/s"), + summary.Log("restore log success summary", + append([]zapcore.Field{zap.Duration("total-take", totalDureTime), + 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.Uint64("total-kv-count", totalKVCount), + zap.Uint64("skipped-kv-count-by-checkpoint", checkpointTotalKVCount), + zap.String("total-size", units.HumanSize(float64(totalSize))), + zap.String("skipped-size-by-checkpoint", units.HumanSize(float64(checkpointTotalSize))), + zap.String("average-speed (log)", units.HumanSize(float64(totalSize)/totalDureTime.Seconds())+"/s")}, + extraFields...)..., ) } }() @@ -1297,6 +1313,7 @@ func restoreStream( return errors.Annotate(err, "failed to create restore client") } defer client.Close(ctx) + defer client.RestoreSSTStatisticFields(&extraFields) if taskInfo != nil && taskInfo.Metadata != nil { // reuse the task's rewrite ts diff --git a/tests/realtikvtest/brietest/pitr_test.go b/tests/realtikvtest/brietest/pitr_test.go new file mode 100644 index 0000000000000..e9e2b5ebdb1ed --- /dev/null +++ b/tests/realtikvtest/brietest/pitr_test.go @@ -0,0 +1,121 @@ +package brietest + +import ( + "context" + "math" + "testing" + + "github.com/pingcap/tidb/br/pkg/glue" + "github.com/pingcap/tidb/br/pkg/gluetidb" + "github.com/pingcap/tidb/br/pkg/streamhelper" + "github.com/pingcap/tidb/br/pkg/task" + "github.com/pingcap/tidb/pkg/domain" + "github.com/pingcap/tidb/pkg/kv" + "github.com/pingcap/tidb/pkg/testkit" + "github.com/pingcap/tidb/pkg/util/printer" + "github.com/stretchr/testify/require" + pd "github.com/tikv/pd/client" +) + +type TestKitGlue struct { + tk *testkit.TestKit +} + +func (tk TestKitGlue) GetDomain(_ kv.Storage) (*domain.Domain, error) { + return domain.GetDomain(tk.tk.Session()), nil +} + +func (tk TestKitGlue) CreateSession(_ kv.Storage) (glue.Session, error) { + return gluetidb.WrapSession(tk.tk.Session()), nil +} + +func (tk TestKitGlue) Open(path string, option pd.SecurityOption) (kv.Storage, error) { + return tk.tk.Session().GetStore(), nil +} + +// OwnsStorage returns whether the storage returned by Open() is owned +// If this method returns false, the connection manager will never close the storage. +func (tk TestKitGlue) OwnsStorage() bool { + return false +} + +func (tk TestKitGlue) StartProgress(ctx context.Context, cmdName string, total int64, redirectLog bool) glue.Progress { + return &glue.CounterProgress{} +} + +// Record records some information useful for log-less summary. +func (tk TestKitGlue) Record(name string, value uint64) { +} + +// GetVersion gets BR package version to run backup/restore job +func (tk TestKitGlue) GetVersion() string { + return "In Test\n" + printer.GetTiDBInfo() +} + +// UseOneShotSession temporary creates session from store when run backup job. +// because we don't have to own domain/session during the whole backup. +// we can close domain as soon as possible. +// and we must reuse the exists session and don't close it in SQL backup job. +func (tk TestKitGlue) UseOneShotSession(_ kv.Storage, _ bool, fn func(se glue.Session) error) error { + return fn(gluetidb.WrapSession(tk.tk.Session())) +} + +// GetClient returns the client type of the glue +func (tk TestKitGlue) GetClient() glue.GlueClient { + return glue.ClientSql +} + +type LogBackupKit struct { + t *testing.T + tk *testkit.TestKit + metaCli *streamhelper.MetaDataClient + base string +} + +func NewLogBackupKit(t *testing.T) *LogBackupKit { + tk := initTestKit(t) + metaCli := streamhelper.NewMetaDataClient(domain.GetDomain(tk.Session()).EtcdClient()) + return &LogBackupKit{ + tk: tk, + t: t, + metaCli: metaCli, + base: t.TempDir(), + } +} + +func (kit *LogBackupKit) RunLogStart(taskName string) { + kit.mustExec(func(ctx context.Context) error { + cfg := task.DefaultStreamConfig(task.DefineStreamCommonFlags) + cfg.TaskName = taskName + return task.RunStreamStop(ctx, kit.Glue(), "stream stop[intest]", &cfg) + }) + + kit.mustExec(func(ctx context.Context) error { + cfg := task.DefaultStreamConfig(task.DefineStreamStartFlags) + cfg.Storage = "local://" + kit.base + cfg.TaskName = taskName + cfg.EndTS = math.MaxUint64 + err := task.RunStreamStart(ctx, kit.Glue(), "stream start[intest]", &cfg) + return err + }) +} + +func (kit *LogBackupKit) RunLogStatus(taskName string) { + +} + +func (kit *LogBackupKit) Glue() glue.Glue { + return &TestKitGlue{tk: kit.tk} +} + +func (kit *LogBackupKit) mustExec(f func(context.Context) error) { + ctx, cancel := context.WithCancel(context.Background()) + err := f(ctx) + cancel() + require.NoError(kit.t, err) +} + +func TestPiTR(t *testing.T) { + kit := NewLogBackupKit(t) + kit.RunLogStart("fiolvit") +} From f91c7ca695727471b917721d0f4f0574489f7252 Mon Sep 17 00:00:00 2001 From: hillium Date: Wed, 27 Nov 2024 18:07:48 +0800 Subject: [PATCH 05/36] added test cases Signed-off-by: hillium --- br/pkg/storage/local.go | 3 + tests/realtikvtest/brietest/pitr_test.go | 142 +++++++++++++++++++++-- 2 files changed, 138 insertions(+), 7 deletions(-) diff --git a/br/pkg/storage/local.go b/br/pkg/storage/local.go index 180410c97be12..b3e9ec399bc3b 100644 --- a/br/pkg/storage/local.go +++ b/br/pkg/storage/local.go @@ -266,6 +266,9 @@ func (l *LocalStorage) CopyFrom(ctx context.Context, e ExternalStorage, spec Cop } from := filepath.Join(sl.base, spec.From) to := filepath.Join(l.base, spec.To) + if err := mkdirAll(filepath.Dir(to)); err != nil { + return errors.Trace(err) + } return os.Link(from, to) } diff --git a/tests/realtikvtest/brietest/pitr_test.go b/tests/realtikvtest/brietest/pitr_test.go index e9e2b5ebdb1ed..bb5a9c1cc6492 100644 --- a/tests/realtikvtest/brietest/pitr_test.go +++ b/tests/realtikvtest/brietest/pitr_test.go @@ -2,8 +2,11 @@ package brietest import ( "context" + "fmt" "math" + "strings" "testing" + "time" "github.com/pingcap/tidb/br/pkg/glue" "github.com/pingcap/tidb/br/pkg/gluetidb" @@ -14,6 +17,8 @@ import ( "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/util/printer" "github.com/stretchr/testify/require" + "github.com/tikv/client-go/v2/oracle" + "github.com/tikv/client-go/v2/tikv" pd "github.com/tikv/pd/client" ) @@ -44,8 +49,7 @@ func (tk TestKitGlue) StartProgress(ctx context.Context, cmdName string, total i } // Record records some information useful for log-less summary. -func (tk TestKitGlue) Record(name string, value uint64) { -} +func (tk TestKitGlue) Record(name string, value uint64) {} // GetVersion gets BR package version to run backup/restore job func (tk TestKitGlue) GetVersion() string { @@ -75,6 +79,8 @@ type LogBackupKit struct { func NewLogBackupKit(t *testing.T) *LogBackupKit { tk := initTestKit(t) metaCli := streamhelper.NewMetaDataClient(domain.GetDomain(tk.Session()).EtcdClient()) + // So the cases can finish faster... + tk.MustExec("set config tikv `log-backup.max-flush-interval` = '30s';") return &LogBackupKit{ tk: tk, t: t, @@ -83,25 +89,77 @@ func NewLogBackupKit(t *testing.T) *LogBackupKit { } } -func (kit *LogBackupKit) RunLogStart(taskName string) { +func (kit *LogBackupKit) RunFullRestore(extConfig func(*task.RestoreConfig)) { + kit.mustExec(func(ctx context.Context) error { + cfg := task.DefaultRestoreConfig(task.DefaultConfig()) + cfg.Storage = "local://" + kit.base + "/full" + + extConfig(&cfg) + return task.RunRestore(ctx, kit.Glue(), task.FullRestoreCmd, &cfg) + }) +} + +func (kit *LogBackupKit) RunStreamRestore(extConfig func(*task.RestoreConfig)) { + kit.mustExec(func(ctx context.Context) error { + cfg := task.DefaultRestoreConfig(task.DefaultConfig()) + cfg.Storage = "local://" + kit.base + "/incr" + cfg.FullBackupStorage = "local://" + kit.base + "/full" + + extConfig(&cfg) + return task.RunRestore(ctx, kit.Glue(), task.PointRestoreCmd, &cfg) + }) +} + +func (kit *LogBackupKit) RunFullBackup(extConfig func(*task.BackupConfig)) { + kit.mustExec(func(ctx context.Context) error { + cfg := task.DefaultBackupConfig(task.DefaultConfig()) + cfg.Storage = "local://" + kit.base + "/full" + extConfig(&cfg) + return task.RunBackup(ctx, kit.Glue(), "backup full[intest]", &cfg) + }) +} + +func (kit *LogBackupKit) StopTaskIfExists(taskName string) { kit.mustExec(func(ctx context.Context) error { cfg := task.DefaultStreamConfig(task.DefineStreamCommonFlags) cfg.TaskName = taskName - return task.RunStreamStop(ctx, kit.Glue(), "stream stop[intest]", &cfg) + err := task.RunStreamStop(ctx, kit.Glue(), "stream stop[intest]", &cfg) + if err != nil && strings.Contains(err.Error(), "task not found") { + return nil + } + return err }) +} +func (kit *LogBackupKit) RunLogStart(taskName string, extConfig func(*task.StreamConfig)) { kit.mustExec(func(ctx context.Context) error { cfg := task.DefaultStreamConfig(task.DefineStreamStartFlags) - cfg.Storage = "local://" + kit.base + cfg.Storage = "local://" + kit.base + "/incr" cfg.TaskName = taskName cfg.EndTS = math.MaxUint64 + extConfig(&cfg) err := task.RunStreamStart(ctx, kit.Glue(), "stream start[intest]", &cfg) return err }) + kit.t.Cleanup(func() { kit.StopTaskIfExists(taskName) }) +} + +func (kit *LogBackupKit) ctx() context.Context { + return context.Background() } -func (kit *LogBackupKit) RunLogStatus(taskName string) { +func (kit *LogBackupKit) TSO() uint64 { + ts, err := kit.tk.Session().GetStore().(tikv.Storage).GetOracle().GetTimestamp(kit.ctx(), &oracle.Option{}) + require.NoError(kit.t, err) + return ts +} +func (kit *LogBackupKit) CheckpointTSOf(taskName string) uint64 { + task, err := kit.metaCli.GetTask(kit.ctx(), taskName) + require.NoError(kit.t, err) + ts, err := task.GetGlobalCheckPointTS(kit.ctx()) + require.NoError(kit.t, err) + return ts } func (kit *LogBackupKit) Glue() glue.Glue { @@ -115,7 +173,77 @@ func (kit *LogBackupKit) mustExec(f func(context.Context) error) { require.NoError(kit.t, err) } +func createSimpleTableWithData(kit *LogBackupKit) { + kit.tk.MustExec(fmt.Sprintf("DROP TABLE IF EXISTs test.%s", kit.t.Name())) + kit.tk.MustExec(fmt.Sprintf("CREATE TABLE test.%s(t text)", kit.t.Name())) + kit.tk.MustExec(fmt.Sprintf("INSERT INTO test.%s VALUES ('Ear'), ('Eye'), ('Nose')", kit.t.Name())) +} + +func insertSimpleIncreaseData(kit *LogBackupKit) { + kit.tk.MustExec(fmt.Sprintf("INSERT INTO test.%s VALUES ('Body')", kit.t.Name())) + kit.tk.MustExec(fmt.Sprintf("INSERT INTO test.%s VALUES ('Mind')", kit.t.Name())) +} + +func verifySimpleData(kit *LogBackupKit) { + kit.tk.MustQuery(fmt.Sprintf("SELECT * FROM test.%s", kit.t.Name())).Check([][]any{{"Ear"}, {"Eye"}, {"Nose"}, {"Body"}, {"Mind"}}) +} + +func cleanSimpleData(kit *LogBackupKit) { + kit.tk.MustExec(fmt.Sprintf("DROP TABLE test.%s", kit.t.Name())) +} + func TestPiTR(t *testing.T) { kit := NewLogBackupKit(t) - kit.RunLogStart("fiolvit") + + taskName := "simple" + createSimpleTableWithData(kit) + + ts := kit.TSO() + kit.RunFullBackup(func(bc *task.BackupConfig) { bc.BackupTS = ts }) + kit.RunLogStart(taskName, func(sc *task.StreamConfig) { sc.StartTS = ts }) + + insertSimpleIncreaseData(kit) + + ts = kit.TSO() + require.Eventually(t, func() bool { + return kit.CheckpointTSOf(taskName) >= ts + }, 300*time.Second, 10*time.Second) + + cleanSimpleData(kit) + + kit.StopTaskIfExists(taskName) + kit.RunStreamRestore(func(rc *task.RestoreConfig) {}) + verifySimpleData(kit) +} + +func TestPiTRAndBackup(t *testing.T) { + kit := NewLogBackupKit(t) + createSimpleTableWithData(kit) + insertSimpleIncreaseData(kit) + + taskName := t.Name() + + kit.RunFullBackup(func(bc *task.BackupConfig) {}) + cleanSimpleData(kit) + + ts := kit.TSO() + kit.RunFullBackup(func(bc *task.BackupConfig) { + bc.Storage = "local://" + kit.base + "/full2" + bc.BackupTS = ts + }) + kit.RunLogStart(taskName, func(sc *task.StreamConfig) { + sc.StartTS = ts + }) + kit.RunFullRestore(func(rc *task.RestoreConfig) {}) + + ts = kit.TSO() + require.Eventually(t, func() bool { + return kit.CheckpointTSOf(taskName) >= ts + }, 300*time.Second, 10*time.Second) + + cleanSimpleData(kit) + kit.RunStreamRestore(func(rc *task.RestoreConfig) { + rc.FullBackupStorage = "local://" + kit.base + "/full2" + }) + verifySimpleData(kit) } From 651b9b5cb8c476464d144b3ae38500f1e3a02df2 Mon Sep 17 00:00:00 2001 From: hillium Date: Mon, 2 Dec 2024 16:38:46 +0800 Subject: [PATCH 06/36] added force-flush to speed up testing Signed-off-by: hillium --- br/pkg/gluetidb/glue_test.go | 4 +- br/pkg/restore/log_client/client.go | 21 ++++++- br/pkg/restore/log_client/client_test.go | 36 +++++------ br/pkg/storage/locking.go | 31 ++++++++++ br/pkg/storage/s3.go | 2 + br/pkg/stream/stream_metas.go | 19 +++--- br/pkg/stream/stream_metas_test.go | 12 ++-- br/pkg/streamhelper/basic_lib_for_test.go | 5 ++ br/pkg/streamhelper/daemon/owner_daemon.go | 2 +- br/pkg/task/operator/config.go | 11 ++++ br/pkg/task/operator/force_flush.go | 71 ++++++++++++++++++++++ br/pkg/task/operator/prepare_snap.go | 20 ++++++ br/pkg/task/stream.go | 13 +++- go.mod | 2 +- go.sum | 4 +- tests/realtikvtest/brietest/main_test.go | 3 +- tests/realtikvtest/brietest/pitr_test.go | 40 +++++++++--- 17 files changed, 245 insertions(+), 51 deletions(-) create mode 100644 br/pkg/task/operator/force_flush.go diff --git a/br/pkg/gluetidb/glue_test.go b/br/pkg/gluetidb/glue_test.go index f9d3287998dd7..ceb43566f503b 100644 --- a/br/pkg/gluetidb/glue_test.go +++ b/br/pkg/gluetidb/glue_test.go @@ -92,9 +92,9 @@ func TestTheSessionIsoation(t *testing.T) { }, } for _, pinfo := range polices { - before := glueSe.(*TiDBSession).Se.GetInfoSchema().SchemaMetaVersion() + before := glueSe.(*tidbSession).se.GetInfoSchema().SchemaMetaVersion() req.NoError(glueSe.CreatePlacementPolicy(ctx, pinfo)) - after := glueSe.(*TiDBSession).Se.GetInfoSchema().SchemaMetaVersion() + after := glueSe.(*tidbSession).se.GetInfoSchema().SchemaMetaVersion() req.Greater(after, before) } req.NoError(glueSe.(glue.BatchCreateTableSession).CreateTables(ctx, map[string][]*model.TableInfo{ diff --git a/br/pkg/restore/log_client/client.go b/br/pkg/restore/log_client/client.go index 3c2ace1b02c70..383316883a932 100644 --- a/br/pkg/restore/log_client/client.go +++ b/br/pkg/restore/log_client/client.go @@ -34,6 +34,7 @@ import ( "github.com/opentracing/opentracing-go" "github.com/pingcap/errors" "github.com/pingcap/failpoint" + backup "github.com/pingcap/kvproto/pkg/brpb" backuppb "github.com/pingcap/kvproto/pkg/brpb" "github.com/pingcap/kvproto/pkg/encryptionpb" "github.com/pingcap/log" @@ -545,13 +546,29 @@ func (rc *LogClient) InitCheckpointMetadataForLogRestore( return gcRatio, nil } -func (rc *LogClient) GetMigrations(ctx context.Context) ([]*backuppb.Migration, error) { +type LockedMigrations struct { + Migs []*backup.Migration + ReadLock storage.RemoteLock +} + +func (rc *LogClient) GetMigrations(ctx context.Context) (*LockedMigrations, error) { ext := stream.MigrationExtension(rc.storage) migs, err := ext.Load(ctx) if err != nil { return nil, errors.Trace(err) } - return migs.ListAll(), nil + + ms := migs.ListAll() + readLock, err := ext.GetReadLock(ctx, "restore stream") + if err != nil { + return nil, err + } + + lms := &LockedMigrations{ + Migs: ms, + ReadLock: readLock, + } + return lms, nil } func (rc *LogClient) InstallLogFileManager(ctx context.Context, startTS, restoreTS uint64, metadataDownloadBatchSize uint, diff --git a/br/pkg/restore/log_client/client_test.go b/br/pkg/restore/log_client/client_test.go index 504d7bb798d72..d78b49f126bf3 100644 --- a/br/pkg/restore/log_client/client_test.go +++ b/br/pkg/restore/log_client/client_test.go @@ -1662,11 +1662,11 @@ func TestCompactedSplitStrategy(t *testing.T) { } cases := []struct { - MockSubcompationIter iter.TryNextor[*backuppb.LogFileSubcompaction] + MockSubcompationIter iter.TryNextor[logclient.SSTs] ExpectRegionEndKeys [][]byte }{ { - iter.FromSlice([]*backuppb.LogFileSubcompaction{ + iter.FromSlice([]logclient.SSTs{ fakeSubCompactionWithOneSst(1, 100, 16*units.MiB, 100), fakeSubCompactionWithOneSst(1, 200, 32*units.MiB, 200), fakeSubCompactionWithOneSst(2, 100, 48*units.MiB, 300), @@ -1681,7 +1681,7 @@ func TestCompactedSplitStrategy(t *testing.T) { }, }, { - iter.FromSlice([]*backuppb.LogFileSubcompaction{ + iter.FromSlice([]logclient.SSTs{ fakeSubCompactionWithOneSst(1, 100, 16*units.MiB, 100), fakeSubCompactionWithOneSst(1, 200, 32*units.MiB, 200), fakeSubCompactionWithOneSst(1, 100, 32*units.MiB, 10), @@ -1697,7 +1697,7 @@ func TestCompactedSplitStrategy(t *testing.T) { }, }, { - iter.FromSlice([]*backuppb.LogFileSubcompaction{ + iter.FromSlice([]logclient.SSTs{ fakeSubCompactionWithOneSst(1, 100, 16*units.MiB, 100), fakeSubCompactionWithOneSst(1, 200, 32*units.MiB, 200), fakeSubCompactionWithOneSst(2, 100, 32*units.MiB, 300), @@ -1722,7 +1722,7 @@ func TestCompactedSplitStrategy(t *testing.T) { mockPDCli.SetRegions(oriRegions) client := split.NewClient(mockPDCli, nil, nil, 100, 4) - wrapper := restore.PipelineRestorerWrapper[*backuppb.LogFileSubcompaction]{ + wrapper := restore.PipelineRestorerWrapper[logclient.SSTs]{ PipelineRegionsSplitter: split.NewPipelineRegionsSplitter(client, 4*units.MB, 400), } @@ -1777,14 +1777,14 @@ func TestCompactedSplitStrategyWithCheckpoint(t *testing.T) { } cases := []struct { - MockSubcompationIter iter.TryNextor[*backuppb.LogFileSubcompaction] + MockSubcompationIter iter.TryNextor[logclient.SSTs] CheckpointSet map[string]struct{} ProcessedKVCount int ProcessedSize int ExpectRegionEndKeys [][]byte }{ { - iter.FromSlice([]*backuppb.LogFileSubcompaction{ + iter.FromSlice([]logclient.SSTs{ fakeSubCompactionWithOneSst(1, 100, 16*units.MiB, 100), fakeSubCompactionWithOneSst(1, 200, 32*units.MiB, 200), fakeSubCompactionWithOneSst(2, 100, 48*units.MiB, 300), @@ -1804,7 +1804,7 @@ func TestCompactedSplitStrategyWithCheckpoint(t *testing.T) { }, }, { - iter.FromSlice([]*backuppb.LogFileSubcompaction{ + iter.FromSlice([]logclient.SSTs{ fakeSubCompactionWithOneSst(1, 100, 16*units.MiB, 100), fakeSubCompactionWithOneSst(1, 200, 32*units.MiB, 200), fakeSubCompactionWithOneSst(1, 100, 32*units.MiB, 10), @@ -1823,7 +1823,7 @@ func TestCompactedSplitStrategyWithCheckpoint(t *testing.T) { }, }, { - iter.FromSlice([]*backuppb.LogFileSubcompaction{ + iter.FromSlice([]logclient.SSTs{ fakeSubCompactionWithOneSst(1, 100, 16*units.MiB, 100), fakeSubCompactionWithOneSst(1, 200, 32*units.MiB, 200), fakeSubCompactionWithOneSst(2, 100, 32*units.MiB, 300), @@ -1846,7 +1846,7 @@ func TestCompactedSplitStrategyWithCheckpoint(t *testing.T) { }, }, { - iter.FromSlice([]*backuppb.LogFileSubcompaction{ + iter.FromSlice([]logclient.SSTs{ fakeSubCompactionWithOneSst(1, 100, 16*units.MiB, 100), fakeSubCompactionWithOneSst(1, 200, 32*units.MiB, 200), fakeSubCompactionWithOneSst(2, 100, 32*units.MiB, 300), @@ -1869,7 +1869,7 @@ func TestCompactedSplitStrategyWithCheckpoint(t *testing.T) { }, }, { - iter.FromSlice([]*backuppb.LogFileSubcompaction{ + iter.FromSlice([]logclient.SSTs{ fakeSubCompactionWithOneSst(1, 100, 16*units.MiB, 100), fakeSubCompactionWithMultiSsts(1, 200, 32*units.MiB, 200), fakeSubCompactionWithOneSst(2, 100, 32*units.MiB, 300), @@ -1900,7 +1900,7 @@ func TestCompactedSplitStrategyWithCheckpoint(t *testing.T) { mockPDCli.SetRegions(oriRegions) client := split.NewClient(mockPDCli, nil, nil, 100, 4) - wrapper := restore.PipelineRestorerWrapper[*backuppb.LogFileSubcompaction]{ + wrapper := restore.PipelineRestorerWrapper[logclient.SSTs]{ PipelineRegionsSplitter: split.NewPipelineRegionsSplitter(client, 4*units.MB, 400), } totalSize := 0 @@ -1932,8 +1932,8 @@ func TestCompactedSplitStrategyWithCheckpoint(t *testing.T) { } } -func fakeSubCompactionWithMultiSsts(tableID, rowID int64, length uint64, num uint64) *backuppb.LogFileSubcompaction { - return &backuppb.LogFileSubcompaction{ +func fakeSubCompactionWithMultiSsts(tableID, rowID int64, length uint64, num uint64) logclient.SSTs { + return &logclient.CompactedSSTs{&backuppb.LogFileSubcompaction{ Meta: &backuppb.LogFileSubcompactionMeta{ TableId: tableID, }, @@ -1953,10 +1953,10 @@ func fakeSubCompactionWithMultiSsts(tableID, rowID int64, length uint64, num uin TotalKvs: num, }, }, - } + }} } -func fakeSubCompactionWithOneSst(tableID, rowID int64, length uint64, num uint64) *backuppb.LogFileSubcompaction { - return &backuppb.LogFileSubcompaction{ +func fakeSubCompactionWithOneSst(tableID, rowID int64, length uint64, num uint64) logclient.SSTs { + return &logclient.CompactedSSTs{&backuppb.LogFileSubcompaction{ Meta: &backuppb.LogFileSubcompactionMeta{ TableId: tableID, }, @@ -1969,7 +1969,7 @@ func fakeSubCompactionWithOneSst(tableID, rowID int64, length uint64, num uint64 TotalKvs: num, }, }, - } + }} } func fakeFile(tableID, rowID int64, length uint64, num int64) *backuppb.DataFileInfo { diff --git a/br/pkg/storage/locking.go b/br/pkg/storage/locking.go index 67ea7c2003081..faa3d015a228c 100644 --- a/br/pkg/storage/locking.go +++ b/br/pkg/storage/locking.go @@ -8,6 +8,7 @@ import ( "encoding/hex" "encoding/json" "fmt" + "math" "math/rand" "os" "path" @@ -18,6 +19,7 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/log" "github.com/pingcap/tidb/br/pkg/logutil" + "github.com/pingcap/tidb/br/pkg/utils" "go.uber.org/multierr" "go.uber.org/zap" ) @@ -253,6 +255,35 @@ func newReadLockName(path string) string { return fmt.Sprintf("%s.READ.%016x", path, readID) } +type Locker = func(ctx context.Context, storage ExternalStorage, path, hint string) (lock RemoteLock, err error) + +func LockWith(ctx context.Context, locker Locker, storage ExternalStorage, path, hint string) (lock RemoteLock, err error) { + const JitterMs = 5000 + + retry := utils.InitialRetryState(math.MaxInt, 1*time.Second, 60*time.Second) + jitter := time.Duration(rand.Uint32()%JitterMs+(JitterMs/2)) * time.Millisecond + for { + lock, err = locker(ctx, storage, path, hint) + if err == nil { + return lock, nil + } + retryAfter := retry.ExponentialBackoff() + jitter + log.Info( + "Encountered lock, will retry then.", + logutil.ShortError(err), + zap.String("path", path), + zap.Duration("retry-after", retryAfter), + ) + + select { + case <-ctx.Done(): + err = ctx.Err() + return + case <-time.After(retryAfter): + } + } +} + func TryLockRemoteWrite(ctx context.Context, storage ExternalStorage, path, hint string) (lock RemoteLock, err error) { target := writeLockName(path) writer := conditionalPut{ diff --git a/br/pkg/storage/s3.go b/br/pkg/storage/s3.go index 0158979411f1a..c1a13ff37892c 100644 --- a/br/pkg/storage/s3.go +++ b/br/pkg/storage/s3.go @@ -106,6 +106,8 @@ func (rs *S3Storage) CopyFrom(ctx context.Context, e ExternalStorage, spec CopyS Key: aws.String(rs.options.Prefix + spec.To), } + // NOTE: Maybe check whether the Go SDK will handle 200 OK errors. + // https://repost.aws/knowledge-center/s3-resolve-200-internalerror _, err := s.svc.CopyObjectWithContext(ctx, copyInput) return err } diff --git a/br/pkg/stream/stream_metas.go b/br/pkg/stream/stream_metas.go index 1909d300f242e..557c24e6aaf97 100644 --- a/br/pkg/stream/stream_metas.go +++ b/br/pkg/stream/stream_metas.go @@ -571,6 +571,11 @@ type Migrations struct { Layers []*OrderedMigration `json:"layers"` } +// GetReadLock locks the storage and make sure there won't be other one modify this backup. +func (m *MigrationExt) GetReadLock(ctx context.Context, hint string) (storage.RemoteLock, error) { + return storage.LockWith(ctx, storage.TryLockRemoteRead, m.s, lockPrefix, hint) +} + // OrderedMigration is a migration with its path and sequence number. type OrderedMigration struct { SeqNum int `json:"seq_num"` @@ -661,7 +666,7 @@ func (m MigrationExt) DryRun(f func(MigrationExt)) []storage.Effect { } func (m MigrationExt) AppendMigration(ctx context.Context, mig *pb.Migration) (int, error) { - lock, err := storage.TryLockRemoteWrite(ctx, m.s, lockPrefix, "AppendMigration") + lock, err := storage.LockWith(ctx, storage.TryLockRemoteWrite, m.s, lockPrefix, "AppendMigration") if err != nil { return 0, err } @@ -752,7 +757,7 @@ func (m MigrationExt) MergeAndMigrateTo( targetSpec int, opts ...MergeAndMigrateToOpt, ) (result MergeAndMigratedTo) { - lock, err := storage.TryLockRemoteWrite(ctx, m.s, lockPrefix, "AppendMigration") + lock, err := storage.LockWith(ctx, storage.TryLockRemoteWrite, m.s, lockPrefix, "AppendMigration") if err != nil { result.MigratedTo = MigratedTo{ Warnings: []error{ @@ -838,7 +843,7 @@ func (m MigrationExt) MergeAndMigrateTo( } } } - result.MigratedTo = m.MigrateTo(ctx, newBase, MTMaybeSkipTruncateLog(!config.alwaysRunTruncate && canSkipTruncate)) + result.MigratedTo = m.migrateTo(ctx, newBase, MTMaybeSkipTruncateLog(!config.alwaysRunTruncate && canSkipTruncate)) // Put the final BASE. err = m.writeBase(ctx, result.MigratedTo.NewBase) @@ -848,7 +853,7 @@ func (m MigrationExt) MergeAndMigrateTo( return } -type MigrateToOpt func(*migToOpt) +type migrateToOpt func(*migToOpt) type migToOpt struct { skipTruncateLog bool @@ -858,17 +863,17 @@ func MTSkipTruncateLog(o *migToOpt) { o.skipTruncateLog = true } -func MTMaybeSkipTruncateLog(cond bool) MigrateToOpt { +func MTMaybeSkipTruncateLog(cond bool) migrateToOpt { if cond { return MTSkipTruncateLog } return func(*migToOpt) {} } -// MigrateTo migrates to a migration. +// migrateTo migrates to a migration. // If encountered some error during executing some operation, the operation will be put // to the new BASE, which can be retryed then. -func (m MigrationExt) MigrateTo(ctx context.Context, mig *pb.Migration, opts ...MigrateToOpt) MigratedTo { +func (m MigrationExt) migrateTo(ctx context.Context, mig *pb.Migration, opts ...migrateToOpt) MigratedTo { opt := migToOpt{} for _, o := range opts { o(&opt) diff --git a/br/pkg/stream/stream_metas_test.go b/br/pkg/stream/stream_metas_test.go index 95dd645ec3d52..52f1d003e144f 100644 --- a/br/pkg/stream/stream_metas_test.go +++ b/br/pkg/stream/stream_metas_test.go @@ -2584,7 +2584,7 @@ func TestBasicMigration(t *testing.T) { requireMigrationsEqual(t, resE, res) ctx := context.Background() - mg := est.MigrateTo(ctx, res) + mg := est.migrateTo(ctx, res) newBaseE := mig(mLogDel("00002.meta", spans("00001.log", 1024, sp(0, 42), sp(42, 18)))) require.Empty(t, mg.Warnings) @@ -2600,7 +2600,7 @@ func TestBasicMigration(t *testing.T) { delRem := mig(mLogDel("00002.meta", spans("00001.log", 1024, sp(60, 1024-60)))) newNewBase := MergeMigrations(mg.NewBase, delRem) - mg = est.MigrateTo(ctx, newNewBase) + mg = est.migrateTo(ctx, newNewBase) require.Empty(t, mg.Warnings) requireMigrationsEqual(t, mg.NewBase, mig()) } @@ -2711,7 +2711,7 @@ func TestRemoveCompaction(t *testing.T) { mTruncatedTo(30), )) - mg := est.MigrateTo(ctx, merged) + mg := est.migrateTo(ctx, merged) requireMigrationsEqual(t, mg.NewBase, mig( mCompaction(cDir(1), aDir(1), 10, 40), mCompaction(cDir(2), aDir(2), 35, 50), @@ -2771,7 +2771,7 @@ func TestRetryRemoveCompaction(t *testing.T) { require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/br/pkg/storage/local_delete_file_err", `1*return("this disk will never forget")`)) est := MigrationExtension(s) - mg := est.MigrateTo(ctx, mig1) + mg := est.migrateTo(ctx, mig1) require.Len(t, mg.Warnings, 1) require.Error(t, mg.Warnings[0], "this disk will never forget") requireMigrationsEqual(t, mg.NewBase, mig( @@ -2780,7 +2780,7 @@ func TestRetryRemoveCompaction(t *testing.T) { mDstrPfx(cDir(1), aDir(1)), )) - mg = est.MigrateTo(ctx, mg.NewBase) + mg = est.migrateTo(ctx, mg.NewBase) require.Empty(t, mg.Warnings) requireMigrationsEqual(t, mg.NewBase, mig( mCompaction(placeholder(cDir(2)), placeholder(aDir(2)), 28, 32), @@ -2817,7 +2817,7 @@ func TestWithSimpleTruncate(t *testing.T) { est := MigrationExtension(s) m := mig(mTruncatedTo(65)) var res MigratedTo - effs := est.DryRun(func(me MigrationExt) { res = me.MigrateTo(ctx, m) }) + effs := est.DryRun(func(me MigrationExt) { res = me.migrateTo(ctx, m) }) require.Empty(t, res.Warnings) for _, eff := range effs { diff --git a/br/pkg/streamhelper/basic_lib_for_test.go b/br/pkg/streamhelper/basic_lib_for_test.go index 52ce89519a23a..62de80da774cd 100644 --- a/br/pkg/streamhelper/basic_lib_for_test.go +++ b/br/pkg/streamhelper/basic_lib_for_test.go @@ -181,6 +181,11 @@ func (f *fakeStore) GetID() uint64 { return f.id } +func (f *fakeStore) FlushNow(ctx context.Context, in *logbackup.FlushNowRequest, opts ...grpc.CallOption) (*logbackup.FlushNowResponse, error) { + f.flush() + return &logbackup.FlushNowResponse{Results: []*logbackup.FlushResult{{TaskName: "Universe", Success: true}}}, nil +} + func (f *fakeStore) SubscribeFlushEvent(ctx context.Context, in *logbackup.SubscribeFlushEventRequest, opts ...grpc.CallOption) (logbackup.LogBackup_SubscribeFlushEventClient, error) { f.clientMu.Lock() defer f.clientMu.Unlock() diff --git a/br/pkg/streamhelper/daemon/owner_daemon.go b/br/pkg/streamhelper/daemon/owner_daemon.go index 5956b643c971d..10b70a9a59580 100644 --- a/br/pkg/streamhelper/daemon/owner_daemon.go +++ b/br/pkg/streamhelper/daemon/owner_daemon.go @@ -89,7 +89,7 @@ func (od *OwnerDaemon) Begin(ctx context.Context) (func(), error) { zap.String("daemon-id", od.daemon.Name())) return case <-tick.C: - log.Debug("daemon tick start", + log.Info("daemon tick start", zap.Bool("is-owner", od.manager.IsOwner()), zap.String("daemon-id", od.daemon.Name())) if od.manager.IsOwner() { diff --git a/br/pkg/task/operator/config.go b/br/pkg/task/operator/config.go index c42382abe504d..d623c6964cf70 100644 --- a/br/pkg/task/operator/config.go +++ b/br/pkg/task/operator/config.go @@ -180,3 +180,14 @@ func (cfg *MigrateToConfig) Verify() error { } return nil } + +type ForceFlushConfig struct { + task.Config +} + +func DefineFlagsForForceFlushConfig(f *pflag.FlagSet) { +} + +func (cfg *ForceFlushConfig) ParseFromFlags(flags *pflag.FlagSet) error { + return cfg.Config.ParseFromFlags(flags) +} diff --git a/br/pkg/task/operator/force_flush.go b/br/pkg/task/operator/force_flush.go new file mode 100644 index 0000000000000..121ebdd9a24b8 --- /dev/null +++ b/br/pkg/task/operator/force_flush.go @@ -0,0 +1,71 @@ +package operator + +import ( + "context" + "slices" + + "github.com/pingcap/errors" + logbackup "github.com/pingcap/kvproto/pkg/logbackuppb" + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/pingcap/log" + "github.com/pingcap/tidb/pkg/util/engine" + pd "github.com/tikv/pd/client" + "go.uber.org/zap" + "golang.org/x/sync/errgroup" + "google.golang.org/grpc" +) + +func getAllTiKVs(ctx context.Context, p pd.Client) ([]*metapb.Store, error) { + stores, err := p.GetAllStores(ctx, pd.WithExcludeTombstone()) + if err != nil { + return nil, err + } + withoutTiFlash := slices.DeleteFunc(stores, engine.IsTiFlash) + return withoutTiFlash, err +} + +func RunForceFlush(ctx context.Context, cfg *ForceFlushConfig) error { + pdMgr, err := dialPD(ctx, &cfg.Config) + if err != nil { + return err + } + defer pdMgr.Close() + + stores, err := createStoreManager(pdMgr.GetPDClient(), &cfg.Config) + if err != nil { + return err + } + defer stores.Close() + + tikvs, err := getAllTiKVs(ctx, pdMgr.GetPDClient()) + if err != nil { + return err + } + eg, ectx := errgroup.WithContext(ctx) + for _, s := range tikvs { + s := s + log.Info("Starting force flush TiKV.", zap.Uint64("store", s.GetId())) + eg.Go(func() error { + var logBackupCli logbackup.LogBackupClient + err := stores.WithConn(ectx, s.GetId(), func(cc *grpc.ClientConn) { + logBackupCli = logbackup.NewLogBackupClient(cc) + }) + if err != nil { + return err + } + + resp, err := logBackupCli.FlushNow(ectx, &logbackup.FlushNowRequest{}) + if err != nil { + return errors.Annotatef(err, "failed to flush store %d", s.GetId()) + } + for _, res := range resp.Results { + if !res.Success { + return errors.Errorf("failed to flush task %s at store %d: %s", res.TaskName, s.GetId(), res.ErrorMessage) + } + log.Info("Force flushed task of TiKV store.", zap.Uint64("store", s.Id), zap.String("task", res.TaskName)) + } + return nil + }) + } + return eg.Wait() +} diff --git a/br/pkg/task/operator/prepare_snap.go b/br/pkg/task/operator/prepare_snap.go index cbe5c3ac2442b..50a624423240b 100644 --- a/br/pkg/task/operator/prepare_snap.go +++ b/br/pkg/task/operator/prepare_snap.go @@ -19,12 +19,32 @@ import ( "github.com/pingcap/tidb/br/pkg/task" "github.com/pingcap/tidb/br/pkg/utils" "github.com/tikv/client-go/v2/tikv" + pd "github.com/tikv/pd/client" "go.uber.org/multierr" "go.uber.org/zap" "golang.org/x/sync/errgroup" "google.golang.org/grpc/keepalive" ) +func createStoreManager(pd pd.Client, cfg *task.Config) (*utils.StoreManager, error) { + var ( + tconf *tls.Config + err error + ) + + if cfg.TLS.IsEnabled() { + tconf, err = cfg.TLS.ToTLSConfig() + if err != nil { + return nil, errors.Annotate(err, "invalid tls config") + } + } + kvMgr := utils.NewStoreManager(pd, keepalive.ClientParameters{ + Time: cfg.GRPCKeepaliveTime, + Timeout: cfg.GRPCKeepaliveTimeout, + }, tconf) + return kvMgr, nil +} + func dialPD(ctx context.Context, cfg *task.Config) (*pdutil.PdController, error) { var tc *tls.Config if cfg.TLS.IsEnabled() { diff --git a/br/pkg/task/stream.go b/br/pkg/task/stream.go index 0e8dd76aa9085..140de7c369da7 100644 --- a/br/pkg/task/stream.go +++ b/br/pkg/task/stream.go @@ -61,6 +61,7 @@ import ( "github.com/spf13/pflag" "github.com/tikv/client-go/v2/oracle" clientv3 "go.etcd.io/etcd/client/v3" + "go.uber.org/multierr" "go.uber.org/zap" "go.uber.org/zap/zapcore" ) @@ -1390,7 +1391,8 @@ func restoreStream( if err != nil { return errors.Trace(err) } - client.BuildMigrations(migs) + client.BuildMigrations(migs.Migs) + defer cleanUpWithRetErr(&err, migs.ReadLock.Unlock) // get full backup meta storage to generate rewrite rules. fullBackupStorage, err := parseFullBackupTablesStorage(cfg) @@ -1916,3 +1918,12 @@ func checkPiTRTaskInfo( return checkInfo, nil } + +func cleanUpWithRetErr(errOut *error, f func(ctx context.Context) error) { + ctx, cancel := context.WithTimeout(context.Background(), time.Minute) + defer cancel() + err := f(ctx) + if errOut != nil { + *errOut = multierr.Combine(*errOut, err) + } +} diff --git a/go.mod b/go.mod index 66e2baa3d01d2..e79ccf7f3434d 100644 --- a/go.mod +++ b/go.mod @@ -328,4 +328,4 @@ replace ( sourcegraph.com/sourcegraph/appdash-data => github.com/sourcegraph/appdash-data v0.0.0-20151005221446-73f23eafcf67 ) -replace github.com/pingcap/kvproto => github.com/yujuncen/kvproto v0.0.0-20241125070101-be2e12233a90 +replace github.com/pingcap/kvproto => github.com/yujuncen/kvproto v0.0.0-20241129083607-e6af9662093b diff --git a/go.sum b/go.sum index 0d69cdbee952a..35f3ae1849332 100644 --- a/go.sum +++ b/go.sum @@ -1731,8 +1731,8 @@ github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9dec github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= github.com/yuin/goldmark v1.4.1/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= github.com/yuin/goldmark v1.4.13/go.mod h1:6yULJ656Px+3vBD8DxQVa3kxgyrAnzto9xy5taEt/CY= -github.com/yujuncen/kvproto v0.0.0-20241125070101-be2e12233a90 h1:EyTrFe4KEIMHfvngHkyz0yb2EzuANIvFUSpVjAqkhu4= -github.com/yujuncen/kvproto v0.0.0-20241125070101-be2e12233a90/go.mod h1:rXxWk2UnwfUhLXha1jxRWPADw9eMZGWEWCg92Tgmb/8= +github.com/yujuncen/kvproto v0.0.0-20241129083607-e6af9662093b h1:/bt8UBEr/4azv8nCcLfzFwdgBUwBfooo81BAv0lELOs= +github.com/yujuncen/kvproto v0.0.0-20241129083607-e6af9662093b/go.mod h1:rXxWk2UnwfUhLXha1jxRWPADw9eMZGWEWCg92Tgmb/8= github.com/yusufpapurcu/wmi v1.2.2/go.mod h1:SBZ9tNy3G9/m5Oi98Zks0QjeHVDvuK0qfxQmPyzfmi0= github.com/yusufpapurcu/wmi v1.2.4 h1:zFUKzehAFReQwLys1b/iSMl+JQGSCSjtVqQn9bBrPo0= github.com/yusufpapurcu/wmi v1.2.4/go.mod h1:SBZ9tNy3G9/m5Oi98Zks0QjeHVDvuK0qfxQmPyzfmi0= diff --git a/tests/realtikvtest/brietest/main_test.go b/tests/realtikvtest/brietest/main_test.go index c8117a7615e96..1b8f820204249 100644 --- a/tests/realtikvtest/brietest/main_test.go +++ b/tests/realtikvtest/brietest/main_test.go @@ -29,7 +29,8 @@ func TestMain(m *testing.M) { goleak.IgnoreTopFunction("github.com/bazelbuild/rules_go/go/tools/bzltestutil.RegisterTimeoutHandler.func1"), goleak.IgnoreTopFunction("github.com/lestrrat-go/httprc.runFetchWorker"), goleak.IgnoreTopFunction("google.golang.org/grpc.(*ccBalancerWrapper).watcher"), - goleak.IgnoreTopFunction("google.golang.org/grpc/internal/transport.(*http2Client).keepalive"), + // The top function now is `sync.runtime_notifyListWait`... + goleak.IgnoreAnyFunction("google.golang.org/grpc/internal/transport.(*http2Client).keepalive"), goleak.IgnoreTopFunction("google.golang.org/grpc/internal/transport.(*controlBuffer).get"), goleak.IgnoreTopFunction("google.golang.org/grpc/internal/grpcsync.(*CallbackSerializer).run"), goleak.IgnoreTopFunction("net/http.(*persistConn).writeLoop"), diff --git a/tests/realtikvtest/brietest/pitr_test.go b/tests/realtikvtest/brietest/pitr_test.go index bb5a9c1cc6492..cf6475afb0830 100644 --- a/tests/realtikvtest/brietest/pitr_test.go +++ b/tests/realtikvtest/brietest/pitr_test.go @@ -8,18 +8,23 @@ import ( "testing" "time" + "github.com/pingcap/log" "github.com/pingcap/tidb/br/pkg/glue" "github.com/pingcap/tidb/br/pkg/gluetidb" "github.com/pingcap/tidb/br/pkg/streamhelper" "github.com/pingcap/tidb/br/pkg/task" + "github.com/pingcap/tidb/br/pkg/task/operator" + "github.com/pingcap/tidb/pkg/config" "github.com/pingcap/tidb/pkg/domain" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/util/printer" + filter "github.com/pingcap/tidb/pkg/util/table-filter" "github.com/stretchr/testify/require" "github.com/tikv/client-go/v2/oracle" "github.com/tikv/client-go/v2/tikv" pd "github.com/tikv/pd/client" + "go.uber.org/zap" ) type TestKitGlue struct { @@ -137,6 +142,8 @@ func (kit *LogBackupKit) RunLogStart(taskName string, extConfig func(*task.Strea cfg.Storage = "local://" + kit.base + "/incr" cfg.TaskName = taskName cfg.EndTS = math.MaxUint64 + cfg.TableFilter = filter.All() + cfg.FilterStr = []string{"*.*"} extConfig(&cfg) err := task.RunStreamStart(ctx, kit.Glue(), "stream start[intest]", &cfg) return err @@ -173,6 +180,27 @@ func (kit *LogBackupKit) mustExec(f func(context.Context) error) { require.NoError(kit.t, err) } +func (kit *LogBackupKit) forceFlush() { + kit.mustExec(func(ctx context.Context) error { + cfg := task.DefaultConfig() + cfg.PD = append(cfg.PD, config.GetGlobalConfig().Path) + return operator.RunForceFlush(ctx, &operator.ForceFlushConfig{ + Config: cfg, + }) + }) +} + +func (kit *LogBackupKit) forceFlushAndWait(taskName string) { + ts := kit.TSO() + kit.forceFlush() + require.Eventually(kit.t, func() bool { + ckpt := kit.CheckpointTSOf(taskName) + log.Info("checkpoint", zap.Uint64("checkpoint", ckpt), zap.Uint64("ts", ts)) + return ckpt >= ts + }, 21*time.Second, 1*time.Second) + time.Sleep(6 * time.Second) // Wait the storage checkpoint uploaded... +} + func createSimpleTableWithData(kit *LogBackupKit) { kit.tk.MustExec(fmt.Sprintf("DROP TABLE IF EXISTs test.%s", kit.t.Name())) kit.tk.MustExec(fmt.Sprintf("CREATE TABLE test.%s(t text)", kit.t.Name())) @@ -204,11 +232,7 @@ func TestPiTR(t *testing.T) { insertSimpleIncreaseData(kit) - ts = kit.TSO() - require.Eventually(t, func() bool { - return kit.CheckpointTSOf(taskName) >= ts - }, 300*time.Second, 10*time.Second) - + kit.forceFlushAndWait(taskName) cleanSimpleData(kit) kit.StopTaskIfExists(taskName) @@ -236,11 +260,7 @@ func TestPiTRAndBackup(t *testing.T) { }) kit.RunFullRestore(func(rc *task.RestoreConfig) {}) - ts = kit.TSO() - require.Eventually(t, func() bool { - return kit.CheckpointTSOf(taskName) >= ts - }, 300*time.Second, 10*time.Second) - + kit.forceFlushAndWait(taskName) cleanSimpleData(kit) kit.RunStreamRestore(func(rc *task.RestoreConfig) { rc.FullBackupStorage = "local://" + kit.base + "/full2" From ec98b3edc108da3810e5e5e203379a529fc10850 Mon Sep 17 00:00:00 2001 From: hillium Date: Wed, 4 Dec 2024 17:02:02 +0800 Subject: [PATCH 07/36] added checksum table --- br/cmd/br/operator.go | 18 ++ br/pkg/task/operator/checksum_table.go | 258 +++++++++++++++++++++++++ br/pkg/task/operator/config.go | 11 ++ 3 files changed, 287 insertions(+) create mode 100644 br/pkg/task/operator/checksum_table.go diff --git a/br/cmd/br/operator.go b/br/cmd/br/operator.go index 4e41adeab329f..d7f08f4c25dc2 100644 --- a/br/cmd/br/operator.go +++ b/br/cmd/br/operator.go @@ -35,6 +35,7 @@ func newOperatorCommand() *cobra.Command { cmd.AddCommand(newBase64ifyCommand()) cmd.AddCommand(newListMigrationsCommand()) cmd.AddCommand(newMigrateToCommand()) + cmd.AddCommand(newChecksumCommand()) return cmd } @@ -109,3 +110,20 @@ func newMigrateToCommand() *cobra.Command { operator.DefineFlagsForMigrateToConfig(cmd.Flags()) return cmd } + +func newChecksumCommand() *cobra.Command { + cmd := &cobra.Command{ + Use: "checksum ", + Short: "utilities for checksumming.", + Args: cobra.NoArgs, + RunE: func(cmd *cobra.Command, args []string) error { + cfg := operator.ChecksumTableConfig{} + if err := cfg.ParseFromFlags(cmd.Flags()); err != nil { + return err + } + ctx := GetDefaultContext() + return operator.RunChecksumTable(ctx, tidbGlue, cfg) + }, + } + return cmd +} diff --git a/br/pkg/task/operator/checksum_table.go b/br/pkg/task/operator/checksum_table.go new file mode 100644 index 0000000000000..a838ceb52d014 --- /dev/null +++ b/br/pkg/task/operator/checksum_table.go @@ -0,0 +1,258 @@ +package operator + +import ( + "context" + "encoding/json" + "os" + "sync" + "sync/atomic" + + kvutil "github.com/tikv/client-go/v2/util" + "golang.org/x/sync/errgroup" + + "github.com/pingcap/errors" + backup "github.com/pingcap/kvproto/pkg/brpb" + "github.com/pingcap/log" + "github.com/pingcap/tidb/br/pkg/checksum" + "github.com/pingcap/tidb/br/pkg/conn" + "github.com/pingcap/tidb/br/pkg/glue" + "github.com/pingcap/tidb/br/pkg/metautil" + "github.com/pingcap/tidb/br/pkg/task" + "github.com/pingcap/tidb/pkg/domain" + "github.com/pingcap/tidb/pkg/meta/model" + "github.com/pingcap/tidb/pkg/util" + "github.com/tikv/client-go/v2/oracle" + "go.uber.org/zap" +) + +type checksumTableCtx struct { + cfg ChecksumTableConfig + + mgr *conn.Mgr + dom *domain.Domain +} + +type tableInDB struct { + info *model.TableInfo + dbName string +} + +func RunChecksumTable(ctx context.Context, g glue.Glue, cfg ChecksumTableConfig) error { + c := &checksumTableCtx{cfg: cfg} + + if err := c.init(ctx, g); err != nil { + return errors.Trace(err) + } + + curr, err := c.getTables(ctx) + if err != nil { + return errors.Trace(err) + } + + old, err := c.loadOldTableIDs(ctx) + if err != nil { + return errors.Trace(err) + } + + reqs, err := c.genRequests(ctx, old, curr) + if err != nil { + return errors.Trace(err) + } + + results, err := c.runChecksum(ctx, reqs) + if err != nil { + return errors.Trace(err) + } + + for _, result := range results { + log.Info("Checksum result", zap.String("db", result.DBName), zap.String("table", result.TableName), zap.Uint64("checksum", result.Checksum), + zap.Uint64("total_bytes", result.TotalBytes), zap.Uint64("total_kvs", result.TotalKVs)) + } + + return json.NewEncoder(os.Stdout).Encode(results) +} + +func (c *checksumTableCtx) init(ctx context.Context, g glue.Glue) error { + cfg := c.cfg + var err error + c.mgr, err = task.NewMgr(ctx, g, cfg.PD, cfg.TLS, task.GetKeepalive(&cfg.Config), cfg.CheckRequirements, true, conn.NormalVersionChecker) + if err != nil { + return err + } + + c.dom, err = g.GetDomain(c.mgr.GetStorage()) + if err != nil { + return err + } + return nil +} + +func (c *checksumTableCtx) getTables(ctx context.Context) (res []tableInDB, err error) { + sch := c.dom.InfoSchema() + dbs := sch.AllSchemas() + for _, db := range dbs { + if !c.cfg.TableFilter.MatchSchema(db.Name.L) { + continue + } + + tbls, err := sch.SchemaTableInfos(ctx, db.Name) + if err != nil { + return nil, errors.Annotatef(err, "failed to load data for db %s", db.Name) + } + for _, tbl := range tbls { + if !c.cfg.TableFilter.MatchTable(db.Name.L, tbl.Name.L) { + continue + } + log.Info("Added table from cluster.", zap.String("db", db.Name.L), zap.String("table", tbl.Name.L)) + res = append(res, tableInDB{ + info: tbl, + dbName: db.Name.L, + }) + } + } + + return +} + +func (c *checksumTableCtx) loadOldTableIDs(ctx context.Context) (res []*metautil.Table, err error) { + _, strg, err := task.GetStorage(ctx, c.cfg.Storage, &c.cfg.Config) + if err != nil { + return nil, errors.Annotate(err, "failed to create storage") + } + + mPath := metautil.MetaFile + metaContent, err := strg.ReadFile(ctx, mPath) + if err != nil { + return nil, errors.Annotatef(err, "failed to open metafile %s", mPath) + } + + var backupMeta backup.BackupMeta + if err := backupMeta.Unmarshal(metaContent); err != nil { + return nil, errors.Annotate(err, "failed to parse backupmeta") + } + + metaReader := metautil.NewMetaReader(&backupMeta, strg, &c.cfg.CipherInfo) + + tblCh := make(chan *metautil.Table) + if err := metaReader.ReadSchemasFiles(ctx, tblCh, metautil.SkipFiles, metautil.SkipStats); err != nil { + return nil, errors.Annotate(err, "failed to read schema files") + } + + for { + select { + case tbl, ok := <-tblCh: + if !ok { + return + } + if !c.cfg.TableFilter.MatchTable(tbl.DB.Name.L, tbl.Info.Name.L) { + continue + } + log.Info("Added table from backup data.", zap.String("db", tbl.DB.Name.L), zap.String("table", tbl.Info.Name.L)) + res = append(res, tbl) + case <-ctx.Done(): + return nil, ctx.Err() + } + } +} + +type request struct { + copReq *checksum.Executor + tableName string + dbName string +} + +func (c *checksumTableCtx) genRequests(ctx context.Context, bkup []*metautil.Table, curr []tableInDB) (reqs []request, err error) { + phy, logi, err := c.mgr.GetPDClient().GetTS(ctx) + if err != nil { + return nil, errors.Annotate(err, "failed to get TSO for checksumming") + } + tso := oracle.ComposeTS(phy, logi) + + bkupTbls := map[string]map[string]*metautil.Table{} + for _, t := range bkup { + m, ok := bkupTbls[t.DB.Name.L] + if !ok { + m = make(map[string]*metautil.Table) + bkupTbls[t.DB.Name.L] = m + } + + m[t.Info.Name.L] = t + } + + for _, t := range curr { + rb := checksum.NewExecutorBuilder(t.info, tso) + rb.SetConcurrency(c.cfg.ChecksumConcurrency) + oldDB, ok := bkupTbls[t.dbName] + if !ok { + log.Warn("db not found, will skip", zap.String("db", t.dbName)) + continue + } + oldTable, ok := oldDB[t.info.Name.L] + if !ok { + log.Warn("table not found, will skip", zap.String("db", t.dbName), zap.String("table", t.info.Name.L)) + continue + } + + rb.SetOldTable(oldTable) + rb.SetExplicitRequestSourceType(kvutil.ExplicitTypeBR) + req, err := rb.Build() + if err != nil { + return nil, errors.Annotatef(err, "failed to build checksum builder for table %s.%s", t.dbName, t.info.Name.L) + } + reqs = append(reqs, request{ + copReq: req, + dbName: t.dbName, + tableName: t.info.Name.L, + }) + } + + return +} + +type ChecksumResult struct { + DBName string `json:"db_name"` + TableName string `json:"table_name"` + + Checksum uint64 `json:"checksum"` + TotalBytes uint64 `json:"total_bytes"` + TotalKVs uint64 `json:"total_kvs"` +} + +func (c *checksumTableCtx) runChecksum(ctx context.Context, reqs []request) ([]ChecksumResult, error) { + wkPool := util.NewWorkerPool(c.cfg.TableConcurrency, "checksum") + eg, ectx := errgroup.WithContext(ctx) + results := make([]ChecksumResult, 0, len(reqs)) + resultsMu := new(sync.Mutex) + + for _, req := range reqs { + req := req + wkPool.ApplyOnErrorGroup(eg, func() error { + total := req.copReq.Len() + finished := new(atomic.Int64) + resp, err := req.copReq.Execute(ectx, c.mgr.GetStorage().GetClient(), func() { + finished.Add(1) + log.Info( + "Finish one request of a table.", + zap.String("db", req.dbName), + zap.String("table", req.tableName), + zap.Int64("finished", finished.Load()), + zap.Int64("total", int64(total)), + ) + }) + res := ChecksumResult{ + DBName: req.dbName, + TableName: req.tableName, + + Checksum: resp.Checksum, + TotalBytes: resp.TotalBytes, + TotalKVs: resp.TotalKvs, + } + resultsMu.Lock() + results = append(results, res) + resultsMu.Unlock() + return err + }) + } + + return results, nil +} diff --git a/br/pkg/task/operator/config.go b/br/pkg/task/operator/config.go index d623c6964cf70..3b34f7f534683 100644 --- a/br/pkg/task/operator/config.go +++ b/br/pkg/task/operator/config.go @@ -191,3 +191,14 @@ func DefineFlagsForForceFlushConfig(f *pflag.FlagSet) { func (cfg *ForceFlushConfig) ParseFromFlags(flags *pflag.FlagSet) error { return cfg.Config.ParseFromFlags(flags) } + +type ChecksumTableConfig struct { + task.Config +} + +func DefineFlagsForChecksumTableConfig(f *pflag.FlagSet) { +} + +func (cfg *ChecksumTableConfig) ParseFromFlags(flags *pflag.FlagSet) error { + return cfg.Config.ParseFromFlags(flags) +} From 2dfc800338236570a04249d862a6fae759582c3e Mon Sep 17 00:00:00 2001 From: hillium Date: Wed, 4 Dec 2024 18:14:10 +0800 Subject: [PATCH 08/36] make checksum work Signed-off-by: hillium --- br/cmd/br/operator.go | 2 ++ br/pkg/task/operator/checksum_table.go | 18 ++++++++++++++---- br/pkg/task/operator/config.go | 13 ++++++++++++- 3 files changed, 28 insertions(+), 5 deletions(-) diff --git a/br/cmd/br/operator.go b/br/cmd/br/operator.go index d7f08f4c25dc2..ab8adb1a859b4 100644 --- a/br/cmd/br/operator.go +++ b/br/cmd/br/operator.go @@ -125,5 +125,7 @@ func newChecksumCommand() *cobra.Command { return operator.RunChecksumTable(ctx, tidbGlue, cfg) }, } + task.DefineFilterFlags(cmd, []string{"!*.*"}, false) + operator.DefineFlagsForChecksumTableConfig(cmd.Flags()) return cmd } diff --git a/br/pkg/task/operator/checksum_table.go b/br/pkg/task/operator/checksum_table.go index a838ceb52d014..19f0179810ea2 100644 --- a/br/pkg/task/operator/checksum_table.go +++ b/br/pkg/task/operator/checksum_table.go @@ -133,13 +133,19 @@ func (c *checksumTableCtx) loadOldTableIDs(ctx context.Context) (res []*metautil metaReader := metautil.NewMetaReader(&backupMeta, strg, &c.cfg.CipherInfo) - tblCh := make(chan *metautil.Table) - if err := metaReader.ReadSchemasFiles(ctx, tblCh, metautil.SkipFiles, metautil.SkipStats); err != nil { - return nil, errors.Annotate(err, "failed to read schema files") - } + tblCh := make(chan *metautil.Table, 1024) + errCh := make(chan error, 1) + go func() { + if err := metaReader.ReadSchemasFiles(ctx, tblCh, metautil.SkipFiles, metautil.SkipStats); err != nil { + errCh <- errors.Annotate(err, "failed to read schema files") + } + close(tblCh) + }() for { select { + case err := <-errCh: + return nil, err case tbl, ok := <-tblCh: if !ok { return @@ -254,5 +260,9 @@ func (c *checksumTableCtx) runChecksum(ctx context.Context, reqs []request) ([]C }) } + if err := eg.Wait(); err != nil { + return nil, err + } + return results, nil } diff --git a/br/pkg/task/operator/config.go b/br/pkg/task/operator/config.go index 3b34f7f534683..882279a8a0bc9 100644 --- a/br/pkg/task/operator/config.go +++ b/br/pkg/task/operator/config.go @@ -6,12 +6,17 @@ import ( "time" "github.com/pingcap/errors" + "github.com/pingcap/tidb/br/pkg/backup" berrors "github.com/pingcap/tidb/br/pkg/errors" "github.com/pingcap/tidb/br/pkg/storage" "github.com/pingcap/tidb/br/pkg/task" "github.com/spf13/pflag" ) +const ( + flagTableConcurrency = "table-concurrency" +) + type PauseGcConfig struct { task.Config @@ -197,8 +202,14 @@ type ChecksumTableConfig struct { } func DefineFlagsForChecksumTableConfig(f *pflag.FlagSet) { + f.Uint(flagTableConcurrency, backup.DefaultSchemaConcurrency, "The size of a BR thread pool used for backup table metas, "+ + "including tableInfo/checksum and stats.") } -func (cfg *ChecksumTableConfig) ParseFromFlags(flags *pflag.FlagSet) error { +func (cfg *ChecksumTableConfig) ParseFromFlags(flags *pflag.FlagSet) (err error) { + cfg.TableConcurrency, err = flags.GetUint(flagTableConcurrency) + if err != nil { + return + } return cfg.Config.ParseFromFlags(flags) } From c16d21b5c4bb4d47f232685eeae8b3678bc3fbb3 Mon Sep 17 00:00:00 2001 From: yujuncen Date: Fri, 6 Dec 2024 11:45:46 +0800 Subject: [PATCH 09/36] adapt the new proto --- br/pkg/checkpoint/restore.go | 3 + br/pkg/restore/snap_client/client.go | 9 +++ br/pkg/restore/snap_client/pitr_collector.go | 82 +++++++++++++++----- br/pkg/task/restore.go | 18 ++--- go.mod | 2 +- go.sum | 4 +- 6 files changed, 87 insertions(+), 31 deletions(-) diff --git a/br/pkg/checkpoint/restore.go b/br/pkg/checkpoint/restore.go index 88ff6f8f204de..4fcac0beddfce 100644 --- a/br/pkg/checkpoint/restore.go +++ b/br/pkg/checkpoint/restore.go @@ -19,6 +19,7 @@ import ( "encoding/json" "time" + "github.com/google/uuid" "github.com/pingcap/errors" "github.com/pingcap/tidb/br/pkg/glue" "github.com/pingcap/tidb/br/pkg/pdutil" @@ -107,6 +108,8 @@ type CheckpointMetadataForSnapshotRestore struct { UpstreamClusterID uint64 `json:"upstream-cluster-id"` RestoredTS uint64 `json:"restored-ts"` SchedulersConfig *pdutil.ClusterConfig `json:"schedulers-config"` + + RestoreUUID uuid.UUID `json:"restore-uuid"` } func LoadCheckpointMetadataForSnapshotRestore( diff --git a/br/pkg/restore/snap_client/client.go b/br/pkg/restore/snap_client/client.go index 0bcc03c5d85c0..5d95b500220e7 100644 --- a/br/pkg/restore/snap_client/client.go +++ b/br/pkg/restore/snap_client/client.go @@ -25,6 +25,7 @@ import ( "sync" "time" + "github.com/google/uuid" "github.com/opentracing/opentracing-go" "github.com/pingcap/errors" "github.com/pingcap/failpoint" @@ -152,6 +153,9 @@ type SnapClient struct { checkpointChecksum map[int64]*checkpoint.ChecksumItem pitrColl *pitrCollector + // restoreUUID is the UUID of this restore. + // restore from a checkpoint inherits the same restoreUUID. + restoreUUID uuid.UUID } // NewRestoreClient returns a new RestoreClient. @@ -323,6 +327,7 @@ func (rc *SnapClient) InitCheckpoint( if err != nil { return checkpointSetWithTableID, nil, errors.Trace(err) } + rc.restoreUUID = meta.RestoreUUID if meta.UpstreamClusterID != rc.backupMeta.ClusterId { return checkpointSetWithTableID, nil, errors.Errorf( @@ -374,10 +379,13 @@ func (rc *SnapClient) InitCheckpoint( } } else { // initialize the checkpoint metadata since it is the first time to restore. + restoreID := uuid.New() meta := &checkpoint.CheckpointMetadataForSnapshotRestore{ UpstreamClusterID: rc.backupMeta.ClusterId, RestoredTS: rc.backupMeta.EndVersion, + RestoreUUID: restoreID, } + rc.restoreUUID = restoreID // a nil config means undo function if config != nil { meta.SchedulersConfig = &pdutil.ClusterConfig{Schedulers: config.Schedulers, ScheduleCfg: config.ScheduleCfg} @@ -422,6 +430,7 @@ func (rc *SnapClient) InstallPiTRSupport(ctx context.Context, deps PiTRCollDep) return errors.Trace(err) } rc.pitrColl = collector + collector.restoreUUID = rc.restoreUUID rc.restorer = restore.Dual(rc.pitrColl.createRestorer(ctx), rc.restorer) return nil } diff --git a/br/pkg/restore/snap_client/pitr_collector.go b/br/pkg/restore/snap_client/pitr_collector.go index 9284606e65aef..be213879bcd4a 100644 --- a/br/pkg/restore/snap_client/pitr_collector.go +++ b/br/pkg/restore/snap_client/pitr_collector.go @@ -6,6 +6,7 @@ import ( "path/filepath" "sync" + "github.com/google/uuid" "github.com/pingcap/errors" pb "github.com/pingcap/kvproto/pkg/brpb" berrors "github.com/pingcap/tidb/br/pkg/errors" @@ -45,6 +46,10 @@ func (c *pitrCollector) createRestorer(ctx context.Context) *pitrCollectorRestor // GoRestore imports the specified backup file sets into TiKV asynchronously. // The onProgress function is called with progress updates as files are processed. func (p pitrCollectorRestorer) GoRestore(onProgress func(int64), batchFileSets ...restore.BatchBackupFileSet) error { + if !p.coll.enabled { + return nil + } + p.wg.Go(func() error { for _, fileSets := range batchFileSets { for _, fileSet := range fileSets { @@ -67,12 +72,22 @@ func (p pitrCollectorRestorer) GoRestore(onProgress func(int64), batchFileSets . // WaitUntilFinish blocks until all pending restore files have completed processing. func (p pitrCollectorRestorer) WaitUntilFinish() error { - return errors.Annotate(p.wg.Wait(), "failed to wait on pitrCollector") + if !p.coll.enabled { + return nil + } + err := p.wg.Wait() + if err != nil { + return errors.Annotate(err, "failed to wait on pitrCollector") + } + return errors.Annotatef(p.coll.persist(p.cx), "failed to persist the metadata of uploaded SSTs") } // Close releases any resources associated with the restoration process. func (p pitrCollectorRestorer) Close() error { - return errors.Annotate(p.coll.Commit(p.cx), "failed to commit pitrCollector") + if !p.coll.enabled { + return nil + } + return errors.Annotate(p.coll.commit(p.cx), "failed to commit pitrCollector") } type pitrCollector struct { @@ -81,6 +96,7 @@ type pitrCollector struct { restoreStorage storage.ExternalStorage name string enabled bool + restoreUUID uuid.UUID // Mutable state. committing committing @@ -95,6 +111,10 @@ type committing struct { rewrites map[int64]int64 } +func (c *committing) commit() { + c.msg.Finished = true +} + func (c *committing) genMsg() *pb.ExtraFullBackup { msg := util.ProtoV1Clone(&c.msg) for old, new := range c.rewrites { @@ -115,6 +135,14 @@ func (c *pitrCollector) outputPath(segs ...string) string { return filepath.Join(append([]string{"v1", "ext_backups", c.name}, segs...)...) } +func (c *pitrCollector) metaPath() string { + return c.outputPath("extbackupmeta") +} + +func (c *pitrCollector) sstPath(name string) string { + return c.outputPath("sst_files", name) +} + // PutSST records an SST file. func (c *pitrCollector) PutSST(ctx context.Context, f *pb.File) error { if !c.enabled { @@ -122,7 +150,7 @@ func (c *pitrCollector) PutSST(ctx context.Context, f *pb.File) error { } f = util.ProtoV1Clone(f) - out := c.outputPath(f.GetName()) + out := c.sstPath(f.Name) copier, ok := c.taskStorage.(storage.Copier) if !ok { @@ -163,42 +191,58 @@ func (c *pitrCollector) PutRewriteRule(_ context.Context, oldID int64, newID int return err } +func (c *pitrCollector) persist(ctx context.Context) (err error) { + c.committingLock.Lock() + defer c.committingLock.Unlock() + + msg := c.committing.genMsg() + bs, err := msg.Marshal() + if err != nil { + return errors.Annotate(err, "failed to marsal the committing message") + } + err = c.taskStorage.WriteFile(ctx, c.metaPath(), bs) + if err != nil { + return errors.Annotatef(err, "failed to put content to meta to %s", c.metaPath()) + } + return nil +} + // Commit commits the collected SSTs to a migration. -func (c *pitrCollector) Commit(ctx context.Context) error { +func (c *pitrCollector) prepareMig(ctx context.Context) error { if !c.enabled { return nil } est := stream.MigrationExtension(c.taskStorage) - m := stream.NewMigration() - var msg *pb.ExtraFullBackup - tso, err := c.tso(ctx) - if err != nil { - return errors.Trace(err) - } - c.doWithCommittingLock(func() { - msg = c.committing.genMsg() - c.committing.msg.AsIfTs = tso - }) - m.ExtraFullBackups = append(m.ExtraFullBackups, msg) + m := stream.NewMigration() + m.ExtraFullBackupPaths = append(m.ExtraFullBackupPaths, c.metaPath()) - _, err = est.AppendMigration(ctx, m) + _, err := est.AppendMigration(ctx, m) if err != nil { - return errors.Trace(err) + return errors.Annotatef(err, "failed to add the extra backup at path %s", c.metaPath()) } c.doWithCommittingLock(func() { c.resetCommitting() }) - return nil + // Persist the metadata in case of SSTs were uploaded but the meta wasn't, + // which leads to a leakage. + return c.persist(ctx) +} + +func (c *pitrCollector) commit(ctx context.Context) error { + c.committing.commit() + return c.persist(ctx) } func (c *pitrCollector) resetCommitting() { c.committing = committing{ rewrites: map[int64]int64{}, } - c.committing.msg.FilesPrefixHint = c.outputPath() + c.committing.msg.FilesPrefixHint = c.sstPath("") + c.committing.msg.Finished = false + c.committing.msg.BackupUuid = c.restoreUUID[:] } // PiTRCollDep is the dependencies of a PiTR collector. diff --git a/br/pkg/task/restore.go b/br/pkg/task/restore.go index bf66bd2d6234c..1d696d05ceb5e 100644 --- a/br/pkg/task/restore.go +++ b/br/pkg/task/restore.go @@ -816,15 +816,6 @@ func runSnapshotRestore(c context.Context, mgr *conn.Mgr, g glue.Glue, cmdName s return errors.Trace(err) } - err = client.InstallPiTRSupport(ctx, snapclient.PiTRCollDep{ - PDCli: mgr.GetPDClient(), - EtcdCli: mgr.GetDomain().GetEtcdClient(), - Storage: util.ProtoV1Clone(u), - }) - if err != nil { - return errors.Trace(err) - } - if client.IsRawKvMode() { return errors.Annotate(berrors.ErrRestoreModeMismatch, "cannot do transactional restore from raw kv data") } @@ -942,6 +933,15 @@ func runSnapshotRestore(c context.Context, mgr *conn.Mgr, g glue.Glue, cmdName s }() } + err = client.InstallPiTRSupport(ctx, snapclient.PiTRCollDep{ + PDCli: mgr.GetPDClient(), + EtcdCli: mgr.GetDomain().GetEtcdClient(), + Storage: util.ProtoV1Clone(u), + }) + if err != nil { + return errors.Trace(err) + } + sp := utils.BRServiceSafePoint{ BackupTS: restoreTS, TTL: utils.DefaultBRGCSafePointTTL, diff --git a/go.mod b/go.mod index e79ccf7f3434d..b3cc06c285bb7 100644 --- a/go.mod +++ b/go.mod @@ -328,4 +328,4 @@ replace ( sourcegraph.com/sourcegraph/appdash-data => github.com/sourcegraph/appdash-data v0.0.0-20151005221446-73f23eafcf67 ) -replace github.com/pingcap/kvproto => github.com/yujuncen/kvproto v0.0.0-20241129083607-e6af9662093b +replace github.com/pingcap/kvproto => github.com/yujuncen/kvproto v0.0.0-20241205083059-b7173611eaba diff --git a/go.sum b/go.sum index 35f3ae1849332..68c4b978585a2 100644 --- a/go.sum +++ b/go.sum @@ -1731,8 +1731,8 @@ github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9dec github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= github.com/yuin/goldmark v1.4.1/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= github.com/yuin/goldmark v1.4.13/go.mod h1:6yULJ656Px+3vBD8DxQVa3kxgyrAnzto9xy5taEt/CY= -github.com/yujuncen/kvproto v0.0.0-20241129083607-e6af9662093b h1:/bt8UBEr/4azv8nCcLfzFwdgBUwBfooo81BAv0lELOs= -github.com/yujuncen/kvproto v0.0.0-20241129083607-e6af9662093b/go.mod h1:rXxWk2UnwfUhLXha1jxRWPADw9eMZGWEWCg92Tgmb/8= +github.com/yujuncen/kvproto v0.0.0-20241205083059-b7173611eaba h1:JvHRr6mX+4XtBBQOY3i58tSfui3obXpiN5iyH7tN6mA= +github.com/yujuncen/kvproto v0.0.0-20241205083059-b7173611eaba/go.mod h1:rXxWk2UnwfUhLXha1jxRWPADw9eMZGWEWCg92Tgmb/8= github.com/yusufpapurcu/wmi v1.2.2/go.mod h1:SBZ9tNy3G9/m5Oi98Zks0QjeHVDvuK0qfxQmPyzfmi0= github.com/yusufpapurcu/wmi v1.2.4 h1:zFUKzehAFReQwLys1b/iSMl+JQGSCSjtVqQn9bBrPo0= github.com/yusufpapurcu/wmi v1.2.4/go.mod h1:SBZ9tNy3G9/m5Oi98Zks0QjeHVDvuK0qfxQmPyzfmi0= From d02c216dc610b283c8917c38e673a0d242e37364 Mon Sep 17 00:00:00 2001 From: yujuncen Date: Fri, 6 Dec 2024 12:23:55 +0800 Subject: [PATCH 10/36] make it compile (but not work) --- br/pkg/restore/log_client/migration.go | 30 +++++++++++++++++--- br/pkg/stream/stream_metas.go | 36 +++++++----------------- tests/realtikvtest/brietest/pitr_test.go | 10 +++++-- 3 files changed, 44 insertions(+), 32 deletions(-) diff --git a/br/pkg/restore/log_client/migration.go b/br/pkg/restore/log_client/migration.go index 059115d134286..4b6813486b6da 100644 --- a/br/pkg/restore/log_client/migration.go +++ b/br/pkg/restore/log_client/migration.go @@ -17,6 +17,7 @@ package logclient import ( "context" + "github.com/pingcap/errors" backuppb "github.com/pingcap/kvproto/pkg/brpb" "github.com/pingcap/tidb/br/pkg/storage" "github.com/pingcap/tidb/br/pkg/utils/iter" @@ -144,7 +145,7 @@ func (builder *WithMigrationsBuilder) coarseGrainedFilter(mig *backuppb.Migratio func (builder *WithMigrationsBuilder) Build(migs []*backuppb.Migration) WithMigrations { skipmap := make(metaSkipMap) compactionDirs := make([]string, 0, 8) - fullBackups := make([]*backuppb.ExtraFullBackup, 0, 8) + fullBackups := make([]string, 0, 8) for _, mig := range migs { // TODO: deal with TruncatedTo and DestructPrefix @@ -157,7 +158,7 @@ func (builder *WithMigrationsBuilder) Build(migs []*backuppb.Migration) WithMigr compactionDirs = append(compactionDirs, c.Artifacts) } - fullBackups = append(fullBackups, mig.ExtraFullBackups...) + fullBackups = append(fullBackups, mig.ExtraFullBackupPaths...) } withMigrations := WithMigrations{ skipmap: skipmap, @@ -214,7 +215,7 @@ func (mwm *MetaWithMigrations) Physicals(groupIndexIter GroupIndexIter) Physical type WithMigrations struct { skipmap metaSkipMap compactionDirs []string - fullBackups []*backuppb.ExtraFullBackup + fullBackups []string } func (wm *WithMigrations) Metas(metaNameIter MetaNameIter) MetaMigrationsIter { @@ -245,5 +246,26 @@ func (wm *WithMigrations) Compactions(ctx context.Context, s storage.ExternalSto } func (wm *WithMigrations) ExtraFullBackups(ctx context.Context, s storage.ExternalStorage) iter.TryNextor[*backuppb.ExtraFullBackup] { - return iter.FromSlice(wm.fullBackups) + fullBackupDirIter := iter.FromSlice(wm.fullBackups) + return iter.TryMap(fullBackupDirIter, func(name string) (*backuppb.ExtraFullBackup, error) { + // name is the absolute path in external storage. + bkup, err := readExtraFullBackup(ctx, name, s) + if err != nil { + return nil, errors.Annotatef(err, "failed to read backup at %s", name) + } + return bkup, nil + }) +} + +func readExtraFullBackup(ctx context.Context, name string, s storage.ExternalStorage) (*backuppb.ExtraFullBackup, error) { + reader, err := s.ReadFile(ctx, name) + if err != nil { + return nil, err + } + + var backup backuppb.ExtraFullBackup + if err := backup.Unmarshal(reader); err != nil { + return nil, err + } + return &backup, nil } diff --git a/br/pkg/stream/stream_metas.go b/br/pkg/stream/stream_metas.go index 557c24e6aaf97..4f1c30c9ff317 100644 --- a/br/pkg/stream/stream_metas.go +++ b/br/pkg/stream/stream_metas.go @@ -374,8 +374,8 @@ func AddMigrationToTable(m *pb.Migration, table *glue.Table) { for i, c := range m.DestructPrefix { table.Add(fmt.Sprintf("destruct-prefix[%02d]", i), rd(c)) } - for i, c := range m.ExtraFullBackups { - table.Add(fmt.Sprintf("extra_full_backups[%02d]", i), rd(c.FilesPrefixHint)) + for i, c := range m.GetExtraFullBackupPaths() { + table.Add(fmt.Sprintf("extra_full_backups[%02d]", i), rd(c)) } table.Add("truncate-to", rd(m.TruncatedTo)) } @@ -538,7 +538,7 @@ func MergeMigrations(m1 *pb.Migration, m2 *pb.Migration) *pb.Migration { out.TruncatedTo = max(m1.GetTruncatedTo(), m2.GetTruncatedTo()) out.DestructPrefix = append(out.DestructPrefix, m1.GetDestructPrefix()...) out.DestructPrefix = append(out.DestructPrefix, m2.GetDestructPrefix()...) - out.ExtraFullBackups = append(out.ExtraFullBackups, m1.GetExtraFullBackups()...) + out.ExtraFullBackupPaths = append(out.ExtraFullBackupPaths, m1.GetExtraFullBackupPaths()...) return out } @@ -1093,15 +1093,7 @@ func (m MigrationExt) doTruncating(ctx context.Context, mig *pb.Migration, resul m.tryRemovePrefix(ctx, pfx, result) } - for _, extraBackup := range mig.ExtraFullBackups { - if extraBackup.AsIfTs > mig.TruncatedTo { - result.NewBase.ExtraFullBackups = append(result.NewBase.ExtraFullBackups, extraBackup) - } else { - for _, pfx := range extraBackup.Files { - m.tryRemovePrefix(ctx, pfx.Name, result) - } - } - } + // TODO: Clean up the extra full backup SSTs. result.NewBase.TruncatedTo = mig.TruncatedTo @@ -1375,25 +1367,17 @@ func isEmptyMetadata(md *pb.Metadata) bool { */ func hashMigration(m *pb.Migration) uint64 { - var crc64 uint64 = 0 + var crc64Res uint64 = 0 for _, compaction := range m.Compactions { - crc64 ^= compaction.ArtifactsHash + crc64Res ^= compaction.ArtifactsHash } for _, metaEdit := range m.EditMeta { - crc64 ^= hashMetaEdit(metaEdit) + crc64Res ^= hashMetaEdit(metaEdit) } - for _, extBkup := range m.ExtraFullBackups { - crc64 ^= hashExtraBackup(extBkup) - } - return crc64 ^ m.TruncatedTo -} - -func hashExtraBackup(extBkup *pb.ExtraFullBackup) uint64 { - bs, err := extBkup.Marshal() - if err != nil { - panic(fmt.Sprintf("failed to marshal message, this shouldn't happen: %s", err)) + for _, extBkup := range m.ExtraFullBackupPaths { + crc64Res ^= crc64.Checksum([]byte(extBkup), crc64.MakeTable(crc64.ISO)) } - return crc64.Checksum(bs, crc64.MakeTable(crc64.ISO)) + return crc64Res ^ m.TruncatedTo } func hashMetaEdit(metaEdit *pb.MetaEdit) uint64 { diff --git a/tests/realtikvtest/brietest/pitr_test.go b/tests/realtikvtest/brietest/pitr_test.go index cf6475afb0830..d1e6ca07671cf 100644 --- a/tests/realtikvtest/brietest/pitr_test.go +++ b/tests/realtikvtest/brietest/pitr_test.go @@ -11,6 +11,7 @@ import ( "github.com/pingcap/log" "github.com/pingcap/tidb/br/pkg/glue" "github.com/pingcap/tidb/br/pkg/gluetidb" + "github.com/pingcap/tidb/br/pkg/logutil" "github.com/pingcap/tidb/br/pkg/streamhelper" "github.com/pingcap/tidb/br/pkg/task" "github.com/pingcap/tidb/br/pkg/task/operator" @@ -184,9 +185,14 @@ func (kit *LogBackupKit) forceFlush() { kit.mustExec(func(ctx context.Context) error { cfg := task.DefaultConfig() cfg.PD = append(cfg.PD, config.GetGlobalConfig().Path) - return operator.RunForceFlush(ctx, &operator.ForceFlushConfig{ + err := operator.RunForceFlush(ctx, &operator.ForceFlushConfig{ Config: cfg, }) + if err != nil { + log.Warn("It seems this version of TiKV doesn't support force flush, the test may be much more slower.", + logutil.ShortError(err)) + } + return nil }) } @@ -197,7 +203,7 @@ func (kit *LogBackupKit) forceFlushAndWait(taskName string) { ckpt := kit.CheckpointTSOf(taskName) log.Info("checkpoint", zap.Uint64("checkpoint", ckpt), zap.Uint64("ts", ts)) return ckpt >= ts - }, 21*time.Second, 1*time.Second) + }, 300*time.Second, 1*time.Second) time.Sleep(6 * time.Second) // Wait the storage checkpoint uploaded... } From afad018bd378540d9f80eabe11f6a366f1344feb Mon Sep 17 00:00:00 2001 From: hillium Date: Fri, 6 Dec 2024 15:21:20 +0800 Subject: [PATCH 11/36] fix migration not uploaded Signed-off-by: hillium --- br/pkg/restore/snap_client/pitr_collector.go | 13 ++++++++++++- 1 file changed, 12 insertions(+), 1 deletion(-) diff --git a/br/pkg/restore/snap_client/pitr_collector.go b/br/pkg/restore/snap_client/pitr_collector.go index be213879bcd4a..b60183028d960 100644 --- a/br/pkg/restore/snap_client/pitr_collector.go +++ b/br/pkg/restore/snap_client/pitr_collector.go @@ -22,7 +22,6 @@ import ( ) type pitrCollectorRestorer struct { - restore.SstRestorer // the context used for committing. cx context.Context // the context bound to the errgroup. @@ -50,6 +49,10 @@ func (p pitrCollectorRestorer) GoRestore(onProgress func(int64), batchFileSets . return nil } + if err := p.coll.prepareMigIfNeeded(p.cx); err != nil { + return err + } + p.wg.Go(func() error { for _, fileSets := range batchFileSets { for _, fileSet := range fileSets { @@ -101,6 +104,7 @@ type pitrCollector struct { // Mutable state. committing committing committingLock sync.Mutex + putMigOnce sync.Once // Delegates. tso func(ctx context.Context) (uint64, error) @@ -231,6 +235,13 @@ func (c *pitrCollector) prepareMig(ctx context.Context) error { return c.persist(ctx) } +func (c *pitrCollector) prepareMigIfNeeded(ctx context.Context) (err error) { + c.putMigOnce.Do(func() { + err = c.prepareMig(ctx) + }) + return +} + func (c *pitrCollector) commit(ctx context.Context) error { c.committing.commit() return c.persist(ctx) From 2dfdb30ee5f7bd998311ef3d83332b06096a4128 Mon Sep 17 00:00:00 2001 From: hillium Date: Fri, 6 Dec 2024 17:49:30 +0800 Subject: [PATCH 12/36] try filter out full backup --- br/pkg/restore/log_client/migration.go | 68 +++++++++++++++++++- br/pkg/restore/snap_client/pitr_collector.go | 60 ++++++++--------- 2 files changed, 97 insertions(+), 31 deletions(-) diff --git a/br/pkg/restore/log_client/migration.go b/br/pkg/restore/log_client/migration.go index 4b6813486b6da..8d49334c40fb0 100644 --- a/br/pkg/restore/log_client/migration.go +++ b/br/pkg/restore/log_client/migration.go @@ -17,10 +17,14 @@ package logclient import ( "context" + "github.com/google/uuid" "github.com/pingcap/errors" backuppb "github.com/pingcap/kvproto/pkg/brpb" + "github.com/pingcap/log" + berrors "github.com/pingcap/tidb/br/pkg/errors" "github.com/pingcap/tidb/br/pkg/storage" "github.com/pingcap/tidb/br/pkg/utils/iter" + "go.uber.org/zap" ) type logicalSkipMap map[uint64]struct{} @@ -164,6 +168,7 @@ func (builder *WithMigrationsBuilder) Build(migs []*backuppb.Migration) WithMigr skipmap: skipmap, compactionDirs: compactionDirs, fullBackups: fullBackups, + restoredTS: builder.restoredTS, } return withMigrations } @@ -216,6 +221,7 @@ type WithMigrations struct { skipmap metaSkipMap compactionDirs []string fullBackups []string + restoredTS uint64 } func (wm *WithMigrations) Metas(metaNameIter MetaNameIter) MetaMigrationsIter { @@ -247,7 +253,7 @@ func (wm *WithMigrations) Compactions(ctx context.Context, s storage.ExternalSto func (wm *WithMigrations) ExtraFullBackups(ctx context.Context, s storage.ExternalStorage) iter.TryNextor[*backuppb.ExtraFullBackup] { fullBackupDirIter := iter.FromSlice(wm.fullBackups) - return iter.TryMap(fullBackupDirIter, func(name string) (*backuppb.ExtraFullBackup, error) { + backups := iter.TryMap(fullBackupDirIter, func(name string) (*backuppb.ExtraFullBackup, error) { // name is the absolute path in external storage. bkup, err := readExtraFullBackup(ctx, name, s) if err != nil { @@ -255,6 +261,66 @@ func (wm *WithMigrations) ExtraFullBackups(ctx context.Context, s storage.Extern } return bkup, nil }) + coll := extBackupCollector{ + inner: backups, + restoreTS: wm.restoredTS, + + collected: make(map[uuid.UUID][]*backuppb.ExtraFullBackup), + finished: make(map[uuid.UUID]struct{}), + } + return iter.FlatMap(&coll, iter.FromSlice) +} + +type extBackupCollector struct { + inner iter.TryNextor[*backuppb.ExtraFullBackup] + restoreTS uint64 + + collected map[uuid.UUID][]*backuppb.ExtraFullBackup + finished map[uuid.UUID]struct{} +} + +// Implement iter.TryNextor for extBackupCollector + +func (c *extBackupCollector) TryNext(ctx context.Context) iter.IterResult[[]*backuppb.ExtraFullBackup] { + for { + res := c.inner.TryNext(ctx) + if res.FinishedOrError() { + for id, fbks := range c.collected { + for _, fbk := range fbks { + log.Warn("Dropping unfinished extra full backup.", zap.Stringer("UUID", id), zap.String("path", fbk.FilesPrefixHint)) + } + } + return iter.DoneBy[[]*backuppb.ExtraFullBackup](res) + } + + fbk := res.Item + if fbk.AsIfTs > c.restoreTS { + log.Info("Filter out out-of-ts-range extra full backup.", + zap.Uint64("restoreTS", c.restoreTS), zap.Uint64("asIfTs", fbk.AsIfTs), zap.String("path", fbk.FilesPrefixHint)) + continue + } + + if len(fbk.BackupUuid) != len(uuid.UUID{}) { + return iter.Throw[[]*backuppb.ExtraFullBackup]( + errors.Annotatef(berrors.ErrInvalidArgument, "the full backup UUID has bad length(%d)", len(fbk.BackupUuid)), + ) + } + uid := uuid.UUID(fbk.BackupUuid) + log.Info("Collecting extra full backup", zap.Stringer("UUID", uid), zap.String("path", fbk.FilesPrefixHint), zap.Bool("finished", fbk.Finished)) + + if _, ok := c.finished[uid]; ok { + log.Warn("Encountered a finished full backup.", zap.Stringer("UUID", uid), zap.String("path", fbk.FilesPrefixHint)) + return iter.Emit([]*backuppb.ExtraFullBackup{fbk}) + } + + c.collected[uid] = append(c.collected[uid], fbk) + if fbk.Finished { + c.finished[uid] = struct{}{} + items := c.collected[uid] + delete(c.collected, uid) + return iter.Emit(items) + } + } } func readExtraFullBackup(ctx context.Context, name string, s storage.ExternalStorage) (*backuppb.ExtraFullBackup, error) { diff --git a/br/pkg/restore/snap_client/pitr_collector.go b/br/pkg/restore/snap_client/pitr_collector.go index b60183028d960..15935c24d3a5b 100644 --- a/br/pkg/restore/snap_client/pitr_collector.go +++ b/br/pkg/restore/snap_client/pitr_collector.go @@ -82,7 +82,7 @@ func (p pitrCollectorRestorer) WaitUntilFinish() error { if err != nil { return errors.Annotate(err, "failed to wait on pitrCollector") } - return errors.Annotatef(p.coll.persist(p.cx), "failed to persist the metadata of uploaded SSTs") + return errors.Annotatef(p.coll.persistExtraBackupMeta(p.cx), "failed to persist the metadata of uploaded SSTs") } // Close releases any resources associated with the restoration process. @@ -102,24 +102,20 @@ type pitrCollector struct { restoreUUID uuid.UUID // Mutable state. - committing committing - committingLock sync.Mutex - putMigOnce sync.Once + extraBackupMeta extraBackupMeta + extraBackupMetaLock sync.Mutex + putMigOnce sync.Once // Delegates. tso func(ctx context.Context) (uint64, error) } -type committing struct { +type extraBackupMeta struct { msg pb.ExtraFullBackup rewrites map[int64]int64 } -func (c *committing) commit() { - c.msg.Finished = true -} - -func (c *committing) genMsg() *pb.ExtraFullBackup { +func (c *extraBackupMeta) genMsg() *pb.ExtraFullBackup { msg := util.ProtoV1Clone(&c.msg) for old, new := range c.rewrites { msg.RewrittenTables = append(msg.RewrittenTables, &pb.RewrittenTableID{UpstreamOfUpstream: old, Upstream: new}) @@ -127,11 +123,10 @@ func (c *committing) genMsg() *pb.ExtraFullBackup { return msg } -// doWithCommittingLock edits the committing ExtraFullBackup. -func (c *pitrCollector) doWithCommittingLock(f func()) { - c.committingLock.Lock() +func (c *pitrCollector) doWithExtraBackupMetaLock(f func()) { + c.extraBackupMetaLock.Lock() f() - c.committingLock.Unlock() + c.extraBackupMetaLock.Unlock() } // outputPath constructs the path by a relative path for outputting. @@ -169,7 +164,7 @@ func (c *pitrCollector) PutSST(ctx context.Context, f *pb.File) error { } f.Name = out - c.doWithCommittingLock(func() { c.committing.msg.Files = append(c.committing.msg.Files, f) }) + c.doWithExtraBackupMetaLock(func() { c.extraBackupMeta.msg.Files = append(c.extraBackupMeta.msg.Files, f) }) return nil } @@ -179,8 +174,8 @@ func (c *pitrCollector) PutRewriteRule(_ context.Context, oldID int64, newID int return nil } var err error - c.doWithCommittingLock(func() { - if oldVal, ok := c.committing.rewrites[oldID]; ok && oldVal != newID { + c.doWithExtraBackupMetaLock(func() { + if oldVal, ok := c.extraBackupMeta.rewrites[oldID]; ok && oldVal != newID { err = errors.Annotatef( berrors.ErrInvalidArgument, "pitr coll rewrite rule conflict: we had %v -> %v, but you want rewrite to %v", @@ -190,16 +185,16 @@ func (c *pitrCollector) PutRewriteRule(_ context.Context, oldID int64, newID int ) return } - c.committing.rewrites[oldID] = newID + c.extraBackupMeta.rewrites[oldID] = newID }) return err } -func (c *pitrCollector) persist(ctx context.Context) (err error) { - c.committingLock.Lock() - defer c.committingLock.Unlock() +func (c *pitrCollector) persistExtraBackupMeta(ctx context.Context) (err error) { + c.extraBackupMetaLock.Lock() + defer c.extraBackupMetaLock.Unlock() - msg := c.committing.genMsg() + msg := c.extraBackupMeta.genMsg() bs, err := msg.Marshal() if err != nil { return errors.Annotate(err, "failed to marsal the committing message") @@ -227,12 +222,12 @@ func (c *pitrCollector) prepareMig(ctx context.Context) error { return errors.Annotatef(err, "failed to add the extra backup at path %s", c.metaPath()) } - c.doWithCommittingLock(func() { + c.doWithExtraBackupMetaLock(func() { c.resetCommitting() }) // Persist the metadata in case of SSTs were uploaded but the meta wasn't, // which leads to a leakage. - return c.persist(ctx) + return c.persistExtraBackupMeta(ctx) } func (c *pitrCollector) prepareMigIfNeeded(ctx context.Context) (err error) { @@ -243,17 +238,22 @@ func (c *pitrCollector) prepareMigIfNeeded(ctx context.Context) (err error) { } func (c *pitrCollector) commit(ctx context.Context) error { - c.committing.commit() - return c.persist(ctx) + c.extraBackupMeta.msg.Finished = true + ts, err := c.tso(ctx) + if err != nil { + return err + } + c.extraBackupMeta.msg.AsIfTs = ts + return c.persistExtraBackupMeta(ctx) } func (c *pitrCollector) resetCommitting() { - c.committing = committing{ + c.extraBackupMeta = extraBackupMeta{ rewrites: map[int64]int64{}, } - c.committing.msg.FilesPrefixHint = c.sstPath("") - c.committing.msg.Finished = false - c.committing.msg.BackupUuid = c.restoreUUID[:] + c.extraBackupMeta.msg.FilesPrefixHint = c.sstPath("") + c.extraBackupMeta.msg.Finished = false + c.extraBackupMeta.msg.BackupUuid = c.restoreUUID[:] } // PiTRCollDep is the dependencies of a PiTR collector. From 9fc057e0f7783ca1e23d4f8ff2b5ee5e96810fef Mon Sep 17 00:00:00 2001 From: hillium Date: Mon, 9 Dec 2024 14:57:07 +0800 Subject: [PATCH 13/36] persist progress when exit --- br/pkg/restore/snap_client/pitr_collector.go | 31 +++++++++++++++----- br/pkg/summary/summary.go | 13 ++++++++ 2 files changed, 37 insertions(+), 7 deletions(-) diff --git a/br/pkg/restore/snap_client/pitr_collector.go b/br/pkg/restore/snap_client/pitr_collector.go index 15935c24d3a5b..0005528514b2b 100644 --- a/br/pkg/restore/snap_client/pitr_collector.go +++ b/br/pkg/restore/snap_client/pitr_collector.go @@ -5,19 +5,23 @@ import ( "fmt" "path/filepath" "sync" + "time" "github.com/google/uuid" "github.com/pingcap/errors" pb "github.com/pingcap/kvproto/pkg/brpb" + "github.com/pingcap/log" berrors "github.com/pingcap/tidb/br/pkg/errors" "github.com/pingcap/tidb/br/pkg/restore" "github.com/pingcap/tidb/br/pkg/storage" "github.com/pingcap/tidb/br/pkg/stream" "github.com/pingcap/tidb/br/pkg/streamhelper" + "github.com/pingcap/tidb/br/pkg/summary" "github.com/pingcap/tidb/pkg/util" "github.com/tikv/client-go/v2/oracle" pd "github.com/tikv/pd/client" clientv3 "go.etcd.io/etcd/client/v3" + "go.uber.org/zap" "golang.org/x/sync/errgroup" ) @@ -57,12 +61,12 @@ func (p pitrCollectorRestorer) GoRestore(onProgress func(int64), batchFileSets . for _, fileSets := range batchFileSets { for _, fileSet := range fileSets { for _, file := range fileSet.SSTFiles { - if err := p.coll.PutSST(p.ecx, file); err != nil { + if err := p.coll.putSST(p.ecx, file); err != nil { return errors.Annotatef(err, "failed to put sst %s", file.GetName()) } } for _, hint := range fileSet.RewriteRules.TableIDRemapHint { - if err := p.coll.PutRewriteRule(p.ecx, hint.Origin, hint.Rewritten); err != nil { + if err := p.coll.putRewriteRule(p.ecx, hint.Origin, hint.Rewritten); err != nil { return errors.Annotatef(err, "failed to put rewrite rule of %v", fileSet.RewriteRules) } } @@ -87,10 +91,22 @@ func (p pitrCollectorRestorer) WaitUntilFinish() error { // Close releases any resources associated with the restoration process. func (p pitrCollectorRestorer) Close() error { + if p.cx.Err() != nil { + log.Warn("pitrCollectorRestorer.Close: the context already closed, will use a temporary context.") + var cancel context.CancelFunc + p.cx, cancel = context.WithTimeout(context.Background(), 30*time.Second) + defer cancel() + } + if !p.coll.enabled { return nil } - return errors.Annotate(p.coll.commit(p.cx), "failed to commit pitrCollector") + if summary.OnceSucceed() { + return errors.Annotate(p.coll.commit(p.cx), "failed to commit pitrCollector") + } + + log.Warn("Backup not success, put a half-finished metadata to the log backup.", zap.Stringer("uuid", p.coll.restoreUUID)) + return errors.Annotatef(p.coll.persistExtraBackupMeta(p.cx), "failed to persist the meta") } type pitrCollector struct { @@ -142,8 +158,8 @@ func (c *pitrCollector) sstPath(name string) string { return c.outputPath("sst_files", name) } -// PutSST records an SST file. -func (c *pitrCollector) PutSST(ctx context.Context, f *pb.File) error { +// putSST records an SST file. +func (c *pitrCollector) putSST(ctx context.Context, f *pb.File) error { if !c.enabled { return nil } @@ -168,8 +184,8 @@ func (c *pitrCollector) PutSST(ctx context.Context, f *pb.File) error { return nil } -// PutRewriteRule records a rewrite rule. -func (c *pitrCollector) PutRewriteRule(_ context.Context, oldID int64, newID int64) error { +// putRewriteRule records a rewrite rule. +func (c *pitrCollector) putRewriteRule(_ context.Context, oldID int64, newID int64) error { if !c.enabled { return nil } @@ -194,6 +210,7 @@ func (c *pitrCollector) persistExtraBackupMeta(ctx context.Context) (err error) c.extraBackupMetaLock.Lock() defer c.extraBackupMetaLock.Unlock() + log.Info("Persisting extra backup meta.", zap.Stringer("uuid", c.restoreUUID), zap.String("path", c.metaPath())) msg := c.extraBackupMeta.genMsg() bs, err := msg.Marshal() if err != nil { diff --git a/br/pkg/summary/summary.go b/br/pkg/summary/summary.go index 3642410bdcd16..3b32f9cd79b35 100644 --- a/br/pkg/summary/summary.go +++ b/br/pkg/summary/summary.go @@ -3,11 +3,16 @@ package summary import ( + "sync/atomic" "time" "go.uber.org/zap" ) +var ( + onceSetToSuccess atomic.Bool +) + // SetUnit set unit "backup/restore" for summary log. func SetUnit(unit string) { collector.SetUnit(unit) @@ -40,9 +45,17 @@ func CollectUint(name string, t uint64) { // SetSuccessStatus sets final success status. func SetSuccessStatus(success bool) { + if success { + onceSetToSuccess.Store(true) + } collector.SetSuccessStatus(success) } +// OnceSucceed returns whether a `SetSuccessStatus(true)` was call. +func OnceSucceed() bool { + return onceSetToSuccess.Load() +} + // NowDureTime returns the duration between start time and current time func NowDureTime() time.Duration { return collector.NowDureTime() From a512330339e73a449c89a761c134a9df5cb172d5 Mon Sep 17 00:00:00 2001 From: hillium Date: Mon, 9 Dec 2024 15:21:36 +0800 Subject: [PATCH 14/36] initial try to hook at importer --- br/pkg/checkpoint/storage.go | 5 ++ br/pkg/restore/snap_client/client.go | 12 ++--- br/pkg/restore/snap_client/import.go | 9 +++- br/pkg/restore/snap_client/pitr_collector.go | 52 +++++++++++++++----- 4 files changed, 60 insertions(+), 18 deletions(-) diff --git a/br/pkg/checkpoint/storage.go b/br/pkg/checkpoint/storage.go index 465924f8300f4..272e589e975d0 100644 --- a/br/pkg/checkpoint/storage.go +++ b/br/pkg/checkpoint/storage.go @@ -32,6 +32,11 @@ import ( "go.uber.org/zap" ) +type hookedOnFlush struct { + checkpointStorage + cb func(ctx context.Context) error +} + type checkpointStorage interface { flushCheckpointData(ctx context.Context, data []byte) error flushCheckpointChecksum(ctx context.Context, data []byte) error diff --git a/br/pkg/restore/snap_client/client.go b/br/pkg/restore/snap_client/client.go index 5d95b500220e7..331ff7ea15ff4 100644 --- a/br/pkg/restore/snap_client/client.go +++ b/br/pkg/restore/snap_client/client.go @@ -77,6 +77,7 @@ const minBatchDdlSize = 1 type SnapClient struct { restorer restore.SstRestorer + importer *SnapFileImporter // Tool clients used by SnapClient pdClient pd.Client pdHTTPClient pdhttp.Client @@ -431,7 +432,7 @@ func (rc *SnapClient) InstallPiTRSupport(ctx context.Context, deps PiTRCollDep) } rc.pitrColl = collector collector.restoreUUID = rc.restoreUUID - rc.restorer = restore.Dual(rc.pitrColl.createRestorer(ctx), rc.restorer) + rc.importer.beforeIngestCallbacks = append(rc.importer.beforeIngestCallbacks, collector.onBatch) return nil } @@ -545,7 +546,6 @@ func (rc *SnapClient) initClients(ctx context.Context, backend *backuppb.Storage metaClient := split.NewClient(rc.pdClient, rc.pdHTTPClient, rc.tlsConf, maxSplitKeysOnce, rc.storeCount+1, splitClientOpts...) importCli := importclient.NewImportClient(metaClient, rc.tlsConf, rc.keepaliveConf) - var fileImporter *SnapFileImporter opt := NewSnapFileImporterOptions( rc.cipher, metaClient, importCli, backend, rc.rewriteMode, stores, rc.concurrencyPerStore, createCallBacks, closeCallBacks, @@ -556,20 +556,20 @@ func (rc *SnapClient) initClients(ctx context.Context, backend *backuppb.Storage mode = Txn } // for raw/txn mode. use backupMeta.ApiVersion to create fileImporter - fileImporter, err = NewSnapFileImporter(ctx, rc.backupMeta.ApiVersion, mode, opt) + rc.importer, err = NewSnapFileImporter(ctx, rc.backupMeta.ApiVersion, mode, opt) if err != nil { return errors.Trace(err) } // Raw/Txn restore are not support checkpoint for now - rc.restorer = restore.NewSimpleSstRestorer(ctx, fileImporter, rc.workerPool, nil) + rc.restorer = restore.NewSimpleSstRestorer(ctx, rc.importer, rc.workerPool, nil) } else { // or create a fileImporter with the cluster API version - fileImporter, err = NewSnapFileImporter( + rc.importer, err = NewSnapFileImporter( ctx, rc.dom.Store().GetCodec().GetAPIVersion(), TiDBFull, opt) if err != nil { return errors.Trace(err) } - rc.restorer = restore.NewMultiTablesRestorer(ctx, fileImporter, rc.workerPool, rc.checkpointRunner) + rc.restorer = restore.NewMultiTablesRestorer(ctx, rc.importer, rc.workerPool, rc.checkpointRunner) } return nil } diff --git a/br/pkg/restore/snap_client/import.go b/br/pkg/restore/snap_client/import.go index 1238754c394bf..d0ea353ee9e31 100644 --- a/br/pkg/restore/snap_client/import.go +++ b/br/pkg/restore/snap_client/import.go @@ -141,7 +141,8 @@ type SnapFileImporter struct { downloadTokensMap *storeTokenChannelMap ingestTokensMap *storeTokenChannelMap - closeCallbacks []func(*SnapFileImporter) error + closeCallbacks []func(*SnapFileImporter) error + beforeIngestCallbacks []func(context.Context, restore.BatchBackupFileSet) error concurrencyPerStore uint @@ -369,6 +370,12 @@ func (importer *SnapFileImporter) Import( ctx context.Context, backupFileSets ...restore.BackupFileSet, ) error { + for i, cb := range importer.beforeIngestCallbacks { + if err := cb(ctx, backupFileSets); err != nil { + return errors.Annotatef(err, "failed to executing the callback #%d", i) + } + } + // Rewrite the start key and end key of file to scan regions startKey, endKey, err := importer.getKeyRangeForFiles(backupFileSets) if err != nil { diff --git a/br/pkg/restore/snap_client/pitr_collector.go b/br/pkg/restore/snap_client/pitr_collector.go index 0005528514b2b..a2cfd4e0d8c4b 100644 --- a/br/pkg/restore/snap_client/pitr_collector.go +++ b/br/pkg/restore/snap_client/pitr_collector.go @@ -59,17 +59,8 @@ func (p pitrCollectorRestorer) GoRestore(onProgress func(int64), batchFileSets . p.wg.Go(func() error { for _, fileSets := range batchFileSets { - for _, fileSet := range fileSets { - for _, file := range fileSet.SSTFiles { - if err := p.coll.putSST(p.ecx, file); err != nil { - return errors.Annotatef(err, "failed to put sst %s", file.GetName()) - } - } - for _, hint := range fileSet.RewriteRules.TableIDRemapHint { - if err := p.coll.putRewriteRule(p.ecx, hint.Origin, hint.Rewritten); err != nil { - return errors.Annotatef(err, "failed to put rewrite rule of %v", fileSet.RewriteRules) - } - } + if err := p.coll.onBatch(p.cx, fileSets); err != nil { + return err } } return nil @@ -139,6 +130,45 @@ func (c *extraBackupMeta) genMsg() *pb.ExtraFullBackup { return msg } +func (c *pitrCollector) onBatch(ctx context.Context, fileSets restore.BatchBackupFileSet) error { + if !c.enabled { + return nil + } + + if err := c.prepareMigIfNeeded(ctx); err != nil { + return err + } + + eg, ectx := errgroup.WithContext(ctx) + for _, fileSet := range fileSets { + for _, file := range fileSet.SSTFiles { + file := file + eg.Go(func() error { + if err := c.putSST(ectx, file); err != nil { + return errors.Annotatef(err, "failed to put sst %s", file.GetName()) + } + return nil + }) + } + for _, hint := range fileSet.RewriteRules.TableIDRemapHint { + hint := hint + eg.Go(func() error { + if err := c.putRewriteRule(ectx, hint.Origin, hint.Rewritten); err != nil { + return errors.Annotatef(err, "failed to put rewrite rule of %v", fileSet.RewriteRules) + } + return nil + }) + } + } + + err := eg.Wait() + if err != nil { + return err + } + + return errors.Annotatef(c.persistExtraBackupMeta(ctx), "failed to persist backup meta when finishing batch") +} + func (c *pitrCollector) doWithExtraBackupMetaLock(f func()) { c.extraBackupMetaLock.Lock() f() From a734f24bf85433fe60b95b0df53f4660ede04618 Mon Sep 17 00:00:00 2001 From: hillium Date: Mon, 9 Dec 2024 15:30:38 +0800 Subject: [PATCH 15/36] fix incorrect state --- br/pkg/restore/snap_client/client.go | 3 +++ br/pkg/restore/snap_client/pitr_collector.go | 10 ++++++++++ 2 files changed, 13 insertions(+) diff --git a/br/pkg/restore/snap_client/client.go b/br/pkg/restore/snap_client/client.go index 331ff7ea15ff4..4f0df6abfcea3 100644 --- a/br/pkg/restore/snap_client/client.go +++ b/br/pkg/restore/snap_client/client.go @@ -433,6 +433,9 @@ func (rc *SnapClient) InstallPiTRSupport(ctx context.Context, deps PiTRCollDep) rc.pitrColl = collector collector.restoreUUID = rc.restoreUUID rc.importer.beforeIngestCallbacks = append(rc.importer.beforeIngestCallbacks, collector.onBatch) + rc.importer.closeCallbacks = append(rc.importer.closeCallbacks, func(sfi *SnapFileImporter) error { + return collector.close() + }) return nil } diff --git a/br/pkg/restore/snap_client/pitr_collector.go b/br/pkg/restore/snap_client/pitr_collector.go index a2cfd4e0d8c4b..1d4eac9b8bf12 100644 --- a/br/pkg/restore/snap_client/pitr_collector.go +++ b/br/pkg/restore/snap_client/pitr_collector.go @@ -130,6 +130,16 @@ func (c *extraBackupMeta) genMsg() *pb.ExtraFullBackup { return msg } +func (c *pitrCollector) close() error { + if !c.enabled { + return nil + } + + cx, cancel := context.WithTimeout(context.Background(), 30*time.Second) + defer cancel() + return c.commit(cx) +} + func (c *pitrCollector) onBatch(ctx context.Context, fileSets restore.BatchBackupFileSet) error { if !c.enabled { return nil From eac139ec0f73819fdb80b8a3ff1ccd67be2bbaaf Mon Sep 17 00:00:00 2001 From: hillium Date: Mon, 9 Dec 2024 15:55:01 +0800 Subject: [PATCH 16/36] shouldn't put finished when failed to commit --- br/pkg/restore/snap_client/pitr_collector.go | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/br/pkg/restore/snap_client/pitr_collector.go b/br/pkg/restore/snap_client/pitr_collector.go index 1d4eac9b8bf12..e0673ddb080f1 100644 --- a/br/pkg/restore/snap_client/pitr_collector.go +++ b/br/pkg/restore/snap_client/pitr_collector.go @@ -137,7 +137,14 @@ func (c *pitrCollector) close() error { cx, cancel := context.WithTimeout(context.Background(), 30*time.Second) defer cancel() - return c.commit(cx) + + if summary.OnceSucceed() { + return errors.Annotate(c.commit(cx), "failed to commit pitrCollector") + } + + log.Warn("Backup not success, put a half-finished metadata to the log backup.", + zap.Stringer("uuid", c.restoreUUID)) + return errors.Annotatef(c.persistExtraBackupMeta(cx), "failed to persist the meta") } func (c *pitrCollector) onBatch(ctx context.Context, fileSets restore.BatchBackupFileSet) error { From 9a0659dec6e279959930bb5e8dc1dea08653e78c Mon Sep 17 00:00:00 2001 From: hillium Date: Tue, 10 Dec 2024 11:51:02 +0800 Subject: [PATCH 17/36] added more test cases --- br/pkg/restore/log_client/export_test.go | 8 ++ br/pkg/restore/log_client/migration.go | 12 +- br/pkg/restore/log_client/migration_test.go | 150 ++++++++++++++++++++ 3 files changed, 164 insertions(+), 6 deletions(-) diff --git a/br/pkg/restore/log_client/export_test.go b/br/pkg/restore/log_client/export_test.go index 9c95409c9d754..685d4150236fa 100644 --- a/br/pkg/restore/log_client/export_test.go +++ b/br/pkg/restore/log_client/export_test.go @@ -127,3 +127,11 @@ func (helper *FakeStreamMetadataHelper) ReadFile( ) ([]byte, error) { return helper.Data[offset : offset+length], nil } + +func (w *WithMigrations) AddExtraFullBackup(extPath string) { + w.fullBackups = append(w.fullBackups, extPath) +} + +func (w *WithMigrations) SetRestoredTS(ts uint64) { + w.restoredTS = ts +} diff --git a/br/pkg/restore/log_client/migration.go b/br/pkg/restore/log_client/migration.go index 8d49334c40fb0..5aacc2473bb8e 100644 --- a/br/pkg/restore/log_client/migration.go +++ b/br/pkg/restore/log_client/migration.go @@ -294,12 +294,6 @@ func (c *extBackupCollector) TryNext(ctx context.Context) iter.IterResult[[]*bac } fbk := res.Item - if fbk.AsIfTs > c.restoreTS { - log.Info("Filter out out-of-ts-range extra full backup.", - zap.Uint64("restoreTS", c.restoreTS), zap.Uint64("asIfTs", fbk.AsIfTs), zap.String("path", fbk.FilesPrefixHint)) - continue - } - if len(fbk.BackupUuid) != len(uuid.UUID{}) { return iter.Throw[[]*backuppb.ExtraFullBackup]( errors.Annotatef(berrors.ErrInvalidArgument, "the full backup UUID has bad length(%d)", len(fbk.BackupUuid)), @@ -318,6 +312,12 @@ func (c *extBackupCollector) TryNext(ctx context.Context) iter.IterResult[[]*bac c.finished[uid] = struct{}{} items := c.collected[uid] delete(c.collected, uid) + + if fbk.AsIfTs > c.restoreTS { + log.Info("Filter out out-of-ts-range extra full backup.", + zap.Uint64("restoreTS", c.restoreTS), zap.Uint64("asIfTs", fbk.AsIfTs), zap.String("path", fbk.FilesPrefixHint)) + continue + } return iter.Emit(items) } } diff --git a/br/pkg/restore/log_client/migration_test.go b/br/pkg/restore/log_client/migration_test.go index 5368d7416dadf..0a3bbf395564e 100644 --- a/br/pkg/restore/log_client/migration_test.go +++ b/br/pkg/restore/log_client/migration_test.go @@ -19,8 +19,10 @@ import ( "fmt" "testing" + "github.com/google/uuid" backuppb "github.com/pingcap/kvproto/pkg/brpb" logclient "github.com/pingcap/tidb/br/pkg/restore/log_client" + "github.com/pingcap/tidb/br/pkg/storage" "github.com/pingcap/tidb/br/pkg/utils/iter" "github.com/stretchr/testify/require" ) @@ -350,3 +352,151 @@ func TestMigrations(t *testing.T) { } } } + +type efOP func(*backuppb.ExtraFullBackup) + +func extFullBkup(ops ...efOP) *backuppb.ExtraFullBackup { + ef := &backuppb.ExtraFullBackup{} + for _, op := range ops { + op(ef) + } + return ef +} + +func finished() efOP { + return func(ef *backuppb.ExtraFullBackup) { + ef.Finished = true + } +} + +func identity() efOP { + id := uuid.New() + return func(ef *backuppb.ExtraFullBackup) { + ef.BackupUuid = id[:] + } +} + +func pfx(pfx string) efOP { + return func(ef *backuppb.ExtraFullBackup) { + ef.FilesPrefixHint = pfx + } +} + +func asIfTS(ts uint64) efOP { + return func(ef *backuppb.ExtraFullBackup) { + ef.AsIfTs = ts + } +} + +func pef(t *testing.T, fb *backuppb.ExtraFullBackup, sn int, s storage.ExternalStorage) string { + path := fmt.Sprintf("extbackupmeta_%08d", sn) + bs, err := fb.Marshal() + if err != nil { + require.NoError(t, err) + } + + err = s.WriteFile(context.Background(), path, bs) + require.NoError(t, err) + return path +} + +// tmp creates a temporary storage. +func tmp(t *testing.T) *storage.LocalStorage { + tmpDir := t.TempDir() + s, err := storage.NewLocalStorage(tmpDir) + require.NoError(t, err) + s.IgnoreEnoentForDelete = true + return s +} + +func assertFullBackupPfxs(t *testing.T, it iter.TryNextor[*backuppb.ExtraFullBackup], items ...string) { + i := 0 + for res := it.TryNext(context.Background()); !res.Finished; res = it.TryNext(context.Background()) { + require.NoError(t, res.Err) + require.Equal(t, items[i], res.Item.FilesPrefixHint, + "item %d not match, wants %s, got %s", i, items[i], res.Item.FilesPrefixHint) + i++ + } + require.Equal(t, i, len(items), "not exceeded: %#v, i = %d", items, i) +} + +func TestNotRestoreIncomplete(t *testing.T) { + ctx := context.Background() + strg := tmp(t) + ebk := extFullBkup(pfx("001"), asIfTS(90), identity()) + wm := new(logclient.WithMigrations) + wm.AddExtraFullBackup(pef(t, ebk, 0, strg)) + wm.SetRestoredTS(91) + + assertFullBackupPfxs(t, wm.ExtraFullBackups(ctx, strg)) +} + +func TestRestoreSegmented(t *testing.T) { + ctx := context.Background() + strg := tmp(t) + id := identity() + ebk1 := extFullBkup(pfx("001"), id) + ebk2 := extFullBkup(pfx("002"), asIfTS(90), finished(), id) + wm := new(logclient.WithMigrations) + wm.AddExtraFullBackup(pef(t, ebk1, 0, strg)) + wm.AddExtraFullBackup(pef(t, ebk2, 1, strg)) + wm.SetRestoredTS(91) + + assertFullBackupPfxs(t, wm.ExtraFullBackups(ctx, strg), "001", "002") +} + +func TestFilteredOut(t *testing.T) { + ctx := context.Background() + strg := tmp(t) + id := identity() + ebk1 := extFullBkup(pfx("001"), id) + ebk2 := extFullBkup(pfx("002"), asIfTS(90), finished(), id) + wm := new(logclient.WithMigrations) + wm.AddExtraFullBackup(pef(t, ebk1, 0, strg)) + wm.AddExtraFullBackup(pef(t, ebk2, 1, strg)) + wm.SetRestoredTS(89) + + assertFullBackupPfxs(t, wm.ExtraFullBackups(ctx, strg)) +} + +func TestMultiRestores(t *testing.T) { + ctx := context.Background() + strg := tmp(t) + id := identity() + id2 := identity() + + ebka1 := extFullBkup(pfx("001"), id) + ebkb1 := extFullBkup(pfx("101"), id2) + ebkb2 := extFullBkup(pfx("102"), asIfTS(88), finished(), id2) + ebka2 := extFullBkup(pfx("002"), asIfTS(90), finished(), id) + + wm := new(logclient.WithMigrations) + wm.AddExtraFullBackup(pef(t, ebka1, 0, strg)) + wm.AddExtraFullBackup(pef(t, ebkb1, 2, strg)) + wm.AddExtraFullBackup(pef(t, ebkb2, 3, strg)) + wm.AddExtraFullBackup(pef(t, ebka2, 4, strg)) + wm.SetRestoredTS(91) + + assertFullBackupPfxs(t, wm.ExtraFullBackups(ctx, strg), "101", "102", "001", "002") +} + +func TestMultiFilteredOutOne(t *testing.T) { + ctx := context.Background() + strg := tmp(t) + id := identity() + id2 := identity() + + ebka1 := extFullBkup(pfx("001"), id) + ebkb1 := extFullBkup(pfx("101"), id2) + ebkb2 := extFullBkup(pfx("102"), asIfTS(88), finished(), id2) + ebka2 := extFullBkup(pfx("002"), asIfTS(90), finished(), id) + + wm := new(logclient.WithMigrations) + wm.AddExtraFullBackup(pef(t, ebka1, 0, strg)) + wm.AddExtraFullBackup(pef(t, ebkb1, 2, strg)) + wm.AddExtraFullBackup(pef(t, ebkb2, 3, strg)) + wm.AddExtraFullBackup(pef(t, ebka2, 4, strg)) + wm.SetRestoredTS(89) + + assertFullBackupPfxs(t, wm.ExtraFullBackups(ctx, strg), "101", "102") +} From ed34ea512fa0b1b30ab843b529d1fd56d745d755 Mon Sep 17 00:00:00 2001 From: hillium Date: Tue, 10 Dec 2024 12:07:15 +0800 Subject: [PATCH 18/36] make names more readable --- br/pkg/restore/log_client/migration_test.go | 42 ++++++++++----------- 1 file changed, 21 insertions(+), 21 deletions(-) diff --git a/br/pkg/restore/log_client/migration_test.go b/br/pkg/restore/log_client/migration_test.go index 0a3bbf395564e..c00e501eb675b 100644 --- a/br/pkg/restore/log_client/migration_test.go +++ b/br/pkg/restore/log_client/migration_test.go @@ -369,14 +369,14 @@ func finished() efOP { } } -func identity() efOP { +func makeID() efOP { id := uuid.New() return func(ef *backuppb.ExtraFullBackup) { ef.BackupUuid = id[:] } } -func pfx(pfx string) efOP { +func prefix(pfx string) efOP { return func(ef *backuppb.ExtraFullBackup) { ef.FilesPrefixHint = pfx } @@ -423,7 +423,7 @@ func assertFullBackupPfxs(t *testing.T, it iter.TryNextor[*backuppb.ExtraFullBac func TestNotRestoreIncomplete(t *testing.T) { ctx := context.Background() strg := tmp(t) - ebk := extFullBkup(pfx("001"), asIfTS(90), identity()) + ebk := extFullBkup(prefix("001"), asIfTS(90), makeID()) wm := new(logclient.WithMigrations) wm.AddExtraFullBackup(pef(t, ebk, 0, strg)) wm.SetRestoredTS(91) @@ -434,9 +434,9 @@ func TestNotRestoreIncomplete(t *testing.T) { func TestRestoreSegmented(t *testing.T) { ctx := context.Background() strg := tmp(t) - id := identity() - ebk1 := extFullBkup(pfx("001"), id) - ebk2 := extFullBkup(pfx("002"), asIfTS(90), finished(), id) + id := makeID() + ebk1 := extFullBkup(prefix("001"), id) + ebk2 := extFullBkup(prefix("002"), asIfTS(90), finished(), id) wm := new(logclient.WithMigrations) wm.AddExtraFullBackup(pef(t, ebk1, 0, strg)) wm.AddExtraFullBackup(pef(t, ebk2, 1, strg)) @@ -448,9 +448,9 @@ func TestRestoreSegmented(t *testing.T) { func TestFilteredOut(t *testing.T) { ctx := context.Background() strg := tmp(t) - id := identity() - ebk1 := extFullBkup(pfx("001"), id) - ebk2 := extFullBkup(pfx("002"), asIfTS(90), finished(), id) + id := makeID() + ebk1 := extFullBkup(prefix("001"), id) + ebk2 := extFullBkup(prefix("002"), asIfTS(90), finished(), id) wm := new(logclient.WithMigrations) wm.AddExtraFullBackup(pef(t, ebk1, 0, strg)) wm.AddExtraFullBackup(pef(t, ebk2, 1, strg)) @@ -462,13 +462,13 @@ func TestFilteredOut(t *testing.T) { func TestMultiRestores(t *testing.T) { ctx := context.Background() strg := tmp(t) - id := identity() - id2 := identity() + id := makeID() + id2 := makeID() - ebka1 := extFullBkup(pfx("001"), id) - ebkb1 := extFullBkup(pfx("101"), id2) - ebkb2 := extFullBkup(pfx("102"), asIfTS(88), finished(), id2) - ebka2 := extFullBkup(pfx("002"), asIfTS(90), finished(), id) + ebka1 := extFullBkup(prefix("001"), id) + ebkb1 := extFullBkup(prefix("101"), id2) + ebkb2 := extFullBkup(prefix("102"), asIfTS(88), finished(), id2) + ebka2 := extFullBkup(prefix("002"), asIfTS(90), finished(), id) wm := new(logclient.WithMigrations) wm.AddExtraFullBackup(pef(t, ebka1, 0, strg)) @@ -483,13 +483,13 @@ func TestMultiRestores(t *testing.T) { func TestMultiFilteredOutOne(t *testing.T) { ctx := context.Background() strg := tmp(t) - id := identity() - id2 := identity() + id := makeID() + id2 := makeID() - ebka1 := extFullBkup(pfx("001"), id) - ebkb1 := extFullBkup(pfx("101"), id2) - ebkb2 := extFullBkup(pfx("102"), asIfTS(88), finished(), id2) - ebka2 := extFullBkup(pfx("002"), asIfTS(90), finished(), id) + ebka1 := extFullBkup(prefix("001"), id) + ebkb1 := extFullBkup(prefix("101"), id2) + ebkb2 := extFullBkup(prefix("102"), asIfTS(88), finished(), id2) + ebka2 := extFullBkup(prefix("002"), asIfTS(90), finished(), id) wm := new(logclient.WithMigrations) wm.AddExtraFullBackup(pef(t, ebka1, 0, strg)) From 6d464c363c1a5af46b40d8d10a4c92ef655aa4c8 Mon Sep 17 00:00:00 2001 From: hillium Date: Tue, 10 Dec 2024 14:43:45 +0800 Subject: [PATCH 19/36] reject restore if there is a task for full restore Signed-off-by: hillium --- br/cmd/br/operator.go | 9 +++++---- br/pkg/streamhelper/daemon/owner_daemon.go | 2 +- br/pkg/task/common.go | 4 ++++ br/pkg/task/operator/checksum_table.go | 4 ++-- br/pkg/task/operator/config.go | 4 ++-- br/pkg/task/restore.go | 8 +++++--- tests/realtikvtest/brietest/pitr_test.go | 7 +++++++ 7 files changed, 26 insertions(+), 12 deletions(-) diff --git a/br/cmd/br/operator.go b/br/cmd/br/operator.go index ab8adb1a859b4..a578f3b39b6e1 100644 --- a/br/cmd/br/operator.go +++ b/br/cmd/br/operator.go @@ -113,11 +113,12 @@ func newMigrateToCommand() *cobra.Command { func newChecksumCommand() *cobra.Command { cmd := &cobra.Command{ - Use: "checksum ", - Short: "utilities for checksumming.", - Args: cobra.NoArgs, + Short: "Calculate the checksum of the current cluster (specified by `-u`) " + + "with applying the rewrite rules generated from a backup (specified by `-s`). " + + "This can be used when you have the checksum of upstream elsewhere.", + Args: cobra.NoArgs, RunE: func(cmd *cobra.Command, args []string) error { - cfg := operator.ChecksumTableConfig{} + cfg := operator.ChecksumWithRewriteRulesConfig{} if err := cfg.ParseFromFlags(cmd.Flags()); err != nil { return err } diff --git a/br/pkg/streamhelper/daemon/owner_daemon.go b/br/pkg/streamhelper/daemon/owner_daemon.go index 10b70a9a59580..5956b643c971d 100644 --- a/br/pkg/streamhelper/daemon/owner_daemon.go +++ b/br/pkg/streamhelper/daemon/owner_daemon.go @@ -89,7 +89,7 @@ func (od *OwnerDaemon) Begin(ctx context.Context) (func(), error) { zap.String("daemon-id", od.daemon.Name())) return case <-tick.C: - log.Info("daemon tick start", + log.Debug("daemon tick start", zap.Bool("is-owner", od.manager.IsOwner()), zap.String("daemon-id", od.daemon.Name())) if od.manager.IsOwner() { diff --git a/br/pkg/task/common.go b/br/pkg/task/common.go index 1813741634609..c95bee08c59d8 100644 --- a/br/pkg/task/common.go +++ b/br/pkg/task/common.go @@ -596,6 +596,10 @@ func (cfg *Config) normalizePDURLs() error { return nil } +func (cfg *Config) UserFiltered() bool { + return len(cfg.Schemas) != 0 || len(cfg.Tables) != 0 || len(cfg.FilterStr) != 0 +} + // ParseFromFlags parses the config from the flag set. func (cfg *Config) ParseFromFlags(flags *pflag.FlagSet) error { var err error diff --git a/br/pkg/task/operator/checksum_table.go b/br/pkg/task/operator/checksum_table.go index 19f0179810ea2..d93c71173dc9d 100644 --- a/br/pkg/task/operator/checksum_table.go +++ b/br/pkg/task/operator/checksum_table.go @@ -26,7 +26,7 @@ import ( ) type checksumTableCtx struct { - cfg ChecksumTableConfig + cfg ChecksumWithRewriteRulesConfig mgr *conn.Mgr dom *domain.Domain @@ -37,7 +37,7 @@ type tableInDB struct { dbName string } -func RunChecksumTable(ctx context.Context, g glue.Glue, cfg ChecksumTableConfig) error { +func RunChecksumTable(ctx context.Context, g glue.Glue, cfg ChecksumWithRewriteRulesConfig) error { c := &checksumTableCtx{cfg: cfg} if err := c.init(ctx, g); err != nil { diff --git a/br/pkg/task/operator/config.go b/br/pkg/task/operator/config.go index 882279a8a0bc9..6ddb6205489ab 100644 --- a/br/pkg/task/operator/config.go +++ b/br/pkg/task/operator/config.go @@ -197,7 +197,7 @@ func (cfg *ForceFlushConfig) ParseFromFlags(flags *pflag.FlagSet) error { return cfg.Config.ParseFromFlags(flags) } -type ChecksumTableConfig struct { +type ChecksumWithRewriteRulesConfig struct { task.Config } @@ -206,7 +206,7 @@ func DefineFlagsForChecksumTableConfig(f *pflag.FlagSet) { "including tableInfo/checksum and stats.") } -func (cfg *ChecksumTableConfig) ParseFromFlags(flags *pflag.FlagSet) (err error) { +func (cfg *ChecksumWithRewriteRulesConfig) ParseFromFlags(flags *pflag.FlagSet) (err error) { cfg.TableConcurrency, err = flags.GetUint(flagTableConcurrency) if err != nil { return diff --git a/br/pkg/task/restore.go b/br/pkg/task/restore.go index 1d696d05ceb5e..bd9ed74aa23a6 100644 --- a/br/pkg/task/restore.go +++ b/br/pkg/task/restore.go @@ -677,9 +677,11 @@ func RunRestore(c context.Context, g glue.Glue, cmdName string, cfg *RestoreConf log.Error("failed to close the etcd client", zap.Error(err)) } }() - // if err := checkTaskExists(c, cfg, etcdCLI); err != nil { - // return errors.Annotate(err, "failed to check task exists") - // } + if !cfg.UserFiltered() { + if err := checkTaskExists(c, cfg, etcdCLI); err != nil { + return errors.Annotate(err, "failed to check task exists") + } + } closeF, err := registerTaskToPD(c, etcdCLI) if err != nil { return errors.Annotate(err, "failed to register task to pd") diff --git a/tests/realtikvtest/brietest/pitr_test.go b/tests/realtikvtest/brietest/pitr_test.go index d1e6ca07671cf..1eb4e08baa111 100644 --- a/tests/realtikvtest/brietest/pitr_test.go +++ b/tests/realtikvtest/brietest/pitr_test.go @@ -99,6 +99,11 @@ func (kit *LogBackupKit) RunFullRestore(extConfig func(*task.RestoreConfig)) { kit.mustExec(func(ctx context.Context) error { cfg := task.DefaultRestoreConfig(task.DefaultConfig()) cfg.Storage = "local://" + kit.base + "/full" + cfg.FilterStr = []string{"test.*"} + var err error + cfg.TableFilter, err = filter.Parse(cfg.FilterStr) + cfg.CheckRequirements = false + require.NoError(kit.t, err) extConfig(&cfg) return task.RunRestore(ctx, kit.Glue(), task.FullRestoreCmd, &cfg) @@ -110,6 +115,7 @@ func (kit *LogBackupKit) RunStreamRestore(extConfig func(*task.RestoreConfig)) { cfg := task.DefaultRestoreConfig(task.DefaultConfig()) cfg.Storage = "local://" + kit.base + "/incr" cfg.FullBackupStorage = "local://" + kit.base + "/full" + cfg.CheckRequirements = false extConfig(&cfg) return task.RunRestore(ctx, kit.Glue(), task.PointRestoreCmd, &cfg) @@ -268,6 +274,7 @@ func TestPiTRAndBackup(t *testing.T) { kit.forceFlushAndWait(taskName) cleanSimpleData(kit) + kit.StopTaskIfExists(taskName) kit.RunStreamRestore(func(rc *task.RestoreConfig) { rc.FullBackupStorage = "local://" + kit.base + "/full2" }) From 6dd39469468520f9820e66119740ba87225ee89f Mon Sep 17 00:00:00 2001 From: hillium Date: Thu, 12 Dec 2024 16:00:49 +0800 Subject: [PATCH 20/36] added more test cases --- br/pkg/restore/log_client/migration.go | 94 ++----------- br/pkg/stream/stream_metas.go | 184 +++++++++++++++++++++---- br/pkg/stream/stream_metas_test.go | 183 +++++++++++++++++++++++- br/pkg/utils/iter/iter.go | 21 +++ 4 files changed, 369 insertions(+), 113 deletions(-) diff --git a/br/pkg/restore/log_client/migration.go b/br/pkg/restore/log_client/migration.go index 5aacc2473bb8e..2fb8ed4c34a8f 100644 --- a/br/pkg/restore/log_client/migration.go +++ b/br/pkg/restore/log_client/migration.go @@ -17,14 +17,10 @@ package logclient import ( "context" - "github.com/google/uuid" - "github.com/pingcap/errors" backuppb "github.com/pingcap/kvproto/pkg/brpb" - "github.com/pingcap/log" - 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/iter" - "go.uber.org/zap" ) type logicalSkipMap map[uint64]struct{} @@ -252,86 +248,12 @@ func (wm *WithMigrations) Compactions(ctx context.Context, s storage.ExternalSto } func (wm *WithMigrations) ExtraFullBackups(ctx context.Context, s storage.ExternalStorage) iter.TryNextor[*backuppb.ExtraFullBackup] { - fullBackupDirIter := iter.FromSlice(wm.fullBackups) - backups := iter.TryMap(fullBackupDirIter, func(name string) (*backuppb.ExtraFullBackup, error) { - // name is the absolute path in external storage. - bkup, err := readExtraFullBackup(ctx, name, s) - if err != nil { - return nil, errors.Annotatef(err, "failed to read backup at %s", name) - } - return bkup, nil + filteredOut := iter.FilterOut(stream.LoadExtraFullBackups(ctx, s, wm.fullBackups), func(ebk stream.ExtraFullBackups) bool { + return !ebk.GroupFinished() || ebk.GroupTS() > wm.restoredTS + }) + return iter.FlatMap(filteredOut, func(ebk stream.ExtraFullBackups) iter.TryNextor[*backuppb.ExtraFullBackup] { + return iter.Map(iter.FromSlice(ebk), func(p stream.PathedExtraFullBackup) *backuppb.ExtraFullBackup { + return p.ExtraFullBackup + }) }) - coll := extBackupCollector{ - inner: backups, - restoreTS: wm.restoredTS, - - collected: make(map[uuid.UUID][]*backuppb.ExtraFullBackup), - finished: make(map[uuid.UUID]struct{}), - } - return iter.FlatMap(&coll, iter.FromSlice) -} - -type extBackupCollector struct { - inner iter.TryNextor[*backuppb.ExtraFullBackup] - restoreTS uint64 - - collected map[uuid.UUID][]*backuppb.ExtraFullBackup - finished map[uuid.UUID]struct{} -} - -// Implement iter.TryNextor for extBackupCollector - -func (c *extBackupCollector) TryNext(ctx context.Context) iter.IterResult[[]*backuppb.ExtraFullBackup] { - for { - res := c.inner.TryNext(ctx) - if res.FinishedOrError() { - for id, fbks := range c.collected { - for _, fbk := range fbks { - log.Warn("Dropping unfinished extra full backup.", zap.Stringer("UUID", id), zap.String("path", fbk.FilesPrefixHint)) - } - } - return iter.DoneBy[[]*backuppb.ExtraFullBackup](res) - } - - fbk := res.Item - if len(fbk.BackupUuid) != len(uuid.UUID{}) { - return iter.Throw[[]*backuppb.ExtraFullBackup]( - errors.Annotatef(berrors.ErrInvalidArgument, "the full backup UUID has bad length(%d)", len(fbk.BackupUuid)), - ) - } - uid := uuid.UUID(fbk.BackupUuid) - log.Info("Collecting extra full backup", zap.Stringer("UUID", uid), zap.String("path", fbk.FilesPrefixHint), zap.Bool("finished", fbk.Finished)) - - if _, ok := c.finished[uid]; ok { - log.Warn("Encountered a finished full backup.", zap.Stringer("UUID", uid), zap.String("path", fbk.FilesPrefixHint)) - return iter.Emit([]*backuppb.ExtraFullBackup{fbk}) - } - - c.collected[uid] = append(c.collected[uid], fbk) - if fbk.Finished { - c.finished[uid] = struct{}{} - items := c.collected[uid] - delete(c.collected, uid) - - if fbk.AsIfTs > c.restoreTS { - log.Info("Filter out out-of-ts-range extra full backup.", - zap.Uint64("restoreTS", c.restoreTS), zap.Uint64("asIfTs", fbk.AsIfTs), zap.String("path", fbk.FilesPrefixHint)) - continue - } - return iter.Emit(items) - } - } -} - -func readExtraFullBackup(ctx context.Context, name string, s storage.ExternalStorage) (*backuppb.ExtraFullBackup, error) { - reader, err := s.ReadFile(ctx, name) - if err != nil { - return nil, err - } - - var backup backuppb.ExtraFullBackup - if err := backup.Unmarshal(reader); err != nil { - return nil, err - } - return &backup, nil } diff --git a/br/pkg/stream/stream_metas.go b/br/pkg/stream/stream_metas.go index 4f1c30c9ff317..271488a8683a3 100644 --- a/br/pkg/stream/stream_metas.go +++ b/br/pkg/stream/stream_metas.go @@ -7,6 +7,7 @@ import ( "encoding/binary" "fmt" "hash/crc64" + "maps" "math" "path" "slices" @@ -17,6 +18,7 @@ import ( "github.com/docker/go-units" "github.com/fatih/color" + "github.com/google/uuid" "github.com/pingcap/errors" pb "github.com/pingcap/kvproto/pkg/brpb" "github.com/pingcap/log" @@ -556,12 +558,16 @@ type MergeAndMigratedTo struct { // The term "migrate to" means, try to performance all possible operations // from a migration to the storage. type MigratedTo struct { - // Errors happen during executing the migration. + // Non-fatal errors happen during executing the migration. Warnings []error // The new BASE migration after the operation. NewBase *pb.Migration } +func (m *MigratedTo) Warn(err error) { + m.Warnings = append(m.Warnings, err) +} + // Migrations represents living migrations from the storage. type Migrations struct { // The BASE migration. @@ -883,14 +889,12 @@ func (m MigrationExt) migrateTo(ctx context.Context, mig *pb.Migration, opts ... NewBase: NewMigration(), } // Fills: EditMeta for new Base. - m.doMetaEdits(ctx, mig, &result) - // Fills: TruncatedTo, Compactions, DesctructPrefix. + m.processMetaEdits(ctx, mig, &result) + m.processCompactions(ctx, mig, &result) + m.processDestroyPrefixes(ctx, mig, &result) + m.processExtFullBackup(ctx, mig, &result) if !opt.skipTruncateLog { - m.doTruncating(ctx, mig, &result) - } else { - // Fast path: `truncate_to` wasn't updated, just copy the compactions and truncated to. - result.NewBase.Compactions = mig.Compactions - result.NewBase.TruncatedTo = mig.TruncatedTo + m.processTruncatedTo(ctx, mig, &result) } return result @@ -908,8 +912,8 @@ func (m MigrationExt) writeBase(ctx context.Context, mig *pb.Migration) error { return m.s.Rename(ctx, path.Join(m.prefix, baseTmp), path.Join(m.prefix, baseMigrationName)) } -// doMetaEdits applies the modification to the meta files in the storage. -func (m MigrationExt) doMetaEdits(ctx context.Context, mig *pb.Migration, out *MigratedTo) { +// processMetaEdits applies the modification to the meta files in the storage. +func (m MigrationExt) processMetaEdits(ctx context.Context, mig *pb.Migration, out *MigratedTo) { m.Hooks.StartHandlingMetaEdits(mig.EditMeta) handleAMetaEdit := func(medit *pb.MetaEdit) { @@ -1076,10 +1080,29 @@ func (m MigrationExt) tryRemovePrefix(ctx context.Context, pfx string, out *Migr } } -// doTruncating tries to remove outdated compaction, filling the not-yet removed compactions to the new migration. -func (m MigrationExt) doTruncating(ctx context.Context, mig *pb.Migration, result *MigratedTo) { - // NOTE: Execution of truncation wasn't implemented here. - // If we are going to truncate some files, for now we still need to use `br log truncate`. +// processTruncatedTo tries to remove outdated compaction, filling the not-yet removed compactions to the new migration. +func (m MigrationExt) processTruncatedTo(ctx context.Context, mig *pb.Migration, result *MigratedTo) { + result.NewBase.TruncatedTo = mig.TruncatedTo + m.Hooks.StartLoadingMetaForTruncating() + mdSet := new(StreamMetadataSet) + mdSet.MetadataDownloadBatchSize = 128 + shiftTS, err := mdSet.LoadUntilAndCalculateShiftTS(ctx, m.s, mig.TruncatedTo) + if err != nil { + result.Warnings = append(result.Warnings, errors.Annotatef(err, "failed to open meta storage")) + return + } + m.Hooks.EndLoadingMetaForTruncating() + + m.doTruncateLogs(ctx, mdSet, shiftTS, result) +} + +func (m MigrationExt) processDestroyPrefixes(ctx context.Context, mig *pb.Migration, result *MigratedTo) { + for _, pfx := range mig.DestructPrefix { + m.tryRemovePrefix(ctx, pfx, result) + } +} + +func (m MigrationExt) processCompactions(ctx context.Context, mig *pb.Migration, result *MigratedTo) { for _, compaction := range mig.Compactions { // Can we also remove the compaction when `until-ts` is equal to `truncated-to`...? if compaction.CompactionUntilTs > mig.TruncatedTo { @@ -1089,25 +1112,134 @@ func (m MigrationExt) doTruncating(ctx context.Context, mig *pb.Migration, resul m.tryRemovePrefix(ctx, compaction.GeneratedFiles, result) } } - for _, pfx := range mig.DestructPrefix { - m.tryRemovePrefix(ctx, pfx, result) +} + +func (m MigrationExt) processExtFullBackup(ctx context.Context, mig *pb.Migration, result *MigratedTo) { + groups := LoadExtraFullBackups(ctx, m.s, mig.ExtraFullBackupPaths) + processGroup := func(outErr error, e ExtraFullBackups) (copyToNewMig bool, err error) { + if outErr != nil { + return true, outErr + } + + if !e.GroupFinished() { + return true, nil + } + + if e.GroupTS() >= mig.TruncatedTo { + return true, nil + } + + for _, b := range e { + m.tryRemovePrefix(ctx, b.FilesPrefixHint, result) + } + return false, nil + } + for err, item := range iter.ToSeq(ctx, groups) { + copyToNewMig, err := processGroup(err, item) + if err != nil { + result.Warn(err) + } + if copyToNewMig { + for _, exb := range item { + result.NewBase.ExtraFullBackupPaths = append(result.NewBase.ExtraFullBackupPaths, exb.path) + } + } + } +} + +type PathedExtraFullBackup struct { + *pb.ExtraFullBackup + path string +} + +type ExtraFullBackups []PathedExtraFullBackup + +func (ebs ExtraFullBackups) GroupFinished() bool { + for _, b := range ebs { + if b.Finished { + return true + } } + return false +} - // TODO: Clean up the extra full backup SSTs. +func (ebs ExtraFullBackups) GroupTS() uint64 { + for _, b := range ebs { + if b.Finished { + return b.AsIfTs + } + } + return math.MaxUint64 +} - result.NewBase.TruncatedTo = mig.TruncatedTo +func LoadExtraFullBackups(ctx context.Context, s storage.ExternalStorage, paths []string) iter.TryNextor[ExtraFullBackups] { - m.Hooks.StartLoadingMetaForTruncating() - mdSet := new(StreamMetadataSet) - mdSet.MetadataDownloadBatchSize = 128 - shiftTS, err := mdSet.LoadUntilAndCalculateShiftTS(ctx, m.s, mig.TruncatedTo) + fullBackupDirIter := iter.FromSlice(paths) + backups := iter.TryMap(fullBackupDirIter, func(name string) (PathedExtraFullBackup, error) { + // name is the absolute path in external storage. + bkup, err := readExtraFullBackup(ctx, name, s) + if err != nil { + return PathedExtraFullBackup{}, errors.Annotatef(err, "failed to read backup at %s", name) + } + return PathedExtraFullBackup{ExtraFullBackup: bkup, path: name}, nil + }) + extBackups, err := groupExtraBackups(ctx, backups) if err != nil { - result.Warnings = append(result.Warnings, errors.Annotatef(err, "failed to open meta storage")) - return + return iter.Fail[ExtraFullBackups](err) } - m.Hooks.EndLoadingMetaForTruncating() + return iter.FromSlice(extBackups) +} - m.doTruncateLogs(ctx, mdSet, shiftTS, result) +func groupExtraBackups(ctx context.Context, i iter.TryNextor[PathedExtraFullBackup]) ([]ExtraFullBackups, error) { + var ( + collected = map[uuid.UUID]ExtraFullBackups{} + finished = map[uuid.UUID]struct{}{} + ) + + for { + res := i.TryNext(ctx) + if res.FinishedOrError() { + res := make([]ExtraFullBackups, 0, len(collected)) + for v := range maps.Values(collected) { + res = append(res, v) + } + return res, nil + } + + fbk := res.Item + if len(fbk.BackupUuid) != len(uuid.UUID{}) { + return nil, errors.Annotatef(berrors.ErrInvalidArgument, "the full backup UUID has bad length(%d)", len(fbk.BackupUuid)) + } + uid := uuid.UUID(fbk.BackupUuid) + log.Info("Collecting extra full backup", zap.Stringer("UUID", uid), zap.String("path", fbk.FilesPrefixHint), zap.Bool("finished", fbk.Finished)) + + if _, ok := finished[uid]; ok { + log.Warn("Encountered a finished full backup.", zap.Stringer("UUID", uid), zap.String("path", fbk.FilesPrefixHint)) + return nil, errors.Annotatef( + berrors.ErrInvalidArgument, + "the extra full backup group %s at %s encounters an extra full backup meta after a finished one", + uid, fbk.FilesPrefixHint, + ) + } + + collected[uid] = append(collected[uid], fbk) + if fbk.Finished { + finished[uid] = struct{}{} + } + } +} + +func readExtraFullBackup(ctx context.Context, name string, s storage.ExternalStorage) (*pb.ExtraFullBackup, error) { + reader, err := s.ReadFile(ctx, name) + if err != nil { + return nil, err + } + + var backup pb.ExtraFullBackup + if err := backup.Unmarshal(reader); err != nil { + return nil, err + } + return &backup, nil } func (m MigrationExt) loadFilesOfPrefix(ctx context.Context, prefix string) (out []string, err error) { diff --git a/br/pkg/stream/stream_metas_test.go b/br/pkg/stream/stream_metas_test.go index 52f1d003e144f..d01d94f37a6c6 100644 --- a/br/pkg/stream/stream_metas_test.go +++ b/br/pkg/stream/stream_metas_test.go @@ -16,6 +16,7 @@ import ( "testing" "github.com/fsouza/fake-gcs-server/fakestorage" + "github.com/google/uuid" "github.com/pingcap/errors" "github.com/pingcap/failpoint" backuppb "github.com/pingcap/kvproto/pkg/brpb" @@ -23,6 +24,7 @@ import ( "github.com/pingcap/tidb/br/pkg/storage" "github.com/pingcap/tidb/pkg/util/intest" "github.com/stretchr/testify/require" + "go.uber.org/multierr" "go.uber.org/zap" "golang.org/x/exp/maps" ) @@ -573,6 +575,53 @@ func TestReplaceMetadataTs(t *testing.T) { require.Equal(t, m.MaxTs, uint64(4)) } +func pef(t *testing.T, fb *backuppb.ExtraFullBackup, sn int, s storage.ExternalStorage) string { + path := fmt.Sprintf("extbackupmeta_%08d", sn) + bs, err := fb.Marshal() + if err != nil { + require.NoError(t, err) + } + + err = s.WriteFile(context.Background(), path, bs) + require.NoError(t, err) + return path +} + +type efOP func(*backuppb.ExtraFullBackup) + +func extFullBkup(ops ...efOP) *backuppb.ExtraFullBackup { + ef := &backuppb.ExtraFullBackup{} + for _, op := range ops { + op(ef) + } + return ef +} + +func finished() efOP { + return func(ef *backuppb.ExtraFullBackup) { + ef.Finished = true + } +} + +func makeID() efOP { + id := uuid.New() + return func(ef *backuppb.ExtraFullBackup) { + ef.BackupUuid = id[:] + } +} + +func prefix(pfx string) efOP { + return func(ef *backuppb.ExtraFullBackup) { + ef.FilesPrefixHint = pfx + } +} + +func asIfTS(ts uint64) efOP { + return func(ef *backuppb.ExtraFullBackup) { + ef.AsIfTs = ts + } +} + func m(storeId int64, minTS, maxTS uint64) *backuppb.Metadata { return &backuppb.Metadata{ StoreId: storeId, @@ -584,6 +633,12 @@ func m(storeId int64, minTS, maxTS uint64) *backuppb.Metadata { type migOP func(*backuppb.Migration) +func mExtFullBackup(path ...string) migOP { + return func(m *backuppb.Migration) { + m.ExtraFullBackupPaths = append(m.ExtraFullBackupPaths, path...) + } +} + func mDstrPfx(path ...string) migOP { return func(m *backuppb.Migration) { m.DestructPrefix = append(m.DestructPrefix, path...) @@ -749,7 +804,7 @@ func tmp(t *testing.T) *storage.LocalStorage { } func mig(ops ...migOP) *backuppb.Migration { - mig := &backuppb.Migration{} + mig := NewMigration() for _, op := range ops { op(mig) } @@ -2854,3 +2909,129 @@ func TestCreator(t *testing.T) { require.Contains(t, mig.Creator, "br") require.Equal(t, mig.Version, SupportedMigVersion) } + +func TestGroupedExtFullBackup(t *testing.T) { + ctx := context.Background() + s := tmp(t) + placeholder := func(pfx string) string { + path := path.Join(pfx, "monolith") + require.NoError(t, s.WriteFile(ctx, path, []byte("🪨"))) + return path + } + idx := 0 + somewhere := func() string { + idx += 1 + return placeholder(fmt.Sprintf("%06d", idx)) + } + + type Case struct { + InputGroups []*backuppb.ExtraFullBackup + TruncatedTo uint64 + + RequireRem []int + } + + cases := []Case{ + { + InputGroups: []*backuppb.ExtraFullBackup{ + extFullBkup(prefix(somewhere()), asIfTS(10), makeID(), finished()), + extFullBkup(prefix(somewhere()), asIfTS(12), makeID(), finished()), + }, + TruncatedTo: 11, + RequireRem: []int{1}, + }, + { + InputGroups: []*backuppb.ExtraFullBackup{ + extFullBkup(prefix(somewhere()), asIfTS(10), makeID(), finished()), + extFullBkup(prefix(somewhere()), asIfTS(12), makeID(), finished()), + }, + TruncatedTo: 13, + RequireRem: []int{}, + }, + { + InputGroups: []*backuppb.ExtraFullBackup{ + extFullBkup(prefix(somewhere()), asIfTS(10), makeID(), finished()), + extFullBkup(prefix(somewhere()), asIfTS(12), makeID(), finished()), + }, + TruncatedTo: 10, + RequireRem: []int{0, 1}, + }, + { + InputGroups: func() []*backuppb.ExtraFullBackup { + id := makeID() + return []*backuppb.ExtraFullBackup{ + extFullBkup(prefix(somewhere()), id), + extFullBkup(prefix(somewhere()), asIfTS(10), id, finished()), + extFullBkup(prefix(somewhere()), asIfTS(12), makeID(), finished()), + } + }(), + TruncatedTo: 11, + RequireRem: []int{2}, + }, + { + InputGroups: func() []*backuppb.ExtraFullBackup { + id := makeID() + return []*backuppb.ExtraFullBackup{ + extFullBkup(prefix(somewhere()), id), + extFullBkup(prefix(somewhere()), asIfTS(12), id, finished()), + extFullBkup(prefix(somewhere()), asIfTS(10), makeID(), finished()), + } + }(), + TruncatedTo: 11, + RequireRem: []int{0, 1}, + }, + { + InputGroups: func() []*backuppb.ExtraFullBackup { + id := makeID() + return []*backuppb.ExtraFullBackup{ + extFullBkup(prefix(somewhere()), asIfTS(999), id), + extFullBkup(prefix(somewhere()), asIfTS(10), id, finished()), + extFullBkup(prefix(somewhere()), asIfTS(12), makeID(), finished()), + } + }(), + TruncatedTo: 11, + RequireRem: []int{2}, + }, + { + InputGroups: []*backuppb.ExtraFullBackup{ + extFullBkup(prefix(somewhere()), asIfTS(10), makeID()), + extFullBkup(prefix(somewhere()), asIfTS(12), makeID()), + extFullBkup(prefix(somewhere()), asIfTS(14), makeID()), + }, + TruncatedTo: 11, + RequireRem: []int{0, 1, 2}, + }, + } + + for i, c := range cases { + t.Run(fmt.Sprintf("#%d", i), func(t *testing.T) { + m := mig() + paths := []PathedExtraFullBackup{} + for i, input := range c.InputGroups { + p := pef(t, input, i, s) + paths = append(paths, PathedExtraFullBackup{ + path: p, + ExtraFullBackup: input, + }) + mExtFullBackup(p)(m) + require.FileExists(t, path.Join(s.Base(), input.FilesPrefixHint)) + } + mTruncatedTo(c.TruncatedTo)(m) + est := MigrationExtension(s) + res := est.migrateTo(ctx, m) + require.NoError(t, multierr.Combine(res.Warnings...)) + chosen := []string{} + nonChosen := []PathedExtraFullBackup{} + forgottenIdx := 0 + for _, i := range c.RequireRem { + chosen = append(chosen, paths[i].path) + nonChosen = append(nonChosen, paths[forgottenIdx:i]...) + forgottenIdx = i + 1 + } + require.ElementsMatch(t, chosen, res.NewBase.ExtraFullBackupPaths) + for _, p := range nonChosen { + require.NoFileExists(t, path.Join(s.Base(), p.FilesPrefixHint, "monolith")) + } + }) + } +} diff --git a/br/pkg/utils/iter/iter.go b/br/pkg/utils/iter/iter.go index 6f8f280905c2c..0a925fdea5f65 100644 --- a/br/pkg/utils/iter/iter.go +++ b/br/pkg/utils/iter/iter.go @@ -5,6 +5,7 @@ package iter import ( "context" "fmt" + goiter "iter" ) // IterResult is the result of try to advancing an impure iterator. @@ -121,3 +122,23 @@ func Tap[T any](i TryNextor[T], with func(T)) TryNextor[T] { tapper: with, } } + +// ToSeq wraps an `TryNextor` to a Seq2. +func ToSeq[T any](ctx context.Context, i TryNextor[T]) goiter.Seq2[error, T] { + return func(yield func(error, T) bool) { + for { + res := i.TryNext(ctx) + cont := true + if res.Err != nil { + cont = yield(res.Err, *new(T)) + } else if res.Finished { + cont = false + } else { + cont = yield(nil, res.Item) + } + if !cont { + break + } + } + } +} From 547e1923f212e56b68d20d9e0ad1a5b2ea097597 Mon Sep 17 00:00:00 2001 From: hillium Date: Thu, 12 Dec 2024 18:26:25 +0800 Subject: [PATCH 21/36] try not allow user when local encryption enabled, add progress for SSTs --- .../log_client/compacted_file_strategy.go | 14 ++-- br/pkg/restore/log_client/log_file_manager.go | 12 +++ br/pkg/restore/log_client/ssts.go | 22 ++++++ br/pkg/restore/snap_client/pitr_collector.go | 75 ------------------- br/pkg/stream/stream_metas.go | 2 +- br/pkg/task/restore.go | 11 ++- br/pkg/task/stream.go | 19 +++-- br/pkg/utils/iter/iter.go | 4 +- 8 files changed, 65 insertions(+), 94 deletions(-) diff --git a/br/pkg/restore/log_client/compacted_file_strategy.go b/br/pkg/restore/log_client/compacted_file_strategy.go index 0c9a8a7a81272..80613512ee745 100644 --- a/br/pkg/restore/log_client/compacted_file_strategy.go +++ b/br/pkg/restore/log_client/compacted_file_strategy.go @@ -116,11 +116,13 @@ func (cs *CompactedFileSplitStrategy) ShouldSkip(ssts SSTs) bool { log.Info("all files in sub compaction skipped") return true } - // TODO: Add a method for removing it. - // if len(sstOutputs) != len(subCompaction.SstOutputs) { - // log.Info("partial files in sub compaction skipped due to checkpoint") - // subCompaction.SstOutputs = sstOutputs - // return false - // } + if len(sstOutputs) != len(ssts.GetSSTs()) { + log.Info( + "partial files in sub compaction skipped due to checkpoint", + zap.Int("origin", len(ssts.GetSSTs())), zap.Int("output", len(sstOutputs)), + ) + ssts.SetSSTs(sstOutputs) + return false + } return false } diff --git a/br/pkg/restore/log_client/log_file_manager.go b/br/pkg/restore/log_client/log_file_manager.go index 351fbbbe22e1d..f90a4da663ebb 100644 --- a/br/pkg/restore/log_client/log_file_manager.go +++ b/br/pkg/restore/log_client/log_file_manager.go @@ -373,6 +373,18 @@ func (rc *LogFileManager) GetExtraFullBackupSSTs(ctx context.Context) iter.TryNe }) } +func (rc *LogFileManager) CountExtraSSTs(ctx context.Context) (int, error) { + count := 0 + ssts := iter.ConcatAll(rc.GetCompactionIter(ctx), rc.GetExtraFullBackupSSTs(ctx)) + for err, sst := range iter.AsSeq(ctx, ssts) { + if err != nil { + return 0, errors.Trace(err) + } + count += len(sst.GetSSTs()) + } + return count +} + // the kv entry with ts, the ts is decoded from entry. type KvEntryWithTS struct { E kv.Entry diff --git a/br/pkg/restore/log_client/ssts.go b/br/pkg/restore/log_client/ssts.go index fc8aa8f26d81a..f8d2a155c9f6e 100644 --- a/br/pkg/restore/log_client/ssts.go +++ b/br/pkg/restore/log_client/ssts.go @@ -5,10 +5,12 @@ package logclient import ( "encoding/hex" "fmt" + "log" "sync/atomic" backuppb "github.com/pingcap/kvproto/pkg/brpb" "github.com/pingcap/tidb/pkg/tablecodec" + "go.uber.org/zap" ) var ( @@ -36,6 +38,9 @@ type SSTs interface { TableID() int64 // GetSSTs returns a slice of pointers to backuppb.File, representing the SST files. GetSSTs() []*backuppb.File + // SetSSTs allows the user to override the internal SSTs to be restored. + // The input SST set should already be a subset of `GetSSTs.` + SetSSTs([]*backuppb.File) } type CompactedSSTs struct { @@ -50,6 +55,10 @@ func (s *CompactedSSTs) GetSSTs() []*backuppb.File { return s.SstOutputs } +func (s *CompactedSSTs) SetSSTs(files []*backuppb.File) { + s.SstOutputs = files +} + type AddedSSTs struct { File *backuppb.File Rewritten backuppb.RewrittenTableID @@ -79,9 +88,22 @@ func (s *AddedSSTs) TableID() int64 { } func (s *AddedSSTs) GetSSTs() []*backuppb.File { + if s.File == nil { + return nil + } return []*backuppb.File{s.File} } +func (s *AddedSSTs) SetSSTs(fs []*backuppb.File) { + if len(fs) == 0 { + s.File = nil + } + if len(fs) == 1 { + s.File = fs[0] + } + log.Panic("Too many files passed to AddedSSTs.SetSSTs.", zap.Any("input", fs)) +} + func (s *AddedSSTs) RewrittenTo() int64 { return s.Rewritten.Upstream } diff --git a/br/pkg/restore/snap_client/pitr_collector.go b/br/pkg/restore/snap_client/pitr_collector.go index e0673ddb080f1..bef42b9fdfc99 100644 --- a/br/pkg/restore/snap_client/pitr_collector.go +++ b/br/pkg/restore/snap_client/pitr_collector.go @@ -25,81 +25,6 @@ import ( "golang.org/x/sync/errgroup" ) -type pitrCollectorRestorer struct { - // the context used for committing. - cx context.Context - // the context bound to the errgroup. - ecx context.Context - - coll *pitrCollector - wg *errgroup.Group -} - -// wrapRestorer wraps a restorer and the restorer will upload the SST file to the collector during restoring. -func (c *pitrCollector) createRestorer(ctx context.Context) *pitrCollectorRestorer { - wg, ecx := errgroup.WithContext(ctx) - return &pitrCollectorRestorer{ - cx: ctx, - ecx: ecx, - coll: c, - wg: wg, - } -} - -// GoRestore imports the specified backup file sets into TiKV asynchronously. -// The onProgress function is called with progress updates as files are processed. -func (p pitrCollectorRestorer) GoRestore(onProgress func(int64), batchFileSets ...restore.BatchBackupFileSet) error { - if !p.coll.enabled { - return nil - } - - if err := p.coll.prepareMigIfNeeded(p.cx); err != nil { - return err - } - - p.wg.Go(func() error { - for _, fileSets := range batchFileSets { - if err := p.coll.onBatch(p.cx, fileSets); err != nil { - return err - } - } - return nil - }) - return nil -} - -// WaitUntilFinish blocks until all pending restore files have completed processing. -func (p pitrCollectorRestorer) WaitUntilFinish() error { - if !p.coll.enabled { - return nil - } - err := p.wg.Wait() - if err != nil { - return errors.Annotate(err, "failed to wait on pitrCollector") - } - return errors.Annotatef(p.coll.persistExtraBackupMeta(p.cx), "failed to persist the metadata of uploaded SSTs") -} - -// Close releases any resources associated with the restoration process. -func (p pitrCollectorRestorer) Close() error { - if p.cx.Err() != nil { - log.Warn("pitrCollectorRestorer.Close: the context already closed, will use a temporary context.") - var cancel context.CancelFunc - p.cx, cancel = context.WithTimeout(context.Background(), 30*time.Second) - defer cancel() - } - - if !p.coll.enabled { - return nil - } - if summary.OnceSucceed() { - return errors.Annotate(p.coll.commit(p.cx), "failed to commit pitrCollector") - } - - log.Warn("Backup not success, put a half-finished metadata to the log backup.", zap.Stringer("uuid", p.coll.restoreUUID)) - return errors.Annotatef(p.coll.persistExtraBackupMeta(p.cx), "failed to persist the meta") -} - type pitrCollector struct { // Immutable state. taskStorage storage.ExternalStorage diff --git a/br/pkg/stream/stream_metas.go b/br/pkg/stream/stream_metas.go index 271488a8683a3..11a9888d740ab 100644 --- a/br/pkg/stream/stream_metas.go +++ b/br/pkg/stream/stream_metas.go @@ -1134,7 +1134,7 @@ func (m MigrationExt) processExtFullBackup(ctx context.Context, mig *pb.Migratio } return false, nil } - for err, item := range iter.ToSeq(ctx, groups) { + for err, item := range iter.AsSeq(ctx, groups) { copyToNewMig, err := processGroup(err, item) if err != nil { result.Warn(err) diff --git a/br/pkg/task/restore.go b/br/pkg/task/restore.go index bd9ed74aa23a6..9672915328f71 100644 --- a/br/pkg/task/restore.go +++ b/br/pkg/task/restore.go @@ -16,6 +16,7 @@ import ( "github.com/opentracing/opentracing-go" "github.com/pingcap/errors" backuppb "github.com/pingcap/kvproto/pkg/brpb" + "github.com/pingcap/kvproto/pkg/encryptionpb" "github.com/pingcap/log" "github.com/pingcap/tidb/br/pkg/checkpoint" pconfig "github.com/pingcap/tidb/br/pkg/config" @@ -277,6 +278,10 @@ type RestoreConfig struct { UseFSR bool `json:"use-fsr" toml:"use-fsr"` } +func (r *RestoreConfig) LocalEncryptionEnabled() bool { + return r.MasterKeyConfig.EncryptionType != encryptionpb.EncryptionMethod_PLAINTEXT +} + // DefineRestoreFlags defines common flags for the restore tidb command. func DefineRestoreFlags(flags *pflag.FlagSet) { flags.Bool(flagNoSchema, false, "skip creating schemas and tables, reuse existing empty ones") @@ -677,10 +682,8 @@ func RunRestore(c context.Context, g glue.Glue, cmdName string, cfg *RestoreConf log.Error("failed to close the etcd client", zap.Error(err)) } }() - if !cfg.UserFiltered() { - if err := checkTaskExists(c, cfg, etcdCLI); err != nil { - return errors.Annotate(err, "failed to check task exists") - } + if err := checkConflictingLogBackup(c, cfg, etcdCLI); err != nil { + return errors.Annotate(err, "failed to check task exists") } closeF, err := registerTaskToPD(c, etcdCLI) if err != nil { diff --git a/br/pkg/task/stream.go b/br/pkg/task/stream.go index 140de7c369da7..ae3281fdea925 100644 --- a/br/pkg/task/stream.go +++ b/br/pkg/task/stream.go @@ -1128,9 +1128,9 @@ func RunStreamTruncate(c context.Context, g glue.Glue, cmdName string, cfg *Stre return nil } -// checkTaskExists checks whether there is a log backup task running. +// checkConflictingLogBackup checks whether there is a log backup task running. // If so, return an error. -func checkTaskExists(ctx context.Context, cfg *RestoreConfig, etcdCLI *clientv3.Client) error { +func checkConflictingLogBackup(ctx context.Context, cfg *RestoreConfig, etcdCLI *clientv3.Client) error { if err := checkConfigForStatus(cfg.PD); err != nil { return err } @@ -1141,7 +1141,8 @@ func checkTaskExists(ctx context.Context, cfg *RestoreConfig, etcdCLI *clientv3. if err != nil { return err } - if len(tasks) > 0 { + exempted := cfg.UserFiltered() && !cfg.LocalEncryptionEnabled() + if len(tasks) > 0 && !exempted { return errors.Errorf("log backup task is running: %s, "+ "please stop the task before restore, and after PITR operation finished, "+ "create log-backup task again and create a full backup on this cluster", tasks[0].Info.Name) @@ -1457,9 +1458,11 @@ func restoreStream( return errors.Trace(err) } - addedSSTsIter := client.LogFileManager.GetExtraFullBackupSSTs(ctx) - compactionIter := client.LogFileManager.GetCompactionIter(ctx) - sstsIter := iter.ConcatAll(addedSSTsIter, compactionIter) + count, err := client.LogFileManager.CountExtraSSTs(ctx) + if err != nil { + return err + } + logclient.TotalEntryCount += int64(count) se, err := g.CreateSession(mgr.GetStorage()) if err != nil { @@ -1469,6 +1472,10 @@ func restoreStream( splitSize, splitKeys := utils.GetRegionSplitInfo(execCtx) log.Info("[Log Restore] get split threshold from tikv config", zap.Uint64("split-size", splitSize), zap.Int64("split-keys", splitKeys)) + addedSSTsIter := client.LogFileManager.GetExtraFullBackupSSTs(ctx) + compactionIter := client.LogFileManager.GetCompactionIter(ctx) + sstsIter := iter.ConcatAll(addedSSTsIter, compactionIter) + pd := g.StartProgress(ctx, "Restore Files(SST + KV)", logclient.TotalEntryCount, !cfg.LogProgress) err = withProgress(pd, func(p glue.Progress) (pErr error) { updateStatsWithCheckpoint := func(kvCount, size uint64) { diff --git a/br/pkg/utils/iter/iter.go b/br/pkg/utils/iter/iter.go index 0a925fdea5f65..f1207cdec3b3b 100644 --- a/br/pkg/utils/iter/iter.go +++ b/br/pkg/utils/iter/iter.go @@ -123,8 +123,8 @@ func Tap[T any](i TryNextor[T], with func(T)) TryNextor[T] { } } -// ToSeq wraps an `TryNextor` to a Seq2. -func ToSeq[T any](ctx context.Context, i TryNextor[T]) goiter.Seq2[error, T] { +// AsSeq wraps an `TryNextor` to a Seq2. +func AsSeq[T any](ctx context.Context, i TryNextor[T]) goiter.Seq2[error, T] { return func(yield func(error, T) bool) { for { res := i.TryNext(ctx) From 9e0dfd7e5fd5c772b33b06e92b866a6a99eded8c Mon Sep 17 00:00:00 2001 From: hillium Date: Fri, 13 Dec 2024 14:48:42 +0800 Subject: [PATCH 22/36] make a more friendly error message --- br/pkg/restore/log_client/log_file_manager.go | 2 +- br/pkg/restore/snap_client/pitr_collector.go | 23 +++++++++------ br/pkg/summary/summary.go | 4 +-- br/pkg/task/stream.go | 28 +++++++++++++++---- 4 files changed, 41 insertions(+), 16 deletions(-) diff --git a/br/pkg/restore/log_client/log_file_manager.go b/br/pkg/restore/log_client/log_file_manager.go index f90a4da663ebb..d3e87201874db 100644 --- a/br/pkg/restore/log_client/log_file_manager.go +++ b/br/pkg/restore/log_client/log_file_manager.go @@ -382,7 +382,7 @@ func (rc *LogFileManager) CountExtraSSTs(ctx context.Context) (int, error) { } count += len(sst.GetSSTs()) } - return count + return count, nil } // the kv entry with ts, the ts is decoded from entry. diff --git a/br/pkg/restore/snap_client/pitr_collector.go b/br/pkg/restore/snap_client/pitr_collector.go index bef42b9fdfc99..fe22df4119029 100644 --- a/br/pkg/restore/snap_client/pitr_collector.go +++ b/br/pkg/restore/snap_client/pitr_collector.go @@ -63,13 +63,20 @@ func (c *pitrCollector) close() error { cx, cancel := context.WithTimeout(context.Background(), 30*time.Second) defer cancel() - if summary.OnceSucceed() { - return errors.Annotate(c.commit(cx), "failed to commit pitrCollector") + if !summary.Succeed() { + log.Warn("Backup not success, put a half-finished metadata to the log backup.", + zap.Stringer("uuid", c.restoreUUID)) + return errors.Annotatef(c.persistExtraBackupMeta(cx), "failed to persist the meta") } - log.Warn("Backup not success, put a half-finished metadata to the log backup.", - zap.Stringer("uuid", c.restoreUUID)) - return errors.Annotatef(c.persistExtraBackupMeta(cx), "failed to persist the meta") + commitTS, err := c.commit(cx) + if err != nil { + return errors.Annotate(err, "failed to commit pitrCollector") + } + log.Info("Log backup SSTs are committed.", + zap.Uint64("commitTS", commitTS), zap.String("committedTo", c.outputPath())) + return nil + } func (c *pitrCollector) onBatch(ctx context.Context, fileSets restore.BatchBackupFileSet) error { @@ -226,14 +233,14 @@ func (c *pitrCollector) prepareMigIfNeeded(ctx context.Context) (err error) { return } -func (c *pitrCollector) commit(ctx context.Context) error { +func (c *pitrCollector) commit(ctx context.Context) (uint64, error) { c.extraBackupMeta.msg.Finished = true ts, err := c.tso(ctx) if err != nil { - return err + return 0, err } c.extraBackupMeta.msg.AsIfTs = ts - return c.persistExtraBackupMeta(ctx) + return ts, c.persistExtraBackupMeta(ctx) } func (c *pitrCollector) resetCommitting() { diff --git a/br/pkg/summary/summary.go b/br/pkg/summary/summary.go index 3b32f9cd79b35..5e970cfa55b4e 100644 --- a/br/pkg/summary/summary.go +++ b/br/pkg/summary/summary.go @@ -51,8 +51,8 @@ func SetSuccessStatus(success bool) { collector.SetSuccessStatus(success) } -// OnceSucceed returns whether a `SetSuccessStatus(true)` was call. -func OnceSucceed() bool { +// Succeed returns whether a `SetSuccessStatus(true)` was call. +func Succeed() bool { return onceSetToSuccess.Load() } diff --git a/br/pkg/task/stream.go b/br/pkg/task/stream.go index ae3281fdea925..0493e6f96a610 100644 --- a/br/pkg/task/stream.go +++ b/br/pkg/task/stream.go @@ -1141,16 +1141,34 @@ func checkConflictingLogBackup(ctx context.Context, cfg *RestoreConfig, etcdCLI if err != nil { return err } - exempted := cfg.UserFiltered() && !cfg.LocalEncryptionEnabled() - if len(tasks) > 0 && !exempted { - return errors.Errorf("log backup task is running: %s, "+ - "please stop the task before restore, and after PITR operation finished, "+ - "create log-backup task again and create a full backup on this cluster", tasks[0].Info.Name) + for _, task := range tasks { + if err := checkTaskCompat(cfg, task); err != nil { + return err + } } return nil } +func checkTaskCompat(cfg *RestoreConfig, task streamhelper.Task) error { + baseErr := errors.Errorf("log backup task is running: %s, and isn't compatible with your restore."+ + "You may check the extra information to get rid of this. If that doesn't work, you may "+ + "stop the task before restore, and after PITR operation finished, "+ + "create log-backup task again and create a full backup on this cluster.", task.Info.Name) + if !cfg.UserFiltered() { + return errors.Annotate(baseErr, + "you want to restore a whole cluster, you may use `-f` or `restore table|database` to "+ + "specify the tables to restore to continue") + } + if !cfg.LocalEncryptionEnabled() { + return errors.Annotate(baseErr, "the data you want to restore is encrypted, they cannot be copied to the log storage") + } + if task.Info.SecurityConfig != nil { + return errors.Annotate(baseErr, "the running log backup task is encrypted, the data copied to the log storage cannot work") + } + return nil +} + func checkIncompatibleChangefeed(ctx context.Context, backupTS uint64, etcdCLI *clientv3.Client) error { nameSet, err := cdcutil.GetIncompatibleChangefeedsWithSafeTS(ctx, etcdCLI, backupTS) if err != nil { From 8f00ef596e5da15beae976264bda8f2fe3f32f02 Mon Sep 17 00:00:00 2001 From: hillium Date: Fri, 13 Dec 2024 16:08:37 +0800 Subject: [PATCH 23/36] added number of SSTs --- br/pkg/restore/log_client/client.go | 35 +++++++++------- br/pkg/restore/log_client/log_file_manager.go | 41 +++++++++++++++---- br/pkg/restore/restorer.go | 27 ------------ br/pkg/task/stream.go | 6 +-- 4 files changed, 58 insertions(+), 51 deletions(-) diff --git a/br/pkg/restore/log_client/client.go b/br/pkg/restore/log_client/client.go index 383316883a932..2519aeb822fc1 100644 --- a/br/pkg/restore/log_client/client.go +++ b/br/pkg/restore/log_client/client.go @@ -183,6 +183,7 @@ func NewSstRestoreManager( type LogClient struct { *LogFileManager + logRestoreManager *LogRestoreManager sstRestoreManager *SstRestoreManager @@ -216,10 +217,15 @@ type LogClient struct { // checkpoint information for log restore useCheckpoint bool - restoreSSTKVSize atomic.Uint64 - restoreSSTKVCount atomic.Uint64 - restoreSSTPhySize atomic.Uint64 - restoreSSTTakes atomic.Uint64 + logFilesStat *logFilesStatistic + restoreStat *restoreStatistic +} + +type restoreStatistic struct { + restoreSSTKVSize uint64 + restoreSSTKVCount uint64 + restoreSSTPhySize uint64 + restoreSSTTakes uint64 } // NewRestoreClient returns a new RestoreClient. @@ -341,24 +347,23 @@ func (rc *LogClient) RestoreCompactedSstFiles( for _, files := range backupFileSets { for _, f := range files.SSTFiles { log.Info("Collected file.", zap.Uint64("total_kv", f.TotalKvs), zap.Uint64("total_bytes", f.TotalBytes), zap.Uint64("size", f.Size_)) - rc.restoreSSTKVCount.Add(f.TotalKvs) - rc.restoreSSTKVSize.Add(f.TotalBytes) - rc.restoreSSTPhySize.Add(f.Size_) + atomic.AddUint64(&rc.restoreStat.restoreSSTKVCount, f.TotalKvs) + atomic.AddUint64(&rc.restoreStat.restoreSSTKVSize, f.TotalBytes) + atomic.AddUint64(&rc.restoreStat.restoreSSTPhySize, f.Size_) } } - rc.restoreSSTTakes.Add(uint64(time.Since(begin))) - + atomic.AddUint64(&rc.restoreStat.restoreSSTTakes, uint64(time.Since(begin))) return err } func (rc *LogClient) RestoreSSTStatisticFields(pushTo *[]zapcore.Field) { - takes := time.Duration(rc.restoreSSTTakes.Load()) + takes := time.Duration(rc.restoreStat.restoreSSTTakes) fields := []zapcore.Field{ - zap.Uint64("restore-sst-kv-count", rc.restoreSSTKVCount.Load()), - zap.Uint64("restore-sst-kv-size", rc.restoreSSTKVSize.Load()), - zap.Uint64("restore-sst-physical-size (after compression)", rc.restoreSSTPhySize.Load()), + zap.Uint64("restore-sst-kv-count", rc.restoreStat.restoreSSTKVCount), + zap.Uint64("restore-sst-kv-size", rc.restoreStat.restoreSSTKVSize), + zap.Uint64("restore-sst-physical-size (after compression)", rc.restoreStat.restoreSSTPhySize), zap.Duration("restore-sst-total-take", takes), - zap.String("average-speed (sst)", units.HumanSize(float64(rc.restoreSSTKVSize.Load())/takes.Seconds())+"/s"), + zap.String("average-speed (sst)", units.HumanSize(float64(rc.restoreStat.restoreSSTKVSize)/takes.Seconds())+"/s"), } *pushTo = append(*pushTo, fields...) } @@ -590,6 +595,8 @@ func (rc *LogClient) InstallLogFileManager(ctx context.Context, startTS, restore if err != nil { return err } + rc.logFilesStat = new(logFilesStatistic) + rc.LogFileManager.Stats = rc.logFilesStat return nil } diff --git a/br/pkg/restore/log_client/log_file_manager.go b/br/pkg/restore/log_client/log_file_manager.go index d3e87201874db..50ac3ef80fbf8 100644 --- a/br/pkg/restore/log_client/log_file_manager.go +++ b/br/pkg/restore/log_client/log_file_manager.go @@ -9,6 +9,7 @@ import ( "fmt" "strings" "sync" + "sync/atomic" "time" "github.com/pingcap/errors" @@ -27,8 +28,6 @@ import ( "go.uber.org/zap" ) -var TotalEntryCount int64 - // MetaIter is the type of iterator of metadata files' content. type MetaIter = iter.TryNextor[*backuppb.Metadata] @@ -87,6 +86,12 @@ type streamMetadataHelper interface { ParseToMetadata(rawMetaData []byte) (*backuppb.Metadata, error) } +type logFilesStatistic struct { + NumEntries int64 + NumFiles uint64 + Size uint64 +} + // LogFileManager is the manager for log files of a certain restoration, // which supports read / filter from the log backup archive with static start TS / restore TS. type LogFileManager struct { @@ -108,6 +113,10 @@ type LogFileManager struct { withMigrations *WithMigrations metadataDownloadBatchSize uint + + // The output channel for statistics. + // This will be collected when reading the metadata. + Stats *logFilesStatistic } // LogFileManagerInit is the config needed for initializing the log file manager. @@ -311,6 +320,18 @@ func (rc *LogFileManager) LoadDDLFilesAndCountDMLFiles(ctx context.Context) ([]L return rc.collectDDLFilesAndPrepareCache(ctx, mg) } +type loadDMLFilesConfig struct { + Statistic *logFilesStatistic +} + +type loadDMLFilesOption func(*loadDMLFilesConfig) + +func lDOptWithStatistics(s *logFilesStatistic) loadDMLFilesOption { + return func(c *loadDMLFilesConfig) { + c.Statistic = s + } +} + // 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) { @@ -335,7 +356,11 @@ func (rc *LogFileManager) FilterMetaFiles(ms MetaNameIter) MetaGroupIter { return true } // count the progress - TotalEntryCount += d.NumberOfEntries + if rc.Stats != nil { + atomic.AddInt64(&rc.Stats.NumEntries, d.NumberOfEntries) + atomic.AddUint64(&rc.Stats.NumFiles, 1) + atomic.AddUint64(&rc.Stats.Size, d.Length) + } return !d.IsMeta }) return DDLMetaGroup{ @@ -373,14 +398,16 @@ func (rc *LogFileManager) GetExtraFullBackupSSTs(ctx context.Context) iter.TryNe }) } -func (rc *LogFileManager) CountExtraSSTs(ctx context.Context) (int, error) { - count := 0 +func (rc *LogFileManager) CountExtraSSTTotalKVs(ctx context.Context) (int64, error) { + count := int64(0) ssts := iter.ConcatAll(rc.GetCompactionIter(ctx), rc.GetExtraFullBackupSSTs(ctx)) - for err, sst := range iter.AsSeq(ctx, ssts) { + for err, ssts := range iter.AsSeq(ctx, ssts) { if err != nil { return 0, errors.Trace(err) } - count += len(sst.GetSSTs()) + for _, sst := range ssts.GetSSTs() { + count += int64(sst.TotalKvs) + } } return count, nil } diff --git a/br/pkg/restore/restorer.go b/br/pkg/restore/restorer.go index 9f27df71f34e4..75a21b583eb1f 100644 --- a/br/pkg/restore/restorer.go +++ b/br/pkg/restore/restorer.go @@ -31,7 +31,6 @@ import ( "github.com/pingcap/tidb/br/pkg/summary" "github.com/pingcap/tidb/br/pkg/utils/iter" "github.com/pingcap/tidb/pkg/util" - "go.uber.org/multierr" "go.uber.org/zap" "go.uber.org/zap/zapcore" "golang.org/x/sync/errgroup" @@ -110,32 +109,6 @@ func NewFileSet(files []*backuppb.File, rules *utils.RewriteRules) BackupFileSet } } -// DualRestorer is a composite restorer that combines two SstRestorers into a single restorer. -type DualRestorer struct { - A SstRestorer - B SstRestorer -} - -// Dual creates a new DualRestorer from two SstRestorers. -func Dual(a, b SstRestorer) *DualRestorer { - return &DualRestorer{ - A: a, - B: b, - } -} - -func (c *DualRestorer) Close() error { - return multierr.Combine(c.A.Close(), c.B.Close()) -} - -func (c *DualRestorer) WaitUntilFinish() error { - return multierr.Combine(c.A.WaitUntilFinish(), c.B.WaitUntilFinish()) -} - -func (c *DualRestorer) GoRestore(onProgress func(int64), batchFileSets ...BatchBackupFileSet) error { - return multierr.Combine(c.A.GoRestore(onProgress, batchFileSets...), c.B.GoRestore(onProgress, batchFileSets...)) -} - // SstRestorer defines the essential methods required for restoring SST files in various backup formats: // 1. Raw backup SST files // 2. Transactional (Txn) backup SST files diff --git a/br/pkg/task/stream.go b/br/pkg/task/stream.go index 0493e6f96a610..527cc7ca1fa6c 100644 --- a/br/pkg/task/stream.go +++ b/br/pkg/task/stream.go @@ -1476,11 +1476,10 @@ func restoreStream( return errors.Trace(err) } - count, err := client.LogFileManager.CountExtraSSTs(ctx) + numberOfKVsInSST, err := client.LogFileManager.CountExtraSSTTotalKVs(ctx) if err != nil { return err } - logclient.TotalEntryCount += int64(count) se, err := g.CreateSession(mgr.GetStorage()) if err != nil { @@ -1494,7 +1493,8 @@ func restoreStream( compactionIter := client.LogFileManager.GetCompactionIter(ctx) sstsIter := iter.ConcatAll(addedSSTsIter, compactionIter) - pd := g.StartProgress(ctx, "Restore Files(SST + KV)", logclient.TotalEntryCount, !cfg.LogProgress) + totalWorkUnits := numberOfKVsInSST + int64(client.Stats.NumEntries) + pd := g.StartProgress(ctx, "Restore Files(SST + KV)", totalWorkUnits, !cfg.LogProgress) err = withProgress(pd, func(p glue.Progress) (pErr error) { updateStatsWithCheckpoint := func(kvCount, size uint64) { mu.Lock() From d14081ad4c8e6cf1047161c6d7fa55491d89a753 Mon Sep 17 00:00:00 2001 From: hillium Date: Fri, 13 Dec 2024 17:04:16 +0800 Subject: [PATCH 24/36] make progress bar work Signed-off-by: hillium --- br/pkg/restore/log_client/client.go | 8 ++++---- br/pkg/task/stream.go | 4 ++-- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/br/pkg/restore/log_client/client.go b/br/pkg/restore/log_client/client.go index 2519aeb822fc1..fb8cc5046342f 100644 --- a/br/pkg/restore/log_client/client.go +++ b/br/pkg/restore/log_client/client.go @@ -217,8 +217,8 @@ type LogClient struct { // checkpoint information for log restore useCheckpoint bool - logFilesStat *logFilesStatistic - restoreStat *restoreStatistic + logFilesStat logFilesStatistic + restoreStat restoreStatistic } type restoreStatistic struct { @@ -595,8 +595,8 @@ func (rc *LogClient) InstallLogFileManager(ctx context.Context, startTS, restore if err != nil { return err } - rc.logFilesStat = new(logFilesStatistic) - rc.LogFileManager.Stats = rc.logFilesStat + rc.logFilesStat = logFilesStatistic{} + rc.LogFileManager.Stats = &rc.logFilesStat return nil } diff --git a/br/pkg/task/stream.go b/br/pkg/task/stream.go index 527cc7ca1fa6c..45c0fa16a361b 100644 --- a/br/pkg/task/stream.go +++ b/br/pkg/task/stream.go @@ -1163,7 +1163,7 @@ func checkTaskCompat(cfg *RestoreConfig, task streamhelper.Task) error { if !cfg.LocalEncryptionEnabled() { return errors.Annotate(baseErr, "the data you want to restore is encrypted, they cannot be copied to the log storage") } - if task.Info.SecurityConfig != nil { + if task.Info.GetSecurityConfig().GetEncryption() != nil { return errors.Annotate(baseErr, "the running log backup task is encrypted, the data copied to the log storage cannot work") } return nil @@ -1494,7 +1494,7 @@ func restoreStream( sstsIter := iter.ConcatAll(addedSSTsIter, compactionIter) totalWorkUnits := numberOfKVsInSST + int64(client.Stats.NumEntries) - pd := g.StartProgress(ctx, "Restore Files(SST + KV)", totalWorkUnits, !cfg.LogProgress) + pd := g.StartProgress(ctx, "Restore Files(SST + Log)", totalWorkUnits, !cfg.LogProgress) err = withProgress(pd, func(p glue.Progress) (pErr error) { updateStatsWithCheckpoint := func(kvCount, size uint64) { mu.Lock() From 6702ce4160f274e76e27878dd8ce3b069ceeb087 Mon Sep 17 00:00:00 2001 From: hillium Date: Mon, 16 Dec 2024 13:37:25 +0800 Subject: [PATCH 25/36] added a test case for encryption Signed-off-by: hillium --- br/pkg/task/restore.go | 2 +- br/pkg/task/stream.go | 2 +- tests/realtikvtest/brietest/pitr_test.go | 37 ++++++++++++++++++++++++ 3 files changed, 39 insertions(+), 2 deletions(-) diff --git a/br/pkg/task/restore.go b/br/pkg/task/restore.go index 9672915328f71..29e6b057247cb 100644 --- a/br/pkg/task/restore.go +++ b/br/pkg/task/restore.go @@ -279,7 +279,7 @@ type RestoreConfig struct { } func (r *RestoreConfig) LocalEncryptionEnabled() bool { - return r.MasterKeyConfig.EncryptionType != encryptionpb.EncryptionMethod_PLAINTEXT + return r.CipherInfo.CipherType != encryptionpb.EncryptionMethod_PLAINTEXT } // DefineRestoreFlags defines common flags for the restore tidb command. diff --git a/br/pkg/task/stream.go b/br/pkg/task/stream.go index 45c0fa16a361b..0ea70e9a577f4 100644 --- a/br/pkg/task/stream.go +++ b/br/pkg/task/stream.go @@ -1160,7 +1160,7 @@ func checkTaskCompat(cfg *RestoreConfig, task streamhelper.Task) error { "you want to restore a whole cluster, you may use `-f` or `restore table|database` to "+ "specify the tables to restore to continue") } - if !cfg.LocalEncryptionEnabled() { + if cfg.LocalEncryptionEnabled() { return errors.Annotate(baseErr, "the data you want to restore is encrypted, they cannot be copied to the log storage") } if task.Info.GetSecurityConfig().GetEncryption() != nil { diff --git a/tests/realtikvtest/brietest/pitr_test.go b/tests/realtikvtest/brietest/pitr_test.go index 1eb4e08baa111..544bf551c05e0 100644 --- a/tests/realtikvtest/brietest/pitr_test.go +++ b/tests/realtikvtest/brietest/pitr_test.go @@ -2,12 +2,15 @@ package brietest import ( "context" + "encoding/hex" "fmt" "math" "strings" "testing" "time" + backup "github.com/pingcap/kvproto/pkg/brpb" + "github.com/pingcap/kvproto/pkg/encryptionpb" "github.com/pingcap/log" "github.com/pingcap/tidb/br/pkg/glue" "github.com/pingcap/tidb/br/pkg/gluetidb" @@ -180,6 +183,16 @@ func (kit *LogBackupKit) Glue() glue.Glue { return &TestKitGlue{tk: kit.tk} } +func (kit *LogBackupKit) shouldPanic(checker func(v any), f func()) { + defer func() { + v := recover() + require.NotNil(kit.t, v, "should panic not panic") + checker(v) + }() + + f() +} + func (kit *LogBackupKit) mustExec(f func(context.Context) error) { ctx, cancel := context.WithCancel(context.Background()) err := f(ctx) @@ -280,3 +293,27 @@ func TestPiTRAndBackup(t *testing.T) { }) verifySimpleData(kit) } + +func TestEncryptedBackup(t *testing.T) { + kit := NewLogBackupKit(t) + createSimpleTableWithData(kit) + keyContent, _ := hex.DecodeString("9d4cf8f268514d2c38836197008eded1050a5806afa632f7ab1e313bb6697da2") + + kit.RunFullBackup(func(bc *task.BackupConfig) { + bc.CipherInfo = backup.CipherInfo{ + CipherType: encryptionpb.EncryptionMethod_AES256_CTR, + CipherKey: keyContent, + } + }) + + cleanSimpleData(kit) + kit.RunLogStart("something", func(sc *task.StreamConfig) {}) + kit.shouldPanic(func(v any) { fmt.Println(v) }, func() { + kit.RunFullRestore(func(rc *task.RestoreConfig) { + rc.CipherInfo = backup.CipherInfo{ + CipherType: encryptionpb.EncryptionMethod_AES256_CTR, + CipherKey: keyContent, + } + }) + }) +} From aa48140a19703ba79aca811d6a9dd8554976490e Mon Sep 17 00:00:00 2001 From: hillium Date: Mon, 16 Dec 2024 15:12:50 +0800 Subject: [PATCH 26/36] support printing extra full backups --- br/pkg/stream/stream_metas.go | 133 +++++++++++++++++------ br/pkg/task/operator/list_migration.go | 4 +- br/pkg/task/operator/migrate_to.go | 14 ++- tests/realtikvtest/brietest/pitr_test.go | 37 ++++--- 4 files changed, 131 insertions(+), 57 deletions(-) diff --git a/br/pkg/stream/stream_metas.go b/br/pkg/stream/stream_metas.go index 11a9888d740ab..5607eff9c78d4 100644 --- a/br/pkg/stream/stream_metas.go +++ b/br/pkg/stream/stream_metas.go @@ -5,6 +5,7 @@ package stream import ( "context" "encoding/binary" + "encoding/hex" "fmt" "hash/crc64" "maps" @@ -15,6 +16,7 @@ import ( "strconv" "strings" "sync" + "time" "github.com/docker/go-units" "github.com/fatih/color" @@ -30,6 +32,7 @@ import ( "github.com/pingcap/tidb/pkg/util" "github.com/pingcap/tidb/pkg/util/mathutil" "github.com/pingcap/tidb/pkg/util/versioninfo" + "github.com/tikv/client-go/v2/oracle" "go.uber.org/multierr" "go.uber.org/zap" "golang.org/x/sync/errgroup" @@ -345,48 +348,110 @@ func ReplaceMetadata(meta *pb.Metadata, filegroups []*pb.DataFileGroup) { updateMetadataInternalStat(meta) } -func AddMigrationToTable(m *pb.Migration, table *glue.Table) { - rd := color.New(color.FgHiRed).Sprint - for i, c := range m.Compactions { - addCompactionToTable(c, table, i) +type marshalMigrationContext struct { + context.Context + est MigrationExt + + output *glue.Table + keyspace []string +} + +func (m *marshalMigrationContext) emit(key, value string) { + bold := color.New(color.Bold).Sprintf + ks := new(strings.Builder) + for _, k := range m.keyspace { + ks.WriteString(k) + ks.WriteString("/") } + ks.WriteString(key) - if len(m.EditMeta) > 0 { - totalDeletePhyFile := 0 - totalDeleteLgcFile := 0 - for _, edit := range m.EditMeta { - totalDeletePhyFile += len(edit.DeletePhysicalFiles) - for _, dl := range edit.DeleteLogicalFiles { - totalDeleteLgcFile += len(dl.Spans) - } + finalValue := bold(value) + m.output.Add(ks.String(), finalValue) +} + +func (m *marshalMigrationContext) keyspaced(key []string, f func()) { + m.keyspace = append(m.keyspace, key...) + defer func() { + m.keyspace = m.keyspace[:len(m.keyspace)-len(key)] + }() + + f() +} + +func (m *marshalMigrationContext) addCompaction(c *pb.LogFileCompaction) { + m.emit("name", c.Name) + m.emit("time", fmt.Sprintf("%d ~ %d", c.CompactionFromTs, c.CompactionUntilTs)) + m.emit("file", fmt.Sprintf("[%q, %q]", c.Artifacts, c.GeneratedFiles)) +} + +func (m *marshalMigrationContext) addMetaEdits(em []*pb.MetaEdit) { + totalDeletePhyFile := 0 + totalDeleteLgcFile := 0 + for _, edit := range em { + totalDeletePhyFile += len(edit.DeletePhysicalFiles) + for _, dl := range edit.DeleteLogicalFiles { + totalDeleteLgcFile += len(dl.Spans) } - table.Add( - "edit-meta-files", - fmt.Sprintf("%s meta files will be edited.", rd(len(m.EditMeta))), - ) - table.Add( - "delete-physical-file", - fmt.Sprintf("%s physical files will be deleted.", rd(totalDeletePhyFile)), - ) - table.Add( - "delete-logical-file", - fmt.Sprintf("%s logical segments may be deleted, if possible.", rd(totalDeleteLgcFile)), - ) } - for i, c := range m.DestructPrefix { - table.Add(fmt.Sprintf("destruct-prefix[%02d]", i), rd(c)) + m.emit("edit_meta_files", strconv.Itoa(len(em))) + m.emit("delete_physical_file", strconv.Itoa(totalDeletePhyFile)) + m.emit("delete_logical_file", strconv.Itoa(totalDeleteLgcFile)) +} + +func (m *marshalMigrationContext) addTruncatedTo(tso uint64) { + if tso == 0 { + m.emit("truncated_to", "N/A") + return } - for i, c := range m.GetExtraFullBackupPaths() { - table.Add(fmt.Sprintf("extra_full_backups[%02d]", i), rd(c)) + m.emit("truncated_to", strconv.FormatUint(tso, 10)) + t := oracle.GetTimeFromTS(tso) + m.emit("truncated_to_in_rfc3339", t.Format(time.RFC3339)) +} + +func (m *marshalMigrationContext) addMigration(mig *pb.Migration) { + m.addTruncatedTo(mig.TruncatedTo) + for i, c := range mig.Compactions { + m.keyspaced([]string{"compactions", strconv.Itoa(i)}, func() { + m.addCompaction(c) + }) + } + m.keyspaced([]string{"meta_edit"}, func() { + m.addMetaEdits(mig.EditMeta) + }) + for i, d := range mig.DestructPrefix { + m.keyspaced([]string{"destruct_prefix", strconv.Itoa(i)}, func() { + m.emit("value", d) + }) + } + for i, p := range mig.ExtraFullBackupPaths { + m.keyspaced([]string{"extra_full_backup", strconv.Itoa(i)}, func() { + m.addExtraFullBackups(p) + }) } - table.Add("truncate-to", rd(m.TruncatedTo)) } -func addCompactionToTable(m *pb.LogFileCompaction, table *glue.Table, idx int) { - withIdx := func(s string) string { return fmt.Sprintf("compactions[%d].%s", idx, s) } - table.Add(withIdx("name"), m.Name) - table.Add(withIdx("time"), fmt.Sprintf("%d ~ %d", m.CompactionFromTs, m.CompactionUntilTs)) - table.Add(withIdx("file"), fmt.Sprintf("[%q, %q]", m.Artifacts, m.GeneratedFiles)) +func (m *marshalMigrationContext) addExtraFullBackups(path string) { + fullbk, err := readExtraFullBackup(m.Context, path, m.est.s) + if err != nil { + m.emit("err_during_reading", err.Error()) + m.emit("meta_path", path) + return + } + + m.emit("as_if_ts", strconv.FormatUint(fullbk.AsIfTs, 10)) + m.emit("backup_uuid", hex.EncodeToString(fullbk.GetBackupUuid())) + m.emit("files_count", strconv.Itoa(len(fullbk.Files))) + m.emit("files_position", fullbk.FilesPrefixHint) +} + +func (m MigrationExt) AddMigrationToTable(ctx context.Context, mig *pb.Migration, table *glue.Table) { + cx := marshalMigrationContext{ + Context: ctx, + est: m, + output: table, + } + + cx.addMigration(mig) } // MigrationExt is an extension to the `ExternalStorage` type. diff --git a/br/pkg/task/operator/list_migration.go b/br/pkg/task/operator/list_migration.go index 591638656ddbf..544124d1b5707 100644 --- a/br/pkg/task/operator/list_migration.go +++ b/br/pkg/task/operator/list_migration.go @@ -40,12 +40,12 @@ func RunListMigrations(ctx context.Context, cfg ListMigrationConfig) error { console.Println(statusOK(fmt.Sprintf("Total %d Migrations.", len(migs.Layers)+1))) console.Printf("> BASE <\n") tbl := console.CreateTable() - stream.AddMigrationToTable(migs.Base, tbl) + ext.AddMigrationToTable(ctx, migs.Base, tbl) tbl.Print() for _, t := range migs.Layers { console.Printf("> %08d <\n", t.SeqNum) tbl := console.CreateTable() - stream.AddMigrationToTable(&t.Content, tbl) + ext.AddMigrationToTable(ctx, &t.Content, tbl) tbl.Print() } } diff --git a/br/pkg/task/operator/migrate_to.go b/br/pkg/task/operator/migrate_to.go index c00d5bbccfb28..2a086b9868db1 100644 --- a/br/pkg/task/operator/migrate_to.go +++ b/br/pkg/task/operator/migrate_to.go @@ -39,16 +39,16 @@ func (cx migrateToCtx) printErr(errs []error, msg string) { } } -func (cx migrateToCtx) askForContinue(targetMig *backup.Migration) bool { +func (cx migrateToCtx) askForContinue(ctx context.Context, targetMig *backup.Migration) bool { tbl := cx.console.CreateTable() - stream.AddMigrationToTable(targetMig, tbl) + cx.est.AddMigrationToTable(ctx, targetMig, tbl) cx.console.Println("The migration going to be executed will be like: ") tbl.Print() return cx.console.PromptBool("Continue? ") } -func (cx migrateToCtx) dryRun(f func(stream.MigrationExt) stream.MergeAndMigratedTo) error { +func (cx migrateToCtx) dryRun(ctx context.Context, f func(stream.MigrationExt) stream.MergeAndMigratedTo) error { var ( est = cx.est console = cx.console @@ -60,7 +60,7 @@ func (cx migrateToCtx) dryRun(f func(stream.MigrationExt) stream.MergeAndMigrate }) tbl := console.CreateTable() - stream.AddMigrationToTable(estBase.NewBase, tbl) + cx.est.AddMigrationToTable(ctx, estBase.NewBase, tbl) console.Println("The new BASE migration will be like: ") tbl.Print() file, err := storage.SaveJSONEffectsToTmp(effects) @@ -120,12 +120,14 @@ func RunMigrateTo(ctx context.Context, cfg MigrateToConfig) error { return nil } if cfg.DryRun { - run = cx.dryRun + run = func(f func(stream.MigrationExt) stream.MergeAndMigratedTo) error { + return cx.dryRun(ctx, f) + } } return run(func(est stream.MigrationExt) stream.MergeAndMigratedTo { return est.MergeAndMigrateTo(ctx, targetVersion, stream.MMOptInteractiveCheck(func(ctx context.Context, m *backup.Migration) bool { - return cfg.Yes || cx.askForContinue(m) + return cfg.Yes || cx.askForContinue(ctx, m) })) }) } diff --git a/tests/realtikvtest/brietest/pitr_test.go b/tests/realtikvtest/brietest/pitr_test.go index 544bf551c05e0..907b3844ca5c3 100644 --- a/tests/realtikvtest/brietest/pitr_test.go +++ b/tests/realtikvtest/brietest/pitr_test.go @@ -83,6 +83,8 @@ type LogBackupKit struct { tk *testkit.TestKit metaCli *streamhelper.MetaDataClient base string + + checkerF func(err error) } func NewLogBackupKit(t *testing.T) *LogBackupKit { @@ -95,11 +97,14 @@ func NewLogBackupKit(t *testing.T) *LogBackupKit { t: t, metaCli: metaCli, base: t.TempDir(), + checkerF: func(err error) { + require.NoError(t, err) + }, } } func (kit *LogBackupKit) RunFullRestore(extConfig func(*task.RestoreConfig)) { - kit.mustExec(func(ctx context.Context) error { + kit.runAndCheck(func(ctx context.Context) error { cfg := task.DefaultRestoreConfig(task.DefaultConfig()) cfg.Storage = "local://" + kit.base + "/full" cfg.FilterStr = []string{"test.*"} @@ -114,7 +119,7 @@ func (kit *LogBackupKit) RunFullRestore(extConfig func(*task.RestoreConfig)) { } func (kit *LogBackupKit) RunStreamRestore(extConfig func(*task.RestoreConfig)) { - kit.mustExec(func(ctx context.Context) error { + kit.runAndCheck(func(ctx context.Context) error { cfg := task.DefaultRestoreConfig(task.DefaultConfig()) cfg.Storage = "local://" + kit.base + "/incr" cfg.FullBackupStorage = "local://" + kit.base + "/full" @@ -126,7 +131,7 @@ func (kit *LogBackupKit) RunStreamRestore(extConfig func(*task.RestoreConfig)) { } func (kit *LogBackupKit) RunFullBackup(extConfig func(*task.BackupConfig)) { - kit.mustExec(func(ctx context.Context) error { + kit.runAndCheck(func(ctx context.Context) error { cfg := task.DefaultBackupConfig(task.DefaultConfig()) cfg.Storage = "local://" + kit.base + "/full" extConfig(&cfg) @@ -135,7 +140,7 @@ func (kit *LogBackupKit) RunFullBackup(extConfig func(*task.BackupConfig)) { } func (kit *LogBackupKit) StopTaskIfExists(taskName string) { - kit.mustExec(func(ctx context.Context) error { + kit.runAndCheck(func(ctx context.Context) error { cfg := task.DefaultStreamConfig(task.DefineStreamCommonFlags) cfg.TaskName = taskName err := task.RunStreamStop(ctx, kit.Glue(), "stream stop[intest]", &cfg) @@ -147,7 +152,7 @@ func (kit *LogBackupKit) StopTaskIfExists(taskName string) { } func (kit *LogBackupKit) RunLogStart(taskName string, extConfig func(*task.StreamConfig)) { - kit.mustExec(func(ctx context.Context) error { + kit.runAndCheck(func(ctx context.Context) error { cfg := task.DefaultStreamConfig(task.DefineStreamStartFlags) cfg.Storage = "local://" + kit.base + "/incr" cfg.TaskName = taskName @@ -183,25 +188,25 @@ func (kit *LogBackupKit) Glue() glue.Glue { return &TestKitGlue{tk: kit.tk} } -func (kit *LogBackupKit) shouldPanic(checker func(v any), f func()) { +func (kit *LogBackupKit) WithChecker(checker func(v error), f func()) { + oldExpected := kit.checkerF defer func() { - v := recover() - require.NotNil(kit.t, v, "should panic not panic") - checker(v) + kit.checkerF = oldExpected }() + kit.checkerF = checker f() } -func (kit *LogBackupKit) mustExec(f func(context.Context) error) { +func (kit *LogBackupKit) runAndCheck(f func(context.Context) error) { ctx, cancel := context.WithCancel(context.Background()) err := f(ctx) cancel() - require.NoError(kit.t, err) + kit.checkerF(err) } func (kit *LogBackupKit) forceFlush() { - kit.mustExec(func(ctx context.Context) error { + kit.runAndCheck(func(ctx context.Context) error { cfg := task.DefaultConfig() cfg.PD = append(cfg.PD, config.GetGlobalConfig().Path) err := operator.RunForceFlush(ctx, &operator.ForceFlushConfig{ @@ -294,10 +299,11 @@ func TestPiTRAndBackup(t *testing.T) { verifySimpleData(kit) } -func TestEncryptedBackup(t *testing.T) { +func TestEncryptedFullBackup(t *testing.T) { kit := NewLogBackupKit(t) createSimpleTableWithData(kit) - keyContent, _ := hex.DecodeString("9d4cf8f268514d2c38836197008eded1050a5806afa632f7ab1e313bb6697da2") + keyContent, err := hex.DecodeString("9d4cf8f268514d2c38836197008eded1050a5806afa632f7ab1e313bb6697da2") + require.NoError(t, err) kit.RunFullBackup(func(bc *task.BackupConfig) { bc.CipherInfo = backup.CipherInfo{ @@ -308,7 +314,8 @@ func TestEncryptedBackup(t *testing.T) { cleanSimpleData(kit) kit.RunLogStart("something", func(sc *task.StreamConfig) {}) - kit.shouldPanic(func(v any) { fmt.Println(v) }, func() { + chk := func(err error) { require.ErrorContains(t, err, "the data you want to restore is encrypted") } + kit.WithChecker(chk, func() { kit.RunFullRestore(func(rc *task.RestoreConfig) { rc.CipherInfo = backup.CipherInfo{ CipherType: encryptionpb.EncryptionMethod_AES256_CTR, From f8a135fb91378caa2c1f823ee1a68d77ef969369 Mon Sep 17 00:00:00 2001 From: hillium Date: Mon, 16 Dec 2024 15:55:14 +0800 Subject: [PATCH 27/36] return error when no BASE migration found --- br/pkg/errors/errors.go | 1 + br/pkg/stream/stream_metas.go | 16 +++++++++++++++- errors.toml | 5 +++++ 3 files changed, 21 insertions(+), 1 deletion(-) diff --git a/br/pkg/errors/errors.go b/br/pkg/errors/errors.go index 6a9449eff95d1..2db9ece1e1735 100644 --- a/br/pkg/errors/errors.go +++ b/br/pkg/errors/errors.go @@ -40,6 +40,7 @@ var ( ErrEnvNotSpecified = errors.Normalize("environment variable not found", errors.RFCCodeText("BR:Common:ErrEnvNotSpecified")) ErrUnsupportedOperation = errors.Normalize("the operation is not supported", errors.RFCCodeText("BR:Common:ErrUnsupportedOperation")) ErrInvalidRange = errors.Normalize("invalid restore range", errors.RFCCodeText("BR:Common:ErrInvalidRange")) + ErrMigrationNotFound = errors.Normalize("no migrtion found", errors.RFCCodeText("BR:Common:ErrMigrationNotFound")) ErrMigrationVersionNotSupported = errors.Normalize("the migration version isn't supported", errors.RFCCodeText("BR:Common:ErrMigrationVersionNotSupported")) ErrPDUpdateFailed = errors.Normalize("failed to update PD", errors.RFCCodeText("BR:PD:ErrPDUpdateFailed")) diff --git a/br/pkg/stream/stream_metas.go b/br/pkg/stream/stream_metas.go index 5607eff9c78d4..2208f578358bb 100644 --- a/br/pkg/stream/stream_metas.go +++ b/br/pkg/stream/stream_metas.go @@ -385,6 +385,10 @@ func (m *marshalMigrationContext) addCompaction(c *pb.LogFileCompaction) { } func (m *marshalMigrationContext) addMetaEdits(em []*pb.MetaEdit) { + if len(em) == 0 { + return + } + totalDeletePhyFile := 0 totalDeleteLgcFile := 0 for _, edit := range em { @@ -400,7 +404,6 @@ func (m *marshalMigrationContext) addMetaEdits(em []*pb.MetaEdit) { func (m *marshalMigrationContext) addTruncatedTo(tso uint64) { if tso == 0 { - m.emit("truncated_to", "N/A") return } m.emit("truncated_to", strconv.FormatUint(tso, 10)) @@ -705,6 +708,9 @@ func (m MigrationExt) Load(ctx context.Context) (Migrations, error) { if collected.Err != nil { return Migrations{}, collected.Err } + if len(collected.Item) == 0 { + return Migrations{}, errors.Annotatef(berrors.ErrMigrationNotFound, "in the storage %s", m.s.URI()) + } sort.Slice(collected.Item, func(i, j int) bool { return collected.Item[i].SeqNum < collected.Item[j].SeqNum }) @@ -1605,3 +1611,11 @@ func hashMetaEdit(metaEdit *pb.MetaEdit) uint64 { func nameOf(mig *pb.Migration, sn int) string { return fmt.Sprintf("%08d_%016X.mgrt", sn, hashMigration(mig)) } + +func isEmptyMigration(mig *pb.Migration) bool { + return len(mig.Compactions) == 0 && + len(mig.EditMeta) == 0 && + len(mig.ExtraFullBackupPaths) == 0 && + len(mig.DestructPrefix) == 0 && + mig.TruncatedTo == 0 +} diff --git a/errors.toml b/errors.toml index 554ba7cb6f8dc..82a6e85be3584 100644 --- a/errors.toml +++ b/errors.toml @@ -56,6 +56,11 @@ error = ''' invalid restore range ''' +["BR:Common:ErrMigrationNotFound"] +error = ''' +no migrtion found +''' + ["BR:Common:ErrMigrationVersionNotSupported"] error = ''' the migration version isn't supported From cdde94fa973cc26f7c3a2c02e4f9209bf79b3607 Mon Sep 17 00:00:00 2001 From: hillium Date: Mon, 16 Dec 2024 18:36:37 +0800 Subject: [PATCH 28/36] added some unit test for encryption --- br/pkg/restore/log_client/client.go | 2 +- br/pkg/stream/stream_metas.go | 3 +- br/pkg/stream/stream_metas_test.go | 4 +- br/pkg/task/operator/migrate_to.go | 2 +- br/pkg/task/stream.go | 4 +- tests/realtikvtest/brietest/pitr_test.go | 78 +++++++++++++++++++++++- 6 files changed, 84 insertions(+), 9 deletions(-) diff --git a/br/pkg/restore/log_client/client.go b/br/pkg/restore/log_client/client.go index e6539dc75f149..a962137e4409a 100644 --- a/br/pkg/restore/log_client/client.go +++ b/br/pkg/restore/log_client/client.go @@ -349,7 +349,7 @@ func (rc *LogClient) RestoreCompactedSstFiles( if err != nil { return errors.Trace(err) } - err := rc.sstRestoreManager.restorer.WaitUntilFinish() + err = rc.sstRestoreManager.restorer.WaitUntilFinish() for _, files := range backupFileSets { for _, f := range files.SSTFiles { diff --git a/br/pkg/stream/stream_metas.go b/br/pkg/stream/stream_metas.go index 94bbcf10a05c9..8250092ac9668 100644 --- a/br/pkg/stream/stream_metas.go +++ b/br/pkg/stream/stream_metas.go @@ -959,7 +959,6 @@ func (m MigrationExt) migrateTo(ctx context.Context, mig *pb.Migration, opts ... result := MigratedTo{ NewBase: NewMigration(), } - m.processMetaEdits(ctx, mig, &result) m.processCompactions(ctx, mig, &result) m.processDestroyPrefixes(ctx, mig, &result) m.processExtFullBackup(ctx, mig, &result) @@ -969,7 +968,7 @@ func (m MigrationExt) migrateTo(ctx context.Context, mig *pb.Migration, opts ... // We do skip truncate log first, so metas removed by truncating can be removed in this execution. // Fills: EditMeta for new Base. - m.doMetaEdits(ctx, mig, &result) + m.processMetaEdits(ctx, mig, &result) return result } diff --git a/br/pkg/stream/stream_metas_test.go b/br/pkg/stream/stream_metas_test.go index cbd1a5d556ca1..6851f043ffa14 100644 --- a/br/pkg/stream/stream_metas_test.go +++ b/br/pkg/stream/stream_metas_test.go @@ -2802,7 +2802,7 @@ func TestAppendingMigs(t *testing.T) { asp(fi(80, 85, WriteCF, 72), sp(34, 5)), }, }), lN(2)) - est := MigerationExtension(s) + est := MigrationExtension(s) cDir := func(n uint64) string { return fmt.Sprintf("%05d/output", n) } aDir := func(n uint64) string { return fmt.Sprintf("%05d/metas", n) } @@ -2835,7 +2835,7 @@ func TestUserAbort(t *testing.T) { pmig(s, 0, mig(mTruncatedTo(42))) pmig(s, 1, mig(mTruncatedTo(96))) - est := MigerationExtension(s) + est := MigrationExtension(s) var res MergeAndMigratedTo effs := est.DryRun(func(me MigrationExt) { res = me.MergeAndMigrateTo(ctx, 1, MMOptInteractiveCheck(func(ctx context.Context, m *backuppb.Migration) bool { diff --git a/br/pkg/task/operator/migrate_to.go b/br/pkg/task/operator/migrate_to.go index c0d3a33ee4f72..2a086b9868db1 100644 --- a/br/pkg/task/operator/migrate_to.go +++ b/br/pkg/task/operator/migrate_to.go @@ -5,7 +5,7 @@ import ( "github.com/fatih/color" "github.com/pingcap/errors" - backuppb "github.com/pingcap/kvproto/pkg/brpb" + backup "github.com/pingcap/kvproto/pkg/brpb" "github.com/pingcap/tidb/br/pkg/glue" "github.com/pingcap/tidb/br/pkg/storage" "github.com/pingcap/tidb/br/pkg/stream" diff --git a/br/pkg/task/stream.go b/br/pkg/task/stream.go index de009223873d4..db07fb4349226 100644 --- a/br/pkg/task/stream.go +++ b/br/pkg/task/stream.go @@ -1067,13 +1067,13 @@ func RunStreamTruncate(c context.Context, g glue.Glue, cmdName string, cfg *Stre } if cfg.CleanUpCompactions { - est := stream.MigerationExtension(extStorage) + est := stream.MigrationExtension(extStorage) est.Hooks = stream.NewProgressBarHooks(console) newSN := math.MaxInt optPrompt := stream.MMOptInteractiveCheck(func(ctx context.Context, m *backuppb.Migration) bool { console.Println("We are going to do the following: ") tbl := console.CreateTable() - stream.AddMigrationToTable(m, tbl) + est.AddMigrationToTable(ctx, m, tbl) tbl.Print() return console.PromptBool("Continue? ") }) diff --git a/tests/realtikvtest/brietest/pitr_test.go b/tests/realtikvtest/brietest/pitr_test.go index 907b3844ca5c3..d67c9576319c9 100644 --- a/tests/realtikvtest/brietest/pitr_test.go +++ b/tests/realtikvtest/brietest/pitr_test.go @@ -5,6 +5,8 @@ import ( "encoding/hex" "fmt" "math" + "os" + "path/filepath" "strings" "testing" "time" @@ -103,6 +105,12 @@ func NewLogBackupKit(t *testing.T) *LogBackupKit { } } +func (kit *LogBackupKit) tempFile(name string, content []byte) string { + path := filepath.Join(kit.t.TempDir(), name) + require.NoError(kit.t, os.WriteFile(path, content, 0o666)) + return path +} + func (kit *LogBackupKit) RunFullRestore(extConfig func(*task.RestoreConfig)) { kit.runAndCheck(func(ctx context.Context) error { cfg := task.DefaultRestoreConfig(task.DefaultConfig()) @@ -313,7 +321,7 @@ func TestEncryptedFullBackup(t *testing.T) { }) cleanSimpleData(kit) - kit.RunLogStart("something", func(sc *task.StreamConfig) {}) + kit.RunLogStart(t.Name(), func(sc *task.StreamConfig) {}) chk := func(err error) { require.ErrorContains(t, err, "the data you want to restore is encrypted") } kit.WithChecker(chk, func() { kit.RunFullRestore(func(rc *task.RestoreConfig) { @@ -324,3 +332,71 @@ func TestEncryptedFullBackup(t *testing.T) { }) }) } + +func TestEncryptedLogBackup(t *testing.T) { + kit := NewLogBackupKit(t) + createSimpleTableWithData(kit) + + keyContent, err := hex.DecodeString("0ae31c060ff933cabe842430e1716185cc9c6b5cdde8e56976afaff41b92528f") + require.NoError(t, err) + keyFile := kit.tempFile("KEY", keyContent) + + kit.RunFullBackup(func(bc *task.BackupConfig) { + + }) + cleanSimpleData(kit) + + kit.RunLogStart(t.Name(), func(sc *task.StreamConfig) { + sc.MasterKeyConfig.EncryptionType = encryptionpb.EncryptionMethod_AES256_CTR + sc.MasterKeyConfig.MasterKeys = append(sc.MasterKeyConfig.MasterKeys, &encryptionpb.MasterKey{ + Backend: &encryptionpb.MasterKey_File{ + File: &encryptionpb.MasterKeyFile{ + Path: keyFile, + }, + }, + }) + }) + + chk := func(err error) { require.ErrorContains(t, err, "the running log backup task is encrypted") } + kit.WithChecker(chk, func() { + kit.RunFullRestore(func(rc *task.RestoreConfig) {}) + }) +} + +func TestBothEncrypted(t *testing.T) { + kit := NewLogBackupKit(t) + createSimpleTableWithData(kit) + + keyContent, err := hex.DecodeString("0ae31c060ff933cabe842430e1716185cc9c6b5cdde8e56976afaff41b92528f") + require.NoError(t, err) + keyFile := kit.tempFile("KEY", keyContent) + + kit.RunFullBackup(func(bc *task.BackupConfig) { + bc.CipherInfo = backup.CipherInfo{ + CipherType: encryptionpb.EncryptionMethod_AES256_CTR, + CipherKey: keyContent, + } + }) + cleanSimpleData(kit) + + kit.RunLogStart(t.Name(), func(sc *task.StreamConfig) { + sc.MasterKeyConfig.EncryptionType = encryptionpb.EncryptionMethod_AES256_CTR + sc.MasterKeyConfig.MasterKeys = append(sc.MasterKeyConfig.MasterKeys, &encryptionpb.MasterKey{ + Backend: &encryptionpb.MasterKey_File{ + File: &encryptionpb.MasterKeyFile{ + Path: keyFile, + }, + }, + }) + }) + + chk := func(err error) { require.ErrorContains(t, err, "encrypted") } + kit.WithChecker(chk, func() { + kit.RunFullRestore(func(rc *task.RestoreConfig) { + rc.CipherInfo = backup.CipherInfo{ + CipherType: encryptionpb.EncryptionMethod_AES256_CTR, + CipherKey: keyContent, + } + }) + }) +} From 40b8640137b619fa878d9ba25535709dc4891aeb Mon Sep 17 00:00:00 2001 From: hillium Date: Tue, 17 Dec 2024 16:18:39 +0800 Subject: [PATCH 29/36] use different key for testing --- tests/realtikvtest/brietest/pitr_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/realtikvtest/brietest/pitr_test.go b/tests/realtikvtest/brietest/pitr_test.go index d67c9576319c9..54d28ef3542a4 100644 --- a/tests/realtikvtest/brietest/pitr_test.go +++ b/tests/realtikvtest/brietest/pitr_test.go @@ -367,7 +367,7 @@ func TestBothEncrypted(t *testing.T) { kit := NewLogBackupKit(t) createSimpleTableWithData(kit) - keyContent, err := hex.DecodeString("0ae31c060ff933cabe842430e1716185cc9c6b5cdde8e56976afaff41b92528f") + keyContent, err := hex.DecodeString("319b4a104651746f1bf1ad67c9ba7d635d8c4769b03f3e5c63f1da93891ce4f9") require.NoError(t, err) keyFile := kit.tempFile("KEY", keyContent) From a2b8b8edecf1a01b274c5e2fd187736f55ace336 Mon Sep 17 00:00:00 2001 From: hillium Date: Wed, 18 Dec 2024 18:05:38 +0800 Subject: [PATCH 30/36] make bazel_prepare Signed-off-by: hillium --- br/pkg/restore/log_client/BUILD.bazel | 6 +++++- br/pkg/restore/snap_client/BUILD.bazel | 5 +++++ br/pkg/restore/utils/BUILD.bazel | 1 + br/pkg/storage/BUILD.bazel | 1 + br/pkg/stream/BUILD.bazel | 4 +++- br/pkg/task/BUILD.bazel | 2 ++ br/pkg/task/operator/BUILD.bazel | 16 ++++++++++++++++ pkg/util/BUILD.bazel | 3 +++ tests/realtikvtest/brietest/BUILD.bazel | 13 +++++++++++++ 9 files changed, 49 insertions(+), 2 deletions(-) diff --git a/br/pkg/restore/log_client/BUILD.bazel b/br/pkg/restore/log_client/BUILD.bazel index 7fb781e7ad0ef..13bfa3bac9334 100644 --- a/br/pkg/restore/log_client/BUILD.bazel +++ b/br/pkg/restore/log_client/BUILD.bazel @@ -11,6 +11,7 @@ go_library( "log_file_map.go", "log_split_strategy.go", "migration.go", + "ssts.go", ], importpath = "github.com/pingcap/tidb/br/pkg/restore/log_client", visibility = ["//visibility:public"], @@ -43,11 +44,13 @@ go_library( "//pkg/kv", "//pkg/meta", "//pkg/meta/model", + "//pkg/tablecodec", "//pkg/util", "//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", "@com_github_opentracing_opentracing_go//:opentracing-go", @@ -90,7 +93,7 @@ go_test( ], embed = [":log_client"], flaky = True, - shard_count = 45, + shard_count = 50, deps = [ "//br/pkg/errors", "//br/pkg/glue", @@ -119,6 +122,7 @@ go_test( "//pkg/util/sqlexec", "//pkg/util/table-filter", "@com_github_docker_go_units//:go-units", + "@com_github_google_uuid//:uuid", "@com_github_pingcap_errors//:errors", "@com_github_pingcap_failpoint//:failpoint", "@com_github_pingcap_kvproto//pkg/brpb", diff --git a/br/pkg/restore/snap_client/BUILD.bazel b/br/pkg/restore/snap_client/BUILD.bazel index 5df612e4750e6..b100ed5358618 100644 --- a/br/pkg/restore/snap_client/BUILD.bazel +++ b/br/pkg/restore/snap_client/BUILD.bazel @@ -6,6 +6,7 @@ go_library( "client.go", "import.go", "pipeline_items.go", + "pitr_collector.go", "placement_rule_manager.go", "systable_restore.go", "tikv_sender.go", @@ -29,6 +30,8 @@ go_library( "//br/pkg/restore/split", "//br/pkg/restore/utils", "//br/pkg/storage", + "//br/pkg/stream", + "//br/pkg/streamhelper", "//br/pkg/summary", "//br/pkg/utils", "//br/pkg/version", @@ -55,9 +58,11 @@ go_library( "@com_github_pingcap_kvproto//pkg/kvrpcpb", "@com_github_pingcap_kvproto//pkg/metapb", "@com_github_pingcap_log//:log", + "@com_github_tikv_client_go_v2//oracle", "@com_github_tikv_client_go_v2//util", "@com_github_tikv_pd_client//:client", "@com_github_tikv_pd_client//http", + "@io_etcd_go_etcd_client_v3//:client", "@org_golang_google_grpc//codes", "@org_golang_google_grpc//keepalive", "@org_golang_google_grpc//status", diff --git a/br/pkg/restore/utils/BUILD.bazel b/br/pkg/restore/utils/BUILD.bazel index a40f0a883ae09..c29bd0f813428 100644 --- a/br/pkg/restore/utils/BUILD.bazel +++ b/br/pkg/restore/utils/BUILD.bazel @@ -15,6 +15,7 @@ go_library( "//br/pkg/rtree", "//pkg/meta/model", "//pkg/tablecodec", + "//pkg/util", "//pkg/util/codec", "//pkg/util/redact", "@com_github_pingcap_errors//:errors", diff --git a/br/pkg/storage/BUILD.bazel b/br/pkg/storage/BUILD.bazel index 60c587893af9f..ee28627832715 100644 --- a/br/pkg/storage/BUILD.bazel +++ b/br/pkg/storage/BUILD.bazel @@ -28,6 +28,7 @@ go_library( deps = [ "//br/pkg/errors", "//br/pkg/logutil", + "//br/pkg/utils", "//br/pkg/utils/iter", "//pkg/lightning/log", "//pkg/sessionctx/variable", diff --git a/br/pkg/stream/BUILD.bazel b/br/pkg/stream/BUILD.bazel index 225d50cb5a9a9..2955aa2e7c859 100644 --- a/br/pkg/stream/BUILD.bazel +++ b/br/pkg/stream/BUILD.bazel @@ -37,6 +37,7 @@ go_library( "//pkg/util/versioninfo", "@com_github_docker_go_units//:go-units", "@com_github_fatih_color//:color", + "@com_github_google_uuid//:uuid", "@com_github_klauspost_compress//zstd", "@com_github_pingcap_errors//:errors", "@com_github_pingcap_kvproto//pkg/brpb", @@ -65,7 +66,7 @@ go_test( ], embed = [":stream"], flaky = True, - shard_count = 48, + shard_count = 49, deps = [ "//br/pkg/storage", "//br/pkg/streamhelper", @@ -81,6 +82,7 @@ go_test( "//pkg/util/intest", "//pkg/util/table-filter", "@com_github_fsouza_fake_gcs_server//fakestorage", + "@com_github_google_uuid//:uuid", "@com_github_pingcap_errors//:errors", "@com_github_pingcap_failpoint//:failpoint", "@com_github_pingcap_kvproto//pkg/brpb", diff --git a/br/pkg/task/BUILD.bazel b/br/pkg/task/BUILD.bazel index b185c358a5afa..71f07bba8df49 100644 --- a/br/pkg/task/BUILD.bazel +++ b/br/pkg/task/BUILD.bazel @@ -50,6 +50,7 @@ go_library( "//br/pkg/streamhelper/daemon", "//br/pkg/summary", "//br/pkg/utils", + "//br/pkg/utils/iter", "//br/pkg/version", "//pkg/config", "//pkg/ddl", @@ -98,6 +99,7 @@ go_library( "@org_golang_x_sync//errgroup", "@org_uber_go_multierr//:multierr", "@org_uber_go_zap//:zap", + "@org_uber_go_zap//zapcore", ], ) diff --git a/br/pkg/task/operator/BUILD.bazel b/br/pkg/task/operator/BUILD.bazel index 14760027a49b8..508d29b6fbe52 100644 --- a/br/pkg/task/operator/BUILD.bazel +++ b/br/pkg/task/operator/BUILD.bazel @@ -4,7 +4,9 @@ go_library( name = "operator", srcs = [ "base64ify.go", + "checksum_table.go", "config.go", + "force_flush.go", "list_migration.go", "migrate_to.go", "prepare_snap.go", @@ -12,22 +14,36 @@ go_library( importpath = "github.com/pingcap/tidb/br/pkg/task/operator", visibility = ["//visibility:public"], deps = [ + "//br/pkg/backup", "//br/pkg/backup/prepare_snap", + "//br/pkg/checksum", + "//br/pkg/conn", "//br/pkg/errors", "//br/pkg/glue", "//br/pkg/logutil", + "//br/pkg/metautil", "//br/pkg/pdutil", "//br/pkg/storage", "//br/pkg/stream", "//br/pkg/task", "//br/pkg/utils", + "//pkg/domain", + "//pkg/meta/model", + "//pkg/util", + "//pkg/util/engine", "@com_github_fatih_color//:color", "@com_github_pingcap_errors//:errors", "@com_github_pingcap_failpoint//:failpoint", "@com_github_pingcap_kvproto//pkg/brpb", + "@com_github_pingcap_kvproto//pkg/logbackuppb", + "@com_github_pingcap_kvproto//pkg/metapb", "@com_github_pingcap_log//:log", "@com_github_spf13_pflag//:pflag", + "@com_github_tikv_client_go_v2//oracle", "@com_github_tikv_client_go_v2//tikv", + "@com_github_tikv_client_go_v2//util", + "@com_github_tikv_pd_client//:client", + "@org_golang_google_grpc//:grpc", "@org_golang_google_grpc//keepalive", "@org_golang_x_sync//errgroup", "@org_uber_go_multierr//:multierr", diff --git a/pkg/util/BUILD.bazel b/pkg/util/BUILD.bazel index a4f856dc962f0..be45751da4fba 100644 --- a/pkg/util/BUILD.bazel +++ b/pkg/util/BUILD.bazel @@ -58,6 +58,8 @@ go_library( "@io_etcd_go_etcd_client_v3//:client", "@io_etcd_go_etcd_client_v3//concurrency", "@org_golang_google_grpc//:grpc", + "@org_golang_google_protobuf//proto", + "@org_golang_google_protobuf//protoadapt", "@org_golang_x_sync//errgroup", "@org_uber_go_atomic//:atomic", "@org_uber_go_zap//:zap", @@ -101,6 +103,7 @@ go_test( "//pkg/util/memory", "@com_github_ngaut_pools//:pools", "@com_github_pingcap_errors//:errors", + "@com_github_pingcap_kvproto//pkg/brpb", "@com_github_stretchr_testify//assert", "@com_github_stretchr_testify//require", "@org_uber_go_atomic//:atomic", diff --git a/tests/realtikvtest/brietest/BUILD.bazel b/tests/realtikvtest/brietest/BUILD.bazel index 08f31926f3355..a5e07630775af 100644 --- a/tests/realtikvtest/brietest/BUILD.bazel +++ b/tests/realtikvtest/brietest/BUILD.bazel @@ -9,30 +9,43 @@ go_test( "brie_test.go", "main_test.go", "operator_test.go", + "pitr_test.go", ], flaky = True, race = "on", deps = [ + "//br/pkg/glue", + "//br/pkg/gluetidb", + "//br/pkg/logutil", + "//br/pkg/streamhelper", "//br/pkg/task", "//br/pkg/task/operator", "//pkg/config", + "//pkg/domain", "//pkg/executor", + "//pkg/kv", "//pkg/parser/mysql", "//pkg/session", "//pkg/testkit", "//pkg/testkit/testsetup", + "//pkg/util/printer", + "//pkg/util/table-filter", "//tests/realtikvtest", "@com_github_google_uuid//:uuid", "@com_github_pingcap_failpoint//:failpoint", + "@com_github_pingcap_kvproto//pkg/brpb", + "@com_github_pingcap_kvproto//pkg/encryptionpb", "@com_github_pingcap_kvproto//pkg/import_sstpb", "@com_github_pingcap_kvproto//pkg/kvrpcpb", "@com_github_pingcap_log//:log", "@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", "@org_golang_google_grpc//:grpc", "@org_golang_google_grpc//credentials/insecure", "@org_uber_go_goleak//:goleak", + "@org_uber_go_zap//:zap", "@org_uber_go_zap//zapcore", ], ) From 3e3c3da80a9ce4102eca225c9141a79024d70351 Mon Sep 17 00:00:00 2001 From: hillium Date: Thu, 19 Dec 2024 17:20:27 +0800 Subject: [PATCH 31/36] added force-flush command Signed-off-by: hillium --- br/cmd/br/operator.go | 21 +++++++++- br/pkg/task/operator/config.go | 59 ++++++++++++++++++----------- br/pkg/task/operator/force_flush.go | 6 ++- 3 files changed, 62 insertions(+), 24 deletions(-) diff --git a/br/cmd/br/operator.go b/br/cmd/br/operator.go index a578f3b39b6e1..474a6f9fc8644 100644 --- a/br/cmd/br/operator.go +++ b/br/cmd/br/operator.go @@ -35,6 +35,7 @@ func newOperatorCommand() *cobra.Command { cmd.AddCommand(newBase64ifyCommand()) cmd.AddCommand(newListMigrationsCommand()) cmd.AddCommand(newMigrateToCommand()) + cmd.AddCommand(newForceFlushCommand()) cmd.AddCommand(newChecksumCommand()) return cmd } @@ -113,7 +114,8 @@ func newMigrateToCommand() *cobra.Command { func newChecksumCommand() *cobra.Command { cmd := &cobra.Command{ - Short: "Calculate the checksum of the current cluster (specified by `-u`) " + + Short: "calculate the checksum with rewrite rules", + Long: "Calculate the checksum of the current cluster (specified by `-u`) " + "with applying the rewrite rules generated from a backup (specified by `-s`). " + "This can be used when you have the checksum of upstream elsewhere.", Args: cobra.NoArgs, @@ -130,3 +132,20 @@ func newChecksumCommand() *cobra.Command { operator.DefineFlagsForChecksumTableConfig(cmd.Flags()) return cmd } + +func newForceFlushCommand() *cobra.Command { + cmd := &cobra.Command{ + Short: "force a log backup task to flush", + Args: cobra.NoArgs, + RunE: func(cmd *cobra.Command, args []string) error { + cfg := operator.ForceFlushConfig{} + if err := cfg.ParseFromFlags(cmd.Flags()); err != nil { + return err + } + ctx := GetDefaultContext() + return operator.RunForceFlush(ctx, &cfg) + }, + } + operator.DefineFlagsForForceFlushConfig(cmd.Flags()) + return cmd +} diff --git a/br/pkg/task/operator/config.go b/br/pkg/task/operator/config.go index 6ddb6205489ab..03996beed3011 100644 --- a/br/pkg/task/operator/config.go +++ b/br/pkg/task/operator/config.go @@ -3,6 +3,7 @@ package operator import ( + "regexp" "time" "github.com/pingcap/errors" @@ -15,6 +16,17 @@ import ( const ( flagTableConcurrency = "table-concurrency" + flagStorePatterns = "stores" + flagTTL = "ttl" + flagSafePoint = "safepoint" + flagStorage = "storage" + flagLoadCreds = "load-creds" + flagJSON = "json" + flagRecent = "recent" + flagTo = "to" + flagBase = "base" + flagYes = "yes" + flagDryRun = "dry-run" ) type PauseGcConfig struct { @@ -28,8 +40,8 @@ type PauseGcConfig struct { } func DefineFlagsForPrepareSnapBackup(f *pflag.FlagSet) { - _ = f.DurationP("ttl", "i", 2*time.Minute, "The time-to-live of the safepoint.") - _ = f.Uint64P("safepoint", "t", 0, "The GC safepoint to be kept.") + _ = f.DurationP(flagTTL, "i", 2*time.Minute, "The time-to-live of the safepoint.") + _ = f.Uint64P(flagSafePoint, "t", 0, "The GC safepoint to be kept.") } // ParseFromFlags fills the config via the flags. @@ -39,11 +51,11 @@ func (cfg *PauseGcConfig) ParseFromFlags(flags *pflag.FlagSet) error { } var err error - cfg.SafePoint, err = flags.GetUint64("safepoint") + cfg.SafePoint, err = flags.GetUint64(flagSafePoint) if err != nil { return err } - cfg.TTL, err = flags.GetDuration("ttl") + cfg.TTL, err = flags.GetDuration(flagTTL) if err != nil { return err } @@ -59,8 +71,8 @@ type Base64ifyConfig struct { func DefineFlagsForBase64ifyConfig(flags *pflag.FlagSet) { storage.DefineFlags(flags) - flags.StringP("storage", "s", "", "The external storage input.") - flags.Bool("load-creds", false, "whether loading the credientials from current environment and marshal them to the base64 string. [!]") + flags.StringP(flagStorage, "s", "", "The external storage input.") + flags.Bool(flagLoadCreds, false, "whether loading the credientials from current environment and marshal them to the base64 string. [!]") } func (cfg *Base64ifyConfig) ParseFromFlags(flags *pflag.FlagSet) error { @@ -69,11 +81,11 @@ func (cfg *Base64ifyConfig) ParseFromFlags(flags *pflag.FlagSet) error { if err != nil { return err } - cfg.StorageURI, err = flags.GetString("storage") + cfg.StorageURI, err = flags.GetString(flagStorage) if err != nil { return err } - cfg.LoadCerd, err = flags.GetBool("load-creds") + cfg.LoadCerd, err = flags.GetBool(flagLoadCreds) if err != nil { return err } @@ -88,8 +100,8 @@ type ListMigrationConfig struct { func DefineFlagsForListMigrationConfig(flags *pflag.FlagSet) { storage.DefineFlags(flags) - flags.StringP("storage", "s", "", "the external storage input.") - flags.Bool("json", false, "output the result in json format.") + flags.StringP(flagStorage, "s", "", "the external storage input.") + flags.Bool(flagJSON, false, "output the result in json format.") } func (cfg *ListMigrationConfig) ParseFromFlags(flags *pflag.FlagSet) error { @@ -98,11 +110,11 @@ func (cfg *ListMigrationConfig) ParseFromFlags(flags *pflag.FlagSet) error { if err != nil { return err } - cfg.StorageURI, err = flags.GetString("storage") + cfg.StorageURI, err = flags.GetString(flagStorage) if err != nil { return err } - cfg.JSONOutput, err = flags.GetBool("json") + cfg.JSONOutput, err = flags.GetBool(flagJSON) if err != nil { return err } @@ -120,15 +132,6 @@ type MigrateToConfig struct { DryRun bool } -const ( - flagStorage = "storage" - flagRecent = "recent" - flagTo = "to" - flagBase = "base" - flagYes = "yes" - flagDryRun = "dry-run" -) - func DefineFlagsForMigrateToConfig(flags *pflag.FlagSet) { storage.DefineFlags(flags) flags.StringP(flagStorage, "s", "", "the external storage input.") @@ -188,12 +191,24 @@ func (cfg *MigrateToConfig) Verify() error { type ForceFlushConfig struct { task.Config + + StoresPattern *regexp.Regexp } func DefineFlagsForForceFlushConfig(f *pflag.FlagSet) { + f.String(flagStorePatterns, ".*", "The regexp to match the store peer address to be force flushed.") } -func (cfg *ForceFlushConfig) ParseFromFlags(flags *pflag.FlagSet) error { +func (cfg *ForceFlushConfig) ParseFromFlags(flags *pflag.FlagSet) (err error) { + storePat, err := flags.GetString(flagStorePatterns) + if err != nil { + return err + } + cfg.StoresPattern, err = regexp.Compile(storePat) + if err != nil { + return errors.Annotatef(err, "invalid expression in --%s", flagStorePatterns) + } + return cfg.Config.ParseFromFlags(flags) } diff --git a/br/pkg/task/operator/force_flush.go b/br/pkg/task/operator/force_flush.go index 121ebdd9a24b8..0c94f044cdd92 100644 --- a/br/pkg/task/operator/force_flush.go +++ b/br/pkg/task/operator/force_flush.go @@ -42,9 +42,13 @@ func RunForceFlush(ctx context.Context, cfg *ForceFlushConfig) error { return err } eg, ectx := errgroup.WithContext(ctx) + log.Info("About to start force flushing.", zap.Stringer("stores-pattern", cfg.StoresPattern)) for _, s := range tikvs { s := s - log.Info("Starting force flush TiKV.", zap.Uint64("store", s.GetId())) + if !cfg.StoresPattern.MatchString(s.Address) { + log.Info("Skipping not matched TiKV.", zap.Uint64("store", s.GetId()), zap.String("addr", s.Address)) + } + log.Info("Starting force flush TiKV.", zap.Uint64("store", s.GetId()), zap.String("addr", s.Address)) eg.Go(func() error { var logBackupCli logbackup.LogBackupClient err := stores.WithConn(ectx, s.GetId(), func(cc *grpc.ClientConn) { From 7b95659d30bc1e1e34fb36b0abc2350b62828d2a Mon Sep 17 00:00:00 2001 From: hillium Date: Fri, 20 Dec 2024 12:10:16 +0800 Subject: [PATCH 32/36] added two new ops Signed-off-by: hillium --- br/cmd/br/operator.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/br/cmd/br/operator.go b/br/cmd/br/operator.go index 474a6f9fc8644..abd0156a5457b 100644 --- a/br/cmd/br/operator.go +++ b/br/cmd/br/operator.go @@ -114,6 +114,7 @@ func newMigrateToCommand() *cobra.Command { func newChecksumCommand() *cobra.Command { cmd := &cobra.Command{ + Use: "checksum-as", Short: "calculate the checksum with rewrite rules", Long: "Calculate the checksum of the current cluster (specified by `-u`) " + "with applying the rewrite rules generated from a backup (specified by `-s`). " + @@ -135,6 +136,7 @@ func newChecksumCommand() *cobra.Command { func newForceFlushCommand() *cobra.Command { cmd := &cobra.Command{ + Use: "force-flush", Short: "force a log backup task to flush", Args: cobra.NoArgs, RunE: func(cmd *cobra.Command, args []string) error { From b8ef140cfd493a27f4cc8c50a650358837dec7a0 Mon Sep 17 00:00:00 2001 From: hillium Date: Fri, 20 Dec 2024 14:14:34 +0800 Subject: [PATCH 33/36] ignore err migration not found Signed-off-by: hillium --- br/pkg/stream/stream_metas.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/br/pkg/stream/stream_metas.go b/br/pkg/stream/stream_metas.go index 8250092ac9668..a5e288ec7f82f 100644 --- a/br/pkg/stream/stream_metas.go +++ b/br/pkg/stream/stream_metas.go @@ -750,7 +750,7 @@ func (m MigrationExt) AppendMigration(ctx context.Context, mig *pb.Migration) (i defer lock.Unlock(ctx) migs, err := m.Load(ctx) - if err != nil { + if err != nil && !berrors.Is(err, berrors.ErrMigrationNotFound) { return 0, err } newSN := 1 From c1d780062b57e462a09207fc5a2329a91a025b98 Mon Sep 17 00:00:00 2001 From: hillium Date: Fri, 20 Dec 2024 20:49:52 +0800 Subject: [PATCH 34/36] fix ignoring error Signed-off-by: hillium --- br/pkg/task/operator/checksum_table.go | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/br/pkg/task/operator/checksum_table.go b/br/pkg/task/operator/checksum_table.go index d93c71173dc9d..dbf9d751bd370 100644 --- a/br/pkg/task/operator/checksum_table.go +++ b/br/pkg/task/operator/checksum_table.go @@ -245,6 +245,9 @@ func (c *checksumTableCtx) runChecksum(ctx context.Context, reqs []request) ([]C zap.Int64("total", int64(total)), ) }) + if err != nil { + return err + } res := ChecksumResult{ DBName: req.dbName, TableName: req.tableName, @@ -256,7 +259,7 @@ func (c *checksumTableCtx) runChecksum(ctx context.Context, reqs []request) ([]C resultsMu.Lock() results = append(results, res) resultsMu.Unlock() - return err + return nil }) } From 5b262a4305a5aba20ddd8c9133d668f0b9f7d36e Mon Sep 17 00:00:00 2001 From: hillium Date: Mon, 23 Dec 2024 17:07:23 +0800 Subject: [PATCH 35/36] fixed BRIE via SQL Signed-off-by: hillium --- pkg/executor/brie.go | 7 ++++- tests/realtikvtest/brietest/pitr_test.go | 39 +++++++++++++++--------- 2 files changed, 31 insertions(+), 15 deletions(-) diff --git a/pkg/executor/brie.go b/pkg/executor/brie.go index 8ad6ec5b9b2f9..f2a544d4bec1b 100644 --- a/pkg/executor/brie.go +++ b/pkg/executor/brie.go @@ -372,11 +372,16 @@ func (b *executorBuilder) buildBRIE(s *ast.BRIEStmt, schema *expression.Schema) case len(s.Tables) != 0: tables := make([]filter.Table, 0, len(s.Tables)) for _, tbl := range s.Tables { - tables = append(tables, filter.Table{Name: tbl.Name.O, Schema: tbl.Schema.O}) + table := filter.Table{Name: tbl.Name.O, Schema: tbl.Schema.O} + tables = append(tables, table) + cfg.FilterStr = append(cfg.FilterStr, table.String()) } cfg.TableFilter = filter.NewTablesFilter(tables...) case len(s.Schemas) != 0: cfg.TableFilter = filter.NewSchemasFilter(s.Schemas...) + for _, schema := range s.Schemas { + cfg.FilterStr = append(cfg.FilterStr, fmt.Sprintf("`%s`.*", schema)) + } default: cfg.TableFilter = filter.All() } diff --git a/tests/realtikvtest/brietest/pitr_test.go b/tests/realtikvtest/brietest/pitr_test.go index 54d28ef3542a4..3d7410c082bed 100644 --- a/tests/realtikvtest/brietest/pitr_test.go +++ b/tests/realtikvtest/brietest/pitr_test.go @@ -7,6 +7,7 @@ import ( "math" "os" "path/filepath" + "regexp" "strings" "testing" "time" @@ -218,10 +219,11 @@ func (kit *LogBackupKit) forceFlush() { cfg := task.DefaultConfig() cfg.PD = append(cfg.PD, config.GetGlobalConfig().Path) err := operator.RunForceFlush(ctx, &operator.ForceFlushConfig{ - Config: cfg, + Config: cfg, + StoresPattern: regexp.MustCompile(".*"), }) if err != nil { - log.Warn("It seems this version of TiKV doesn't support force flush, the test may be much more slower.", + log.Warn("[TEST.forceFlush] It seems this version of TiKV doesn't support force flush, the test may be much more slower.", logutil.ShortError(err)) } return nil @@ -233,7 +235,7 @@ func (kit *LogBackupKit) forceFlushAndWait(taskName string) { kit.forceFlush() require.Eventually(kit.t, func() bool { ckpt := kit.CheckpointTSOf(taskName) - log.Info("checkpoint", zap.Uint64("checkpoint", ckpt), zap.Uint64("ts", ts)) + log.Info("[TEST.forceFlushAndWait] checkpoint", zap.Uint64("checkpoint", ckpt), zap.Uint64("ts", ts)) return ckpt >= ts }, 300*time.Second, 1*time.Second) time.Sleep(6 * time.Second) // Wait the storage checkpoint uploaded... @@ -255,26 +257,35 @@ func verifySimpleData(kit *LogBackupKit) { } func cleanSimpleData(kit *LogBackupKit) { - kit.tk.MustExec(fmt.Sprintf("DROP TABLE test.%s", kit.t.Name())) + kit.tk.MustExec(fmt.Sprintf("DROP TABLE IF EXISTS test.%s", kit.t.Name())) } -func TestPiTR(t *testing.T) { +func TestPiTRAndBackupInSQL(t *testing.T) { kit := NewLogBackupKit(t) - - taskName := "simple" createSimpleTableWithData(kit) - - ts := kit.TSO() - kit.RunFullBackup(func(bc *task.BackupConfig) { bc.BackupTS = ts }) - kit.RunLogStart(taskName, func(sc *task.StreamConfig) { sc.StartTS = ts }) - insertSimpleIncreaseData(kit) - kit.forceFlushAndWait(taskName) + taskName := t.Name() + kit.RunFullBackup(func(bc *task.BackupConfig) {}) cleanSimpleData(kit) + ts := kit.TSO() + kit.RunFullBackup(func(bc *task.BackupConfig) { + bc.Storage = "local://" + kit.base + "/full2" + bc.BackupTS = ts + }) + kit.RunLogStart(taskName, func(sc *task.StreamConfig) { + sc.StartTS = ts + }) + _ = kit.tk.MustQuery(fmt.Sprintf("RESTORE TABLE test.%s FROM '%s'", t.Name(), "local://"+kit.base+"/full")) + verifySimpleData(kit) + kit.forceFlushAndWait(taskName) + + cleanSimpleData(kit) kit.StopTaskIfExists(taskName) - kit.RunStreamRestore(func(rc *task.RestoreConfig) {}) + kit.RunStreamRestore(func(rc *task.RestoreConfig) { + rc.FullBackupStorage = "local://" + kit.base + "/full2" + }) verifySimpleData(kit) } From d67c740263c99cb068061b0bb30581192633e1a1 Mon Sep 17 00:00:00 2001 From: hillium Date: Tue, 24 Dec 2024 12:43:48 +0800 Subject: [PATCH 36/36] added some metrics Signed-off-by: hillium --- br/pkg/logutil/logging.go | 24 +++++++++++ br/pkg/restore/restorer.go | 7 ++-- br/pkg/restore/snap_client/import.go | 43 +++++++++++++------- br/pkg/restore/snap_client/pitr_collector.go | 35 ++++++++++++---- br/pkg/task/restore.go | 8 ++++ pkg/metrics/br.go | 30 ++++++++++++++ pkg/metrics/metrics.go | 4 ++ 7 files changed, 126 insertions(+), 25 deletions(-) create mode 100644 pkg/metrics/br.go diff --git a/br/pkg/logutil/logging.go b/br/pkg/logutil/logging.go index 353ca6622e896..22480e7b12d48 100644 --- a/br/pkg/logutil/logging.go +++ b/br/pkg/logutil/logging.go @@ -15,7 +15,9 @@ import ( "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/log" "github.com/pingcap/tidb/pkg/kv" + "github.com/pingcap/tidb/pkg/lightning/metric" "github.com/pingcap/tidb/pkg/util/redact" + "github.com/prometheus/client_golang/prometheus" "go.uber.org/zap" "go.uber.org/zap/zapcore" ) @@ -356,3 +358,25 @@ func (b HexBytes) String() string { func (b HexBytes) MarshalJSON() ([]byte, error) { return json.Marshal(hex.EncodeToString(b)) } + +func MarshalHistogram(m prometheus.Histogram) zapcore.ObjectMarshaler { + return zapcore.ObjectMarshalerFunc(func(mal zapcore.ObjectEncoder) error { + if m == nil { + return nil + } + + met := metric.ReadHistogram(m) + if met == nil || met.Histogram == nil { + return nil + } + + hist := met.Histogram + for _, b := range hist.GetBucket() { + key := fmt.Sprintf("lt_%f", b.GetUpperBound()) + mal.AddUint64(key, b.GetCumulativeCount()) + } + mal.AddUint64("count", hist.GetSampleCount()) + mal.AddFloat64("total", hist.GetSampleSum()) + return nil + }) +} diff --git a/br/pkg/restore/restorer.go b/br/pkg/restore/restorer.go index 9d999af9c09fc..c3e9034ce2001 100644 --- a/br/pkg/restore/restorer.go +++ b/br/pkg/restore/restorer.go @@ -276,7 +276,7 @@ func (m *MultiTablesRestorer) GoRestore(onProgress func(int64), batchFileSets .. m.ectx = opentracing.ContextWithSpan(m.ectx, span1) } - for _, batchFileSet := range batchFileSets { + for i, batchFileSet := range batchFileSets { if m.ectx.Err() != nil { log.Warn("Restoring encountered error and already stopped, give up remained files.", logutil.ShortError(m.ectx.Err())) @@ -287,15 +287,16 @@ func (m *MultiTablesRestorer) GoRestore(onProgress func(int64), batchFileSets .. } filesReplica := batchFileSet m.fileImporter.PauseForBackpressure() + cx := logutil.ContextWithField(m.ectx, zap.Int("sn", i)) m.workerPool.ApplyOnErrorGroup(m.eg, func() (restoreErr error) { fileStart := time.Now() defer func() { if restoreErr == nil { - log.Info("import files done", zap.Duration("take", time.Since(fileStart))) + logutil.CL(cx).Info("import files done", zap.Duration("take", time.Since(fileStart))) onProgress(int64(len(filesReplica))) } }() - if importErr := m.fileImporter.Import(m.ectx, filesReplica...); importErr != nil { + if importErr := m.fileImporter.Import(cx, filesReplica...); importErr != nil { return errors.Trace(importErr) } diff --git a/br/pkg/restore/snap_client/import.go b/br/pkg/restore/snap_client/import.go index fed9b2101c559..eabb5be7ede0f 100644 --- a/br/pkg/restore/snap_client/import.go +++ b/br/pkg/restore/snap_client/import.go @@ -41,6 +41,7 @@ import ( "github.com/pingcap/tidb/br/pkg/summary" "github.com/pingcap/tidb/br/pkg/utils" "github.com/pingcap/tidb/pkg/kv" + "github.com/pingcap/tidb/pkg/metrics" "github.com/pingcap/tidb/pkg/util/codec" kvutil "github.com/tikv/client-go/v2/util" "go.uber.org/zap" @@ -142,7 +143,7 @@ type SnapFileImporter struct { ingestTokensMap *storeTokenChannelMap closeCallbacks []func(*SnapFileImporter) error - beforeIngestCallbacks []func(context.Context, restore.BatchBackupFileSet) error + beforeIngestCallbacks []func(context.Context, restore.BatchBackupFileSet) (afterIngest func() error, err error) concurrencyPerStore uint @@ -373,12 +374,18 @@ func (importer *SnapFileImporter) Import( ctx context.Context, backupFileSets ...restore.BackupFileSet, ) error { + delayCbs := []func() error{} for i, cb := range importer.beforeIngestCallbacks { - if err := cb(ctx, backupFileSets); err != nil { + d, err := cb(ctx, backupFileSets) + if err != nil { return errors.Annotatef(err, "failed to executing the callback #%d", i) } + if d != nil { + delayCbs = append(delayCbs, d) + } } + importBegin := time.Now() // Rewrite the start key and end key of file to scan regions startKey, endKey, err := importer.getKeyRangeForFiles(backupFileSets) if err != nil { @@ -393,7 +400,7 @@ func (importer *SnapFileImporter) Import( return errors.Trace(errScanRegion) } - log.Debug("scan regions", logutil.Key("start key", startKey), logutil.Key("end key", endKey), zap.Int("count", len(regionInfos))) + logutil.CL(ctx).Debug("scan regions", logutil.Key("start key", startKey), logutil.Key("end key", endKey), zap.Int("count", len(regionInfos))) start := time.Now() // Try to download and ingest the file in every region for _, regionInfo := range regionInfos { @@ -401,18 +408,18 @@ func (importer *SnapFileImporter) Import( // Try to download file. downloadMetas, errDownload := importer.download(ctx, info, backupFileSets, importer.cipher, importer.apiVersion) if errDownload != nil { - log.Warn("download file failed, retry later", + logutil.CL(ctx).Warn("download file failed, retry later", logutil.Region(info.Region), logutil.Key("startKey", startKey), logutil.Key("endKey", endKey), logutil.ShortError(errDownload)) return errors.Trace(errDownload) } - log.Debug("download file done", zap.Stringer("take", time.Since(start)), + logutil.CL(ctx).Debug("download file done", zap.Stringer("take", time.Since(start)), logutil.Key("start", startKey), logutil.Key("end", endKey)) start = time.Now() if errIngest := importer.ingest(ctx, info, downloadMetas); errIngest != nil { - log.Warn("ingest file failed, retry later", + logutil.CL(ctx).Warn("ingest file failed, retry later", logutil.Key("start", startKey), logutil.Key("end", endKey), logutil.SSTMetas(downloadMetas), @@ -420,14 +427,22 @@ func (importer *SnapFileImporter) Import( zap.Error(errIngest)) return errors.Trace(errIngest) } - log.Debug("ingest file done", logutil.Key("start", startKey), logutil.Key("end", endKey), zap.Stringer("take", time.Since(start))) + logutil.CL(ctx).Debug("ingest file done", logutil.Key("start", startKey), logutil.Key("end", endKey), zap.Stringer("take", time.Since(start))) } return nil }, utils.NewImportSSTBackoffStrategy()) if err != nil { - log.Error("import sst file failed after retry, stop the whole progress", restore.ZapBatchBackupFileSet(backupFileSets), zap.Error(err)) + logutil.CL(ctx).Error("import sst file failed after retry, stop the whole progress", restore.ZapBatchBackupFileSet(backupFileSets), zap.Error(err)) return errors.Trace(err) } + metrics.RestoreImportFileSeconds.Observe(time.Since(importBegin).Seconds()) + + for i, cb := range delayCbs { + if err := cb(); err != nil { + return errors.Annotatef(err, "failed to execute the delaied callback #%d", i) + } + } + for _, files := range backupFileSets { for _, f := range files.SSTFiles { summary.CollectSuccessUnit(summary.TotalKV, 1, f.TotalKvs) @@ -538,15 +553,15 @@ func (importer *SnapFileImporter) download( failpoint.Inject("restore-storage-error", func(val failpoint.Value) { msg := val.(string) - log.Debug("failpoint restore-storage-error injected.", zap.String("msg", msg)) + logutil.CL(ctx).Debug("failpoint restore-storage-error injected.", zap.String("msg", msg)) e = errors.Annotate(e, msg) }) failpoint.Inject("restore-gRPC-error", func(_ failpoint.Value) { - log.Warn("the connection to TiKV has been cut by a neko, meow :3") + logutil.CL(ctx).Warn("the connection to TiKV has been cut by a neko, meow :3") e = status.Error(codes.Unavailable, "the connection to TiKV has been cut by a neko, meow :3") }) if isDecryptSstErr(e) { - log.Info("fail to decrypt when download sst, try again with no-crypt") + logutil.CL(ctx).Info("fail to decrypt when download sst, try again with no-crypt") if importer.kvMode == Raw || importer.kvMode == Txn { downloadMetas, e = importer.downloadRawKVSST(ctx, regionInfo, filesGroup, nil, apiVersion) } else { @@ -847,7 +862,7 @@ func (importer *SnapFileImporter) ingest( break } // do not get region info, wait a second and GetRegion() again. - log.Warn("ingest get region by key return nil", logutil.Region(info.Region), + logutil.CL(ctx).Warn("ingest get region by key return nil", logutil.Region(info.Region), logutil.SSTMetas(downloadMetas), ) time.Sleep(time.Second) @@ -857,7 +872,7 @@ func (importer *SnapFileImporter) ingest( if !split.CheckRegionEpoch(newInfo, info) { return errors.Trace(berrors.ErrKVEpochNotMatch) } - log.Debug("ingest sst returns not leader error, retry it", + logutil.CL(ctx).Debug("ingest sst returns not leader error, retry it", logutil.SSTMetas(downloadMetas), logutil.Region(info.Region), zap.Stringer("newLeader", newInfo.Leader)) @@ -900,7 +915,7 @@ func (importer *SnapFileImporter) ingestSSTs( Context: reqCtx, Ssts: sstMetas, } - log.Debug("ingest SSTs", logutil.SSTMetas(sstMetas), logutil.Leader(leader)) + logutil.CL(ctx).Debug("ingest SSTs", logutil.SSTMetas(sstMetas), logutil.Leader(leader)) resp, err := importer.importClient.MultiIngest(ctx, leader.GetStoreId(), req) return resp, errors.Trace(err) } diff --git a/br/pkg/restore/snap_client/pitr_collector.go b/br/pkg/restore/snap_client/pitr_collector.go index fe22df4119029..3cd798cff5205 100644 --- a/br/pkg/restore/snap_client/pitr_collector.go +++ b/br/pkg/restore/snap_client/pitr_collector.go @@ -12,11 +12,13 @@ import ( pb "github.com/pingcap/kvproto/pkg/brpb" "github.com/pingcap/log" berrors "github.com/pingcap/tidb/br/pkg/errors" + "github.com/pingcap/tidb/br/pkg/logutil" "github.com/pingcap/tidb/br/pkg/restore" "github.com/pingcap/tidb/br/pkg/storage" "github.com/pingcap/tidb/br/pkg/stream" "github.com/pingcap/tidb/br/pkg/streamhelper" "github.com/pingcap/tidb/br/pkg/summary" + "github.com/pingcap/tidb/pkg/metrics" "github.com/pingcap/tidb/pkg/util" "github.com/tikv/client-go/v2/oracle" pd "github.com/tikv/pd/client" @@ -79,19 +81,22 @@ func (c *pitrCollector) close() error { } -func (c *pitrCollector) onBatch(ctx context.Context, fileSets restore.BatchBackupFileSet) error { +func (c *pitrCollector) onBatch(ctx context.Context, fileSets restore.BatchBackupFileSet) (func() error, error) { if !c.enabled { - return nil + return nil, nil } if err := c.prepareMigIfNeeded(ctx); err != nil { - return err + return nil, err } + begin := time.Now() eg, ectx := errgroup.WithContext(ctx) + fileCount := 0 for _, fileSet := range fileSets { for _, file := range fileSet.SSTFiles { file := file + fileCount += 1 eg.Go(func() error { if err := c.putSST(ectx, file); err != nil { return errors.Annotatef(err, "failed to put sst %s", file.GetName()) @@ -110,12 +115,23 @@ func (c *pitrCollector) onBatch(ctx context.Context, fileSets restore.BatchBacku } } - err := eg.Wait() + err := c.persistExtraBackupMeta(ctx) if err != nil { - return err + return nil, errors.Annotatef(err, "failed to persist backup meta when finishing batch") } - return errors.Annotatef(c.persistExtraBackupMeta(ctx), "failed to persist backup meta when finishing batch") + waitDone := func() error { + err := eg.Wait() + if err != nil { + logutil.CL(ctx).Warn("Failed to upload SSTs for future PiTR.", logutil.ShortError(err)) + return err + } + + logutil.CL(ctx).Info("Uploaded a batch of SSTs for future PiTR.", + zap.Duration("take", time.Since(begin)), zap.Int("file-count", fileCount)) + return nil + } + return waitDone, nil } func (c *pitrCollector) doWithExtraBackupMetaLock(f func()) { @@ -143,6 +159,8 @@ func (c *pitrCollector) putSST(ctx context.Context, f *pb.File) error { return nil } + begin := time.Now() + f = util.ProtoV1Clone(f) out := c.sstPath(f.Name) @@ -160,6 +178,8 @@ func (c *pitrCollector) putSST(ctx context.Context, f *pb.File) error { f.Name = out c.doWithExtraBackupMetaLock(func() { c.extraBackupMeta.msg.Files = append(c.extraBackupMeta.msg.Files, f) }) + + metrics.RestoreUploadSSTForPiTRSeconds.Observe(time.Since(begin).Seconds()) return nil } @@ -189,7 +209,7 @@ func (c *pitrCollector) persistExtraBackupMeta(ctx context.Context) (err error) c.extraBackupMetaLock.Lock() defer c.extraBackupMetaLock.Unlock() - log.Info("Persisting extra backup meta.", zap.Stringer("uuid", c.restoreUUID), zap.String("path", c.metaPath())) + logutil.CL(ctx).Info("Persisting extra backup meta.", zap.Stringer("uuid", c.restoreUUID), zap.String("path", c.metaPath())) msg := c.extraBackupMeta.genMsg() bs, err := msg.Marshal() if err != nil { @@ -300,7 +320,6 @@ func newPiTRColl(ctx context.Context, deps PiTRCollDep) (*pitrCollector, error) return nil, errors.Trace(err) } coll.restoreStorage = restoreStrg - coll.resetCommitting() return coll, nil } diff --git a/br/pkg/task/restore.go b/br/pkg/task/restore.go index 26d94ccbfc079..ace31ccf42606 100644 --- a/br/pkg/task/restore.go +++ b/br/pkg/task/restore.go @@ -39,6 +39,7 @@ import ( "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta/model" + "github.com/pingcap/tidb/pkg/metrics" pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/util" "github.com/pingcap/tidb/pkg/util/collate" @@ -671,6 +672,11 @@ func DefaultRestoreConfig(commonConfig Config) RestoreConfig { return cfg } +func printRestoreMetrics() { + log.Info("Metric: import_file_seconds", zap.Object("metric", logutil.MarshalHistogram(metrics.RestoreImportFileSeconds))) + log.Info("Metric: upload_sst_for_pitr_seconds", zap.Object("metric", logutil.MarshalHistogram(metrics.RestoreUploadSSTForPiTRSeconds))) +} + // RunRestore starts a restore task inside the current goroutine. func RunRestore(c context.Context, g glue.Glue, cmdName string, cfg *RestoreConfig) error { etcdCLI, err := dialEtcdWithCfg(c, cfg.Config) @@ -704,6 +710,8 @@ func RunRestore(c context.Context, g glue.Glue, cmdName string, cfg *RestoreConf } defer mgr.Close() + defer printRestoreMetrics() + var restoreError error if IsStreamRestore(cmdName) { if err := version.CheckClusterVersion(c, mgr.GetPDClient(), version.CheckVersionForBRPiTR); err != nil { diff --git a/pkg/metrics/br.go b/pkg/metrics/br.go new file mode 100644 index 0000000000000..1754bd190a5a9 --- /dev/null +++ b/pkg/metrics/br.go @@ -0,0 +1,30 @@ +package metrics + +import "github.com/prometheus/client_golang/prometheus" + +var ( + RestoreImportFileSeconds prometheus.Histogram + RestoreUploadSSTForPiTRSeconds prometheus.Histogram +) + +func InitBRMetrics() { + RestoreImportFileSeconds = prometheus.NewHistogram(prometheus.HistogramOpts{ + Namespace: "tidb", + Subsystem: "br", + Name: "restore_import_file_seconds", + + Help: "The time cost for importing a file (send it to a file's all peers and put metadata)", + + Buckets: prometheus.ExponentialBuckets(0.01, 2, 14), + }) + + RestoreUploadSSTForPiTRSeconds = prometheus.NewHistogram(prometheus.HistogramOpts{ + Namespace: "tidb", + Subsystem: "br", + Name: "restore_upload_sst_for_pitr_seconds", + + Help: "The time cost for uploading SST files for point-in-time recovery", + + Buckets: prometheus.DefBuckets, + }) +} diff --git a/pkg/metrics/metrics.go b/pkg/metrics/metrics.go index 5b2e26ef23b97..a7e010e8a68ac 100644 --- a/pkg/metrics/metrics.go +++ b/pkg/metrics/metrics.go @@ -97,6 +97,10 @@ func InitMetrics() { InitInfoSchemaV2Metrics() timermetrics.InitTimerMetrics() + // For now, those metrics are initialized but not registered. + // They will be printed to log during restoring... + InitBRMetrics() + PanicCounter = NewCounterVec( prometheus.CounterOpts{ Namespace: "tidb",