From 6530d74133e7d0ee210bd85e6a93be9fa2378d6e Mon Sep 17 00:00:00 2001 From: James Liu Date: Wed, 3 Jan 2024 17:09:00 +1100 Subject: [PATCH 1/6] backup: Add a logger to the Manager Adds a logger field to be used by a subsequent commit for logging non-critical errors. --- internal/backup/backup.go | 5 ++++- internal/backup/backup_test.go | 14 +++++++------- internal/cli/gitalybackup/create.go | 2 +- internal/cli/gitalybackup/restore.go | 2 +- 4 files changed, 13 insertions(+), 10 deletions(-) diff --git a/internal/backup/backup.go b/internal/backup/backup.go index bc54cbaf08..32dee0f34e 100644 --- a/internal/backup/backup.go +++ b/internal/backup/backup.go @@ -161,6 +161,7 @@ type Manager struct { sink Sink conns *client.Pool locator Locator + logger log.Logger // repositoryFactory returns an abstraction over git repositories in order // to create and restore backups. @@ -168,7 +169,7 @@ type Manager struct { } // NewManager creates and returns initialized *Manager instance. -func NewManager(sink Sink, locator Locator, pool *client.Pool) *Manager { +func NewManager(sink Sink, logger log.Logger, locator Locator, pool *client.Pool) *Manager { return &Manager{ sink: sink, conns: pool, @@ -185,6 +186,7 @@ func NewManager(sink Sink, locator Locator, pool *client.Pool) *Manager { return newRemoteRepository(repo, conn), nil }, + logger: logger, } } @@ -208,6 +210,7 @@ func NewManagerLocal( return newLocalRepository(logger, storageLocator, gitCmdFactory, txManager, repoCounter, localRepo), nil }, + logger: logger, } } diff --git a/internal/backup/backup_test.go b/internal/backup/backup_test.go index 14f34f3a00..fcdd0a98da 100644 --- a/internal/backup/backup_test.go +++ b/internal/backup/backup_test.go @@ -56,7 +56,7 @@ func TestManager_RemoveRepository(t *testing.T) { locator, err := backup.ResolveLocator("pointer", sink) require.NoError(t, err) - fsBackup := backup.NewManager(sink, locator, pool) + fsBackup := backup.NewManager(sink, testhelper.SharedLogger(t), locator, pool) err = fsBackup.RemoveRepository(ctx, &backup.RemoveRepositoryRequest{ Server: storage.ServerInfo{Address: cfg.SocketPath, Token: cfg.Auth.Token}, Repo: repo, @@ -147,7 +147,7 @@ func TestManager_ListRepositories(t *testing.T) { locator, err := backup.ResolveLocator("pointer", sink) require.NoError(t, err) - fsBackup := backup.NewManager(sink, locator, pool) + fsBackup := backup.NewManager(sink, testhelper.SharedLogger(t), locator, pool) for storageName, repos := range tc.repos { actualRepos, err := fsBackup.ListRepositories(ctx, &backup.ListRepositoriesRequest{ @@ -186,7 +186,7 @@ func TestManager_Create(t *testing.T) { testhelper.MustClose(tb, pool) }) - return backup.NewManager(sink, locator, pool) + return backup.NewManager(sink, testhelper.SharedLogger(t), locator, pool) }, }, { @@ -433,7 +433,7 @@ func TestManager_Create_incremental(t *testing.T) { testhelper.MustClose(tb, pool) }) - return backup.NewManager(sink, locator, pool) + return backup.NewManager(sink, testhelper.SharedLogger(t), locator, pool) }, }, { @@ -579,7 +579,7 @@ func TestManager_Restore_latest(t *testing.T) { testhelper.MustClose(tb, pool) }) - return backup.NewManager(sink, locator, pool) + return backup.NewManager(sink, testhelper.SharedLogger(t), locator, pool) }, }, { @@ -919,7 +919,7 @@ func TestManager_Restore_specific(t *testing.T) { testhelper.MustClose(tb, pool) }) - return backup.NewManager(sink, locator, pool) + return backup.NewManager(sink, testhelper.SharedLogger(t), locator, pool) }, }, { @@ -1211,7 +1211,7 @@ func TestManager_CreateRestore_contextServerInfo(t *testing.T) { locator, err := backup.ResolveLocator("pointer", sink) require.NoError(t, err) - fsBackup := backup.NewManager(sink, locator, pool) + fsBackup := backup.NewManager(sink, testhelper.SharedLogger(t), locator, pool) ctx = testhelper.MergeIncomingMetadata(ctx, testcfg.GitalyServersMetadataFromCfg(t, cfg)) diff --git a/internal/cli/gitalybackup/create.go b/internal/cli/gitalybackup/create.go index be7e75cdbf..6e6396adca 100644 --- a/internal/cli/gitalybackup/create.go +++ b/internal/cli/gitalybackup/create.go @@ -140,7 +140,7 @@ func (cmd *createSubcommand) run(ctx context.Context, logger log.Logger, stdin i return fmt.Errorf("create: resolve locator: %w", err) } - manager = backup.NewManager(sink, locator, pool) + manager = backup.NewManager(sink, logger, locator, pool) } var opts []backup.PipelineOption diff --git a/internal/cli/gitalybackup/restore.go b/internal/cli/gitalybackup/restore.go index 7a3833f80b..c80c1e2ba0 100644 --- a/internal/cli/gitalybackup/restore.go +++ b/internal/cli/gitalybackup/restore.go @@ -132,7 +132,7 @@ func (cmd *restoreSubcommand) run(ctx context.Context, logger log.Logger, stdin if err != nil { return fmt.Errorf("restore: resolve locator: %w", err) } - manager = backup.NewManager(sink, locator, pool) + manager = backup.NewManager(sink, logger, locator, pool) } // Get the set of existing repositories keyed by storage. We'll later use this to determine any -- GitLab From 5b6cf16202ac621db9183c052d9fcfa5a8af29ea Mon Sep 17 00:00:00 2001 From: James Liu Date: Thu, 4 Jan 2024 13:50:15 +1100 Subject: [PATCH 2/6] backup: Restore custom hooks once only Moves the call to restore custom hooks out of the loop that iterates through each incremental backup. This is done for two reasons: - custom hooks are not backed up incrementally, so restoring them at each iteration of the loop isn't required. - a subsequent commit will alter how the Git data are restored, so separating custom hooks out allows for a cleaner approach. --- internal/backup/backup.go | 13 ++++++++++--- 1 file changed, 10 insertions(+), 3 deletions(-) diff --git a/internal/backup/backup.go b/internal/backup/backup.go index 32dee0f34e..d016c13b06 100644 --- a/internal/backup/backup.go +++ b/internal/backup/backup.go @@ -366,6 +366,16 @@ func (mgr *Manager) Restore(ctx context.Context, req *RestoreRequest) error { return fmt.Errorf("manager: %w", err) } + if len(backup.Steps) == 0 { + return fmt.Errorf("manager: no backup steps") + } + + // Restore custom hooks. The path is the same regardless of increment. + latestStep := backup.Steps[len(backup.Steps)-1] + if err := mgr.restoreCustomHooks(ctx, repo, latestStep.CustomHooksPath); err != nil { + return fmt.Errorf("manager: %w", err) + } + for _, step := range backup.Steps { refs, err := mgr.readRefs(ctx, step.RefPath) switch { @@ -396,9 +406,6 @@ func (mgr *Manager) Restore(ctx context.Context, req *RestoreRequest) error { return fmt.Errorf("manager: %w", err) } } - if err := mgr.restoreCustomHooks(ctx, repo, step.CustomHooksPath); err != nil { - return fmt.Errorf("manager: %w", err) - } } return nil } -- GitLab From e532aaed06334a86483959f85ce6999f8c114bcb Mon Sep 17 00:00:00 2001 From: James Liu Date: Thu, 4 Jan 2024 14:30:13 +1100 Subject: [PATCH 3/6] backup: Extract bundle restore into own function Extracts the loop responsible for restoring each step of a backup into a separate function, along with the call to remove the repository. As a consequence of the latter change, the tests have been updated to expect the repository to exist even if the backup does not. This change is done to facilitate a subsequent MR which will introduce another function that optimises the restore process by resetting the ref list rather than using bundles. --- internal/backup/backup.go | 43 ++++++++++++++++------------- internal/backup/server_side_test.go | 2 +- 2 files changed, 25 insertions(+), 20 deletions(-) diff --git a/internal/backup/backup.go b/internal/backup/backup.go index d016c13b06..28c1525b7f 100644 --- a/internal/backup/backup.go +++ b/internal/backup/backup.go @@ -351,29 +351,33 @@ func (mgr *Manager) Restore(ctx context.Context, req *RestoreRequest) error { return fmt.Errorf("manager: %w", err) } + if len(backup.Steps) == 0 { + return fmt.Errorf("manager: no backup steps") + } + + // If we can't reset the refs, perform a full restore by recreating the repo and cloning from the bundle. + if err := mgr.restoreFromBundle(ctx, repo, backup, req.AlwaysCreate); err != nil { + return fmt.Errorf("manager: restore from bundle: %w", err) + } + + // Restore custom hooks. The path is the same regardless of increment. + latestStep := backup.Steps[len(backup.Steps)-1] + return mgr.restoreCustomHooks(ctx, repo, latestStep.CustomHooksPath) +} + +func (mgr *Manager) restoreFromBundle(ctx context.Context, repo Repository, backup *Backup, alwaysCreate bool) error { hash, err := git.ObjectHashByFormat(backup.ObjectFormat) if err != nil { - return fmt.Errorf("manager: %w", err) + return err } defaultBranch, defaultBranchKnown := git.ReferenceName(backup.HeadReference).Branch() if err := repo.Remove(ctx); err != nil { - return fmt.Errorf("manager: %w", err) + return err } - if err := repo.Create(ctx, hash, defaultBranch); err != nil { - return fmt.Errorf("manager: %w", err) - } - - if len(backup.Steps) == 0 { - return fmt.Errorf("manager: no backup steps") - } - - // Restore custom hooks. The path is the same regardless of increment. - latestStep := backup.Steps[len(backup.Steps)-1] - if err := mgr.restoreCustomHooks(ctx, repo, latestStep.CustomHooksPath); err != nil { - return fmt.Errorf("manager: %w", err) + return err } for _, step := range backup.Steps { @@ -386,27 +390,28 @@ func (mgr *Manager) Restore(ctx context.Context, req *RestoreRequest) error { // not know which repository is which type so here we accept a // parameter to tell us to employ this behaviour. Since the // repository has already been created, we simply skip cleaning up. - if req.AlwaysCreate { + if alwaysCreate { return nil } if err := repo.Remove(ctx); err != nil { - return fmt.Errorf("manager: remove on skipped: %w", err) + return fmt.Errorf("remove on skipped: %w", err) } - return fmt.Errorf("manager: %w: %s", ErrSkipped, err.Error()) + return fmt.Errorf("%w: %s", ErrSkipped, err.Error()) case err != nil: - return fmt.Errorf("manager: %w", err) + return fmt.Errorf("read refs: %w", err) } // Git bundles can not be created for empty repositories. Since empty // repository backups do not contain a bundle, skip bundle restoration. if len(refs) > 0 { if err := mgr.restoreBundle(ctx, repo, step.BundlePath, !defaultBranchKnown); err != nil { - return fmt.Errorf("manager: %w", err) + return fmt.Errorf("restore bundle: %w", err) } } } + return nil } diff --git a/internal/backup/server_side_test.go b/internal/backup/server_side_test.go index eedbed3cd6..990bda29f7 100644 --- a/internal/backup/server_side_test.go +++ b/internal/backup/server_side_test.go @@ -212,7 +212,7 @@ func TestServerSideAdapter_Restore(t *testing.T) { backupID: "", } }, - expectedErr: fmt.Errorf("server-side restore: %w: rpc error: code = FailedPrecondition desc = restore repository: manager: repository skipped: read refs: doesn't exist", backup.ErrSkipped), + expectedErr: fmt.Errorf("server-side restore: %w: rpc error: code = FailedPrecondition desc = restore repository: manager: restore from bundle: repository skipped: read refs: doesn't exist", backup.ErrSkipped), }, } { tc := tc -- GitLab From b337460bc9054ebfd21db22513e61a04955deeb9 Mon Sep 17 00:00:00 2001 From: James Liu Date: Mon, 8 Jan 2024 15:58:44 +1100 Subject: [PATCH 4/6] backup: Add ResetRefs to the Repository interface Modifies the Repository interface to add a function called `ResetRefs` which resets the references in the repository to mirror a provided list of references. This will be used to optimise repository restores, as we can simply apply the list of refs from the backup to the existing repo rather than recreating the repo from the backed-up bundle. Additionally, the `NewRemoteRepository` and `NewLocalRepository` constructors have exported to facilitate unit testing. I dislike this approach, but we get an import cycle in `repository_test.go` otherwise, since it'll need to be defined within the `backup` package instead of `backup_test`, and it imports the `testserver` package which in turn imports the `backup` package. --- internal/backup/backup.go | 14 ++- internal/backup/repository.go | 148 ++++++++++++++++++++++++++++- internal/backup/repository_test.go | 129 +++++++++++++++++++++++++ 3 files changed, 287 insertions(+), 4 deletions(-) create mode 100644 internal/backup/repository_test.go diff --git a/internal/backup/backup.go b/internal/backup/backup.go index 28c1525b7f..847f9bc920 100644 --- a/internal/backup/backup.go +++ b/internal/backup/backup.go @@ -131,6 +131,11 @@ type Repository interface { ObjectHash(ctx context.Context) (git.ObjectHash, error) // HeadReference fetches the reference pointed to by HEAD. HeadReference(ctx context.Context) (git.ReferenceName, error) + // ResetRefs attempts to reset the list of refs in the repository to match the + // specified refs slice. This can fail if objects pointed to by a ref no longer + // exists in the repository. The list of refs should not include the symbolic + // HEAD reference. + ResetRefs(ctx context.Context, refs []git.Reference) error } // ResolveLocator returns a locator implementation based on a locator identifier. @@ -184,7 +189,7 @@ func NewManager(sink Sink, logger log.Logger, locator Locator, pool *client.Pool return nil, err } - return newRemoteRepository(repo, conn), nil + return NewRemoteRepository(repo, conn), nil }, logger: logger, } @@ -208,7 +213,7 @@ func NewManagerLocal( repositoryFactory: func(ctx context.Context, repo *gitalypb.Repository, server storage.ServerInfo) (Repository, error) { localRepo := localrepo.New(logger, storageLocator, gitCmdFactory, catfileCache, repo) - return newLocalRepository(logger, storageLocator, gitCmdFactory, txManager, repoCounter, localRepo), nil + return NewLocalRepository(logger, storageLocator, gitCmdFactory, txManager, repoCounter, localRepo), nil }, logger: logger, } @@ -527,6 +532,11 @@ func (mgr *Manager) readRefs(ctx context.Context, path string) ([]git.Reference, return refs, fmt.Errorf("read refs: %w", err) } + // HEAD is tracked as a symbolic reference in the backup manifest and will be restored separately. + if ref.Name == "HEAD" { + continue + } + refs = append(refs, ref) } diff --git a/internal/backup/repository.go b/internal/backup/repository.go index beef0c3d8d..0a67218825 100644 --- a/internal/backup/repository.go +++ b/internal/backup/repository.go @@ -10,6 +10,7 @@ import ( "gitlab.com/gitlab-org/gitaly/v16/internal/git" "gitlab.com/gitlab-org/gitaly/v16/internal/git/localrepo" + "gitlab.com/gitlab-org/gitaly/v16/internal/git/updateref" "gitlab.com/gitlab-org/gitaly/v16/internal/gitaly/repoutil" "gitlab.com/gitlab-org/gitaly/v16/internal/gitaly/storage" "gitlab.com/gitlab-org/gitaly/v16/internal/gitaly/storage/counter" @@ -31,7 +32,9 @@ type remoteRepository struct { conn *grpc.ClientConn } -func newRemoteRepository(repo *gitalypb.Repository, conn *grpc.ClientConn) *remoteRepository { +// NewRemoteRepository returns a repository accessor that operates on a remote +// repository. +func NewRemoteRepository(repo *gitalypb.Repository, conn *grpc.ClientConn) *remoteRepository { return &remoteRepository{ repo: repo, conn: conn, @@ -192,6 +195,27 @@ func (s *createBundleFromRefListSender) Send() error { return s.stream.Send(&s.chunk) } +// updateRefsSender chunks requests to the UpdateReferences RPC. +type updateRefsSender struct { + refs []*gitalypb.UpdateReferencesRequest_Update + send func([]*gitalypb.UpdateReferencesRequest_Update) error +} + +// Reset should create a fresh response message. +func (s *updateRefsSender) Reset() { + s.refs = s.refs[:0] +} + +// Append should append the given item to the slice in the current response message +func (s *updateRefsSender) Append(msg proto.Message) { + s.refs = append(s.refs, msg.(*gitalypb.UpdateReferencesRequest_Update)) +} + +// Send should send the current response message +func (s *updateRefsSender) Send() error { + return s.send(s.refs) +} + // Remove removes the repository. Does not return an error if the repository // cannot be found. func (rr *remoteRepository) Remove(ctx context.Context) error { @@ -208,6 +232,76 @@ func (rr *remoteRepository) Remove(ctx context.Context) error { return nil } +// ResetRefs attempts to reset the list of refs in the repository to match the +// specified refs slice. Do not include the symbolic HEAD reference in the list. +func (rr *remoteRepository) ResetRefs(ctx context.Context, refs []git.Reference) error { + if len(refs) == 0 { + return errors.New("empty refs list") + } + + refClient := rr.newRefClient() + + _, err := refClient.DeleteRefs(ctx, + &gitalypb.DeleteRefsRequest{ + Repository: rr.repo, + // While the DeleteRefs RPC doesn't delete HEAD, we add it to the exceptions list as a + // workaround to instruct the RPC to delete all references in the repository. + // https://gitlab.com/gitlab-org/gitaly/-/issues/5795 tracks the enhancement to the RPC to + // eliminate this workaround. + ExceptWithPrefix: [][]byte{[]byte("HEAD")}, + }) + if err != nil { + return fmt.Errorf("delete existing refs: %w", err) + } + + stream, err := refClient.UpdateReferences(ctx) + if err != nil { + return fmt.Errorf("open stream: %w", err) + } + + // We need to send the first update without the chunker, because the RPC expects the `Repository` field to be + // empty in subsequent messages. We also need to send the first reference because the RPC expects at least one + // update. + if err := stream.Send(&gitalypb.UpdateReferencesRequest{ + Repository: rr.repo, + Updates: []*gitalypb.UpdateReferencesRequest_Update{ + { + Reference: []byte(refs[0].Name), + NewObjectId: []byte(refs[0].Target), + }, + }, + }); err != nil { + return fmt.Errorf("send initial request: %w", err) + } + + chunker := chunk.New(&updateRefsSender{ + send: func(updates []*gitalypb.UpdateReferencesRequest_Update) error { + return stream.Send(&gitalypb.UpdateReferencesRequest{ + Updates: updates, + }) + }, + }) + + for _, ref := range refs[1:] { + if err := chunker.Send(&gitalypb.UpdateReferencesRequest_Update{ + Reference: []byte(ref.Name), + NewObjectId: []byte(ref.Target), + }); err != nil { + return fmt.Errorf("send ref update: %w", err) + } + } + + if err := chunker.Flush(); err != nil { + return fmt.Errorf("flush ref update chunker: %w", err) + } + + if _, err := stream.CloseAndRecv(); err != nil { + return fmt.Errorf("close stream: %w", err) + } + + return nil +} + // Create creates the repository. func (rr *remoteRepository) Create(ctx context.Context, hash git.ObjectHash, defaultBranch string) error { repoClient := rr.newRepoClient() @@ -325,7 +419,9 @@ type localRepository struct { repo *localrepo.Repo } -func newLocalRepository( +// NewLocalRepository returns a repository accessor that operates on a local +// repository. +func NewLocalRepository( logger log.Logger, locator storage.Locator, gitCmdFactory git.CommandFactory, @@ -468,3 +564,51 @@ func (r *localRepository) HeadReference(ctx context.Context) (git.ReferenceName, return head, nil } + +// ResetRefs attempts to reset the list of refs in the repository to match the +// specified refs slice. Do not include the symbolic HEAD reference in the list. +func (r *localRepository) ResetRefs(ctx context.Context, refs []git.Reference) (returnedErr error) { + u, err := updateref.New(ctx, r.repo) + if err != nil { + return fmt.Errorf("error when running creating new updater: %w", err) + } + defer func() { + if err := u.Close(); err != nil && returnedErr == nil { + returnedErr = fmt.Errorf("close updater: %w", err) + } + }() + + if err := u.Start(); err != nil { + return fmt.Errorf("start delete existing refs transaction: %w", err) + } + + existingRefs, err := r.repo.GetReferences(ctx) + if err != nil { + return fmt.Errorf("get existing refs: %w", err) + } + for _, ref := range existingRefs { + if err := u.Delete(ref.Name); err != nil { + return fmt.Errorf("delete existing ref: %w", err) + } + } + + if err := u.Commit(); err != nil { + return fmt.Errorf("commit remove existing refs: %w", err) + } + + if err := u.Start(); err != nil { + return fmt.Errorf("start reset refs transaction: %w", err) + } + + for _, ref := range refs { + if err := u.Update(ref.Name, git.ObjectID(ref.Target), ""); err != nil { + return fmt.Errorf("reset ref: %w", err) + } + } + + if err := u.Commit(); err != nil { + return fmt.Errorf("commit reset refs: %w", err) + } + + return nil +} diff --git a/internal/backup/repository_test.go b/internal/backup/repository_test.go new file mode 100644 index 0000000000..942eb7244c --- /dev/null +++ b/internal/backup/repository_test.go @@ -0,0 +1,129 @@ +package backup_test + +import ( + "testing" + + "github.com/stretchr/testify/require" + "gitlab.com/gitlab-org/gitaly/v16/internal/backup" + "gitlab.com/gitlab-org/gitaly/v16/internal/git" + "gitlab.com/gitlab-org/gitaly/v16/internal/git/catfile" + "gitlab.com/gitlab-org/gitaly/v16/internal/git/gittest" + "gitlab.com/gitlab-org/gitaly/v16/internal/git/localrepo" + "gitlab.com/gitlab-org/gitaly/v16/internal/gitaly/config" + "gitlab.com/gitlab-org/gitaly/v16/internal/gitaly/service/setup" + "gitlab.com/gitlab-org/gitaly/v16/internal/gitaly/storage/counter" + "gitlab.com/gitlab-org/gitaly/v16/internal/gitaly/transaction" + "gitlab.com/gitlab-org/gitaly/v16/internal/grpc/client" + "gitlab.com/gitlab-org/gitaly/v16/internal/testhelper" + "gitlab.com/gitlab-org/gitaly/v16/internal/testhelper/testcfg" + "gitlab.com/gitlab-org/gitaly/v16/internal/testhelper/testserver" + "golang.org/x/exp/slices" +) + +func removeHeadReference(refs []git.Reference) []git.Reference { + for i := range refs { + if refs[i].Name == "HEAD" { + return slices.Delete(refs, i, i+1) + } + } + + return refs +} + +func TestRemoteRepository_ResetRefs(t *testing.T) { + cfg := testcfg.Build(t) + cfg.SocketPath = testserver.RunGitalyServer(t, cfg, setup.RegisterAll) + ctx := testhelper.Context(t) + + repo, repoPath := gittest.CreateRepository(t, ctx, cfg) + + pool := client.NewPool() + defer testhelper.MustClose(t, pool) + + conn, err := pool.Dial(ctx, cfg.SocketPath, "") + require.NoError(t, err) + + rr := backup.NewRemoteRepository(repo, conn) + + // Create some commits + c0 := gittest.WriteCommit(t, cfg, repoPath, gittest.WithBranch("main")) + c1 := gittest.WriteCommit(t, cfg, repoPath, gittest.WithParents(c0), gittest.WithBranch("main")) + c2 := gittest.WriteCommit(t, cfg, repoPath, gittest.WithParents(c1), gittest.WithBranch("branch-1")) + + // "Snapshot" the refs to pretend this is our backup. + backupRefState, err := rr.ListRefs(ctx) + require.NoError(t, err) + backupRefState = removeHeadReference(backupRefState) + + // Create some more commits + gittest.WriteCommit(t, cfg, repoPath, gittest.WithParents(c1), gittest.WithBranch("main")) + gittest.WriteCommit(t, cfg, repoPath, gittest.WithParents(c2), gittest.WithBranch("branch-1")) + gittest.WriteCommit(t, cfg, repoPath, gittest.WithParents(c2), gittest.WithBranch("branch-2")) + + intermediateRefState, err := rr.ListRefs(ctx) + require.NoError(t, err) + require.Equal(t, 4, len(intermediateRefState)) // 3 branches + HEAD + + // Reset the state of the refs to the backup. + require.NoError(t, rr.ResetRefs(ctx, backupRefState)) + + actualRefState, err := rr.ListRefs(ctx) + require.NoError(t, err) + + actualRefState = removeHeadReference(actualRefState) + require.Equal(t, backupRefState, actualRefState) +} + +func TestLocalRepository_ResetRefs(t *testing.T) { + if testhelper.IsPraefectEnabled() { + t.Skip("local backup manager expects to operate on the local filesystem so cannot operate through praefect") + } + + cfg := testcfg.Build(t) + ctx := testhelper.Context(t) + + repo, repoPath := gittest.CreateRepository(t, ctx, cfg, gittest.CreateRepositoryConfig{ + SkipCreationViaService: true, + }) + + gitCmdFactory := gittest.NewCommandFactory(t, cfg) + txManager := transaction.NewTrackingManager() + repoCounter := counter.NewRepositoryCounter(cfg.Storages) + locator := config.NewLocator(cfg) + catfileCache := catfile.NewCache(cfg) + t.Cleanup(catfileCache.Stop) + + lr := localrepo.New(testhelper.SharedLogger(t), locator, gitCmdFactory, catfileCache, repo) + localRepo := backup.NewLocalRepository( + testhelper.SharedLogger(t), + locator, + gitCmdFactory, + txManager, + repoCounter, + lr) + + // Create some commits + c0 := gittest.WriteCommit(t, cfg, repoPath, gittest.WithBranch("main")) + c1 := gittest.WriteCommit(t, cfg, repoPath, gittest.WithParents(c0), gittest.WithBranch("main")) + c2 := gittest.WriteCommit(t, cfg, repoPath, gittest.WithParents(c1), gittest.WithBranch("branch-1")) + + // "Snapshot" the refs to pretend this is our backup. + backupRefState, err := lr.GetReferences(ctx) + require.NoError(t, err) + + // Create some more commits + gittest.WriteCommit(t, cfg, repoPath, gittest.WithParents(c1), gittest.WithBranch("main")) + gittest.WriteCommit(t, cfg, repoPath, gittest.WithParents(c2), gittest.WithBranch("branch-1")) + gittest.WriteCommit(t, cfg, repoPath, gittest.WithParents(c2), gittest.WithBranch("branch-2")) + + intermediateRefState, err := lr.GetReferences(ctx) + require.NoError(t, err) + require.Equal(t, 3, len(intermediateRefState)) // 3 branches + + // Reset the state of the refs to the backup. + require.NoError(t, localRepo.ResetRefs(ctx, backupRefState)) + actualRefState, err := lr.GetReferences(ctx) + require.NoError(t, err) + + require.Equal(t, backupRefState, actualRefState) +} -- GitLab From 63ebf7bda1a778b346c9050283fc3b73b8f04924 Mon Sep 17 00:00:00 2001 From: James Liu Date: Thu, 25 Jan 2024 16:39:29 +1100 Subject: [PATCH 5/6] backup: Add SetHeadReference to Repository Modifies the Repository interface with a new `SetHeadReference` method. This method sets the symbolic HEAD reference of the repo; for local repos it uses `SetDefaultBranch`, and for remote repos it uses the `WriteRef` RPC which handles HEAD as a special case. We update the HEAD separately from the rest of the refs, because the associated branch is tracked by the manifest file rather than the .refs artefact. --- internal/backup/backup.go | 3 ++ internal/backup/repository.go | 21 ++++++++ internal/backup/repository_test.go | 86 ++++++++++++++++++++++++++++++ 3 files changed, 110 insertions(+) diff --git a/internal/backup/backup.go b/internal/backup/backup.go index 847f9bc920..d7760bdf5b 100644 --- a/internal/backup/backup.go +++ b/internal/backup/backup.go @@ -136,6 +136,9 @@ type Repository interface { // exists in the repository. The list of refs should not include the symbolic // HEAD reference. ResetRefs(ctx context.Context, refs []git.Reference) error + // SetHeadReference sets the symbolic HEAD reference of the repository to the + // given target, for example a branch name. + SetHeadReference(ctx context.Context, target git.ReferenceName) error } // ResolveLocator returns a locator implementation based on a locator identifier. diff --git a/internal/backup/repository.go b/internal/backup/repository.go index 0a67218825..859ef6cfbe 100644 --- a/internal/backup/repository.go +++ b/internal/backup/repository.go @@ -302,6 +302,22 @@ func (rr *remoteRepository) ResetRefs(ctx context.Context, refs []git.Reference) return nil } +// SetHeadReference sets the symbolic HEAD reference of the repository. +func (rr *remoteRepository) SetHeadReference(ctx context.Context, target git.ReferenceName) error { + repoClient := rr.newRepoClient() + + _, err := repoClient.WriteRef(ctx, &gitalypb.WriteRefRequest{ + Repository: rr.repo, + Ref: []byte("HEAD"), + Revision: []byte(target), + }) + if err != nil { + return fmt.Errorf("write HEAD ref: %w", err) + } + + return nil +} + // Create creates the repository. func (rr *remoteRepository) Create(ctx context.Context, hash git.ObjectHash, defaultBranch string) error { repoClient := rr.newRepoClient() @@ -612,3 +628,8 @@ func (r *localRepository) ResetRefs(ctx context.Context, refs []git.Reference) ( return nil } + +// SetHeadReference sets the symbolic HEAD reference of the repository. +func (r *localRepository) SetHeadReference(ctx context.Context, target git.ReferenceName) error { + return r.repo.SetDefaultBranch(ctx, r.txManager, target) +} diff --git a/internal/backup/repository_test.go b/internal/backup/repository_test.go index 942eb7244c..bf1bf0fa35 100644 --- a/internal/backup/repository_test.go +++ b/internal/backup/repository_test.go @@ -127,3 +127,89 @@ func TestLocalRepository_ResetRefs(t *testing.T) { require.Equal(t, backupRefState, actualRefState) } + +func TestRemoteRepository_SetHeadReference(t *testing.T) { + testhelper.SkipWithReftable(t, "SetHeadReference modifies HEAD through the filesystem directly") + + cfg := testcfg.Build(t) + testcfg.BuildGitalyHooks(t, cfg) + cfg.SocketPath = testserver.RunGitalyServer(t, cfg, setup.RegisterAll) + ctx := testhelper.Context(t) + + repo, repoPath := gittest.CreateRepository(t, ctx, cfg) + + pool := client.NewPool() + defer testhelper.MustClose(t, pool) + + conn, err := pool.Dial(ctx, cfg.SocketPath, "") + require.NoError(t, err) + + rr := backup.NewRemoteRepository(repo, conn) + + c0 := gittest.WriteCommit(t, cfg, repoPath, gittest.WithBranch(git.DefaultBranch)) + expectedHead, err := rr.HeadReference(ctx) + require.NoError(t, err) + + gittest.WriteCommit(t, cfg, repoPath, gittest.WithParents(c0), gittest.WithBranch("branch-1")) + gittest.Exec(t, cfg, "-C", repoPath, "symbolic-ref", "HEAD", "refs/heads/branch-1") + + newHead, err := rr.HeadReference(ctx) + require.NoError(t, err) + + require.NoError(t, rr.SetHeadReference(ctx, expectedHead)) + + actualHead, err := rr.HeadReference(ctx) + require.NoError(t, err) + + require.Equal(t, expectedHead, actualHead) + require.NotEqual(t, newHead, actualHead) +} + +func TestLocalRepository_SetHeadReference(t *testing.T) { + if testhelper.IsPraefectEnabled() { + t.Skip("local backup manager expects to operate on the local filesystem so cannot operate through praefect") + } + + testhelper.SkipWithReftable(t, "SetHeadReference modifies HEAD through the filesystem directly") + + cfg := testcfg.Build(t) + ctx := testhelper.Context(t) + + repo, repoPath := gittest.CreateRepository(t, ctx, cfg, gittest.CreateRepositoryConfig{ + SkipCreationViaService: true, + }) + + gitCmdFactory := gittest.NewCommandFactory(t, cfg) + txManager := transaction.NewTrackingManager() + repoCounter := counter.NewRepositoryCounter(cfg.Storages) + locator := config.NewLocator(cfg) + catfileCache := catfile.NewCache(cfg) + t.Cleanup(catfileCache.Stop) + + localRepo := backup.NewLocalRepository( + testhelper.SharedLogger(t), + locator, + gitCmdFactory, + txManager, + repoCounter, + localrepo.New(testhelper.SharedLogger(t), locator, gitCmdFactory, catfileCache, repo)) + + c0 := gittest.WriteCommit(t, cfg, repoPath, gittest.WithBranch(git.DefaultBranch)) + + expectedHead, err := localRepo.HeadReference(ctx) + require.NoError(t, err) + + gittest.WriteCommit(t, cfg, repoPath, gittest.WithParents(c0), gittest.WithBranch("branch-1")) + gittest.Exec(t, cfg, "-C", repoPath, "symbolic-ref", "HEAD", "refs/heads/branch-1") + + newHead, err := localRepo.HeadReference(ctx) + require.NoError(t, err) + + require.NoError(t, localRepo.SetHeadReference(ctx, expectedHead)) + + actualHead, err := localRepo.HeadReference(ctx) + require.NoError(t, err) + + require.Equal(t, expectedHead, actualHead) + require.NotEqual(t, newHead, actualHead) +} -- GitLab From 4cb549eead64446d5f74b6b713ba29ba78114a76 Mon Sep 17 00:00:00 2001 From: James Liu Date: Mon, 15 Jan 2024 12:52:24 +1100 Subject: [PATCH 6/6] backup: Try resetting refs for an optimal restore Instead of recreating the repository from scratch during a restore, try resetting the refs in the repository to the same state as they are in the backup. If this doesn't work (for example if objects get pruned), we proceed with recreating the repository from the bundle. --- internal/backup/backup.go | 45 ++- internal/backup/backup_test.go | 334 ++++++++++++------ internal/backup/repository_test.go | 1 + .../repository/restore_repository_test.go | 2 +- 4 files changed, 267 insertions(+), 115 deletions(-) diff --git a/internal/backup/backup.go b/internal/backup/backup.go index d7760bdf5b..38518b611f 100644 --- a/internal/backup/backup.go +++ b/internal/backup/backup.go @@ -13,6 +13,7 @@ import ( "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/promauto" + "github.com/sirupsen/logrus" "gitlab.com/gitlab-org/gitaly/v16/internal/git" "gitlab.com/gitlab-org/gitaly/v16/internal/git/catfile" "gitlab.com/gitlab-org/gitaly/v16/internal/git/localrepo" @@ -363,16 +364,52 @@ func (mgr *Manager) Restore(ctx context.Context, req *RestoreRequest) error { return fmt.Errorf("manager: no backup steps") } - // If we can't reset the refs, perform a full restore by recreating the repo and cloning from the bundle. - if err := mgr.restoreFromBundle(ctx, repo, backup, req.AlwaysCreate); err != nil { - return fmt.Errorf("manager: restore from bundle: %w", err) + // Restore Git objects, potentially from increments. + if err := mgr.restoreFromRefs(ctx, repo, backup); err != nil { + mgr.logger.WithFields(log.Fields{ + "storage": req.Repository.GetStorageName(), + "relative_path": req.Repository.GetRelativePath(), + "backup_id": backup.ID, + logrus.ErrorKey: err, + }).Warn("unable to reset refs. Proceeding with a normal restore") + + // If we can't reset the refs, perform a full restore by recreating the repo and cloning from the bundle. + if err := mgr.restoreFromBundle(ctx, repo, backup, req.AlwaysCreate); err != nil { + return fmt.Errorf("manager: restore from bundle: %w", err) + } } - // Restore custom hooks. The path is the same regardless of increment. + // Restore custom hooks. Each custom hooks archive contains the entirety of the hooks, so + // we can just restore the most recent archive. latestStep := backup.Steps[len(backup.Steps)-1] return mgr.restoreCustomHooks(ctx, repo, latestStep.CustomHooksPath) } +func (mgr *Manager) restoreFromRefs(ctx context.Context, repo Repository, backup *Backup) error { + latestStep := backup.Steps[len(backup.Steps)-1] + refs, err := mgr.readRefs(ctx, latestStep.RefPath) + if err != nil { + return fmt.Errorf("read refs from backup: %w", err) + } + if len(refs) == 0 { + return errors.New("no refs in backup") + } + + // Reset all refs except for HEAD. + if err := repo.ResetRefs(ctx, refs); err != nil { + return fmt.Errorf("reset refs: %w", err) + } + + // Explicitly reset HEAD to the default branch tracked by the manifest if available. In a + // bundle restore, this would've been done during repository creation. + headRef := git.ReferenceName(backup.HeadReference) + if headRef == "" { + return errors.New("expected HEAD to be a symbolic reference") + } + + return repo.SetHeadReference(ctx, headRef) +} + func (mgr *Manager) restoreFromBundle(ctx context.Context, repo Repository, backup *Backup, alwaysCreate bool) error { hash, err := git.ObjectHashByFormat(backup.ObjectFormat) if err != nil { diff --git a/internal/backup/backup_test.go b/internal/backup/backup_test.go index fcdd0a98da..01321e63be 100644 --- a/internal/backup/backup_test.go +++ b/internal/backup/backup_test.go @@ -21,6 +21,7 @@ import ( "gitlab.com/gitlab-org/gitaly/v16/internal/gitaly/transaction" "gitlab.com/gitlab-org/gitaly/v16/internal/grpc/client" "gitlab.com/gitlab-org/gitaly/v16/internal/helper/perm" + "gitlab.com/gitlab-org/gitaly/v16/internal/log" "gitlab.com/gitlab-org/gitaly/v16/internal/testhelper" "gitlab.com/gitlab-org/gitaly/v16/internal/testhelper/testcfg" "gitlab.com/gitlab-org/gitaly/v16/internal/testhelper/testserver" @@ -569,22 +570,22 @@ func TestManager_Restore_latest(t *testing.T) { for _, managerTC := range []struct { desc string - setup func(t testing.TB, sink backup.Sink, locator backup.Locator) *backup.Manager + setup func(t testing.TB, sink backup.Sink, locator backup.Locator, logger log.LogrusLogger) *backup.Manager }{ { desc: "RPC manager", - setup: func(tb testing.TB, sink backup.Sink, locator backup.Locator) *backup.Manager { + setup: func(tb testing.TB, sink backup.Sink, locator backup.Locator, logger log.LogrusLogger) *backup.Manager { pool := client.NewPool() tb.Cleanup(func() { testhelper.MustClose(tb, pool) }) - return backup.NewManager(sink, testhelper.SharedLogger(t), locator, pool) + return backup.NewManager(sink, logger, locator, pool) }, }, { desc: "Local manager", - setup: func(tb testing.TB, sink backup.Sink, locator backup.Locator) *backup.Manager { + setup: func(tb testing.TB, sink backup.Sink, locator backup.Locator, logger log.LogrusLogger) *backup.Manager { if testhelper.IsPraefectEnabled() { tb.Skip("local backup manager expects to operate on the local filesystem so cannot operate through praefect") } @@ -595,7 +596,7 @@ func TestManager_Restore_latest(t *testing.T) { tb.Cleanup(catfileCache.Stop) txManager := transaction.NewTrackingManager() - return backup.NewManagerLocal(sink, testhelper.SharedLogger(t), locator, storageLocator, gitCmdFactory, catfileCache, txManager, repoCounter) + return backup.NewManagerLocal(sink, logger, locator, storageLocator, gitCmdFactory, catfileCache, txManager, repoCounter) }, }, } { @@ -612,62 +613,93 @@ func TestManager_Restore_latest(t *testing.T) { repoClient := gitalypb.NewRepositoryServiceClient(cc) - _, repoPath := gittest.CreateRepository(t, ctx, cfg) - commitID := gittest.WriteCommit(t, cfg, repoPath, gittest.WithBranch("main")) - gittest.WriteTag(t, cfg, repoPath, "v1.0.0", commitID.Revision()) - repoChecksum := gittest.ChecksumRepo(t, cfg, repoPath) - repoBundle := gittest.BundleRepo(t, cfg, repoPath, "-") - repoRefs := gittest.Exec(t, cfg, "-C", repoPath, "show-ref", "--head") - backupRoot := testhelper.TempDir(t) for _, tc := range []struct { - desc string - setup func(tb testing.TB) (*gitalypb.Repository, *git.Checksum) - alwaysCreate bool - expectExists bool - expectedPaths []string - expectedErrAs error + desc string + setup func(tb testing.TB) (*gitalypb.Repository, *git.Checksum) + alwaysCreate bool + expectExists bool + expectedPaths []string + expectedErrAs error + shouldUseResetRefsOptimisation bool }{ { - desc: "existing repo, without hooks", + desc: "non-optimised", setup: func(tb testing.TB) (*gitalypb.Repository, *git.Checksum) { + _, repoPath, _ := createAndSeedRepository(t, ctx, cfg) + repoChecksum, repoBundle, repoRefs := createBackupArtifacts(t, cfg, repoPath) + + // Restoring into an empty repo, so ref reset won't work. repo, _ := gittest.CreateRepository(t, ctx, cfg) relativePath := stripRelativePath(tb, repo) testhelper.WriteFiles(tb, backupRoot, map[string]any{ filepath.Join("manifests", repo.GetStorageName(), repo.GetRelativePath(), "+latest.toml"): fmt.Sprintf(` object_format = '%[1]s' +head_reference = '%[3]s' [[steps]] bundle_path = '%[2]s.bundle' ref_path = '%[2]s.refs' custom_hooks_path = '%[2]s/custom_hooks.tar' - `, gittest.DefaultObjectHash.Format, relativePath), + `, gittest.DefaultObjectHash.Format, relativePath, git.DefaultRef.String()), relativePath + ".bundle": repoBundle, relativePath + ".refs": repoRefs, }) + require.NoError(tb, os.MkdirAll(filepath.Join(backupRoot, relativePath), perm.PublicDir)) + return repo, repoChecksum }, expectExists: true, }, + { + desc: "existing repo, without hooks", + setup: func(tb testing.TB) (*gitalypb.Repository, *git.Checksum) { + repo, repoPath, head := createAndSeedRepository(t, ctx, cfg) + repoChecksum, repoBundle, repoRefs := createBackupArtifacts(t, cfg, repoPath) + gittest.WriteCommit(t, cfg, repoPath, gittest.WithParents(head), gittest.WithBranch("main")) + + relativePath := stripRelativePath(tb, repo) + testhelper.WriteFiles(tb, backupRoot, map[string]any{ + filepath.Join("manifests", repo.GetStorageName(), repo.GetRelativePath(), "+latest.toml"): fmt.Sprintf(` +object_format = '%[1]s' +head_reference = '%[3]s' + +[[steps]] +bundle_path = '%[2]s.bundle' +ref_path = '%[2]s.refs' +custom_hooks_path = '%[2]s/custom_hooks.tar' + `, gittest.DefaultObjectHash.Format, relativePath, git.DefaultRef.String()), + relativePath + ".bundle": repoBundle, + relativePath + ".refs": repoRefs, + }) + + return repo, repoChecksum + }, + expectExists: true, + shouldUseResetRefsOptimisation: true, + }, { desc: "existing repo, with hooks", setup: func(tb testing.TB) (*gitalypb.Repository, *git.Checksum) { - repo, _ := gittest.CreateRepository(t, ctx, cfg) + repo, repoPath, head := createAndSeedRepository(t, ctx, cfg) + repoChecksum, repoBundle, repoRefs := createBackupArtifacts(t, cfg, repoPath) + gittest.WriteCommit(t, cfg, repoPath, gittest.WithParents(head), gittest.WithBranch("main")) relativePath := stripRelativePath(tb, repo) customHooksPath := filepath.Join(backupRoot, relativePath, "custom_hooks.tar") testhelper.WriteFiles(tb, backupRoot, map[string]any{ filepath.Join("manifests", repo.GetStorageName(), repo.GetRelativePath(), "+latest.toml"): fmt.Sprintf(` object_format = '%[1]s' +head_reference = '%[3]s' [[steps]] bundle_path = '%[2]s.bundle' ref_path = '%[2]s.refs' custom_hooks_path = '%[2]s/custom_hooks.tar' - `, gittest.DefaultObjectHash.Format, relativePath), + `, gittest.DefaultObjectHash.Format, relativePath, git.DefaultRef.String()), relativePath + ".bundle": repoBundle, relativePath + ".refs": repoRefs, }) @@ -682,7 +714,8 @@ custom_hooks_path = '%[2]s/custom_hooks.tar' "custom_hooks/prepare-commit-msg.sample", "custom_hooks/pre-push.sample", }, - expectExists: true, + expectExists: true, + shouldUseResetRefsOptimisation: true, }, { desc: "missing backup", @@ -710,11 +743,12 @@ custom_hooks_path = '%[2]s/custom_hooks.tar' testhelper.WriteFiles(tb, backupRoot, map[string]any{ filepath.Join("manifests", repo.GetStorageName(), repo.GetRelativePath(), "+latest.toml"): fmt.Sprintf(` object_format = '%[1]s' +head_reference = '%[3]s' [[steps]] ref_path = '%[2]s.refs' custom_hooks_path = '%[2]s/custom_hooks.tar' - `, gittest.DefaultObjectHash.Format, relativePath), + `, gittest.DefaultObjectHash.Format, relativePath, git.DefaultRef.String()), relativePath + ".refs": "", }) @@ -731,11 +765,12 @@ custom_hooks_path = '%[2]s/custom_hooks.tar' testhelper.WriteFiles(tb, backupRoot, map[string]any{ filepath.Join("manifests", repo.GetStorageName(), repo.GetRelativePath(), "+latest.toml"): fmt.Sprintf(` object_format = '%[1]s' +head_reference = '%[3]s' [[steps]] ref_path = '%[2]s.refs' custom_hooks_path = '%[2]s/custom_hooks.tar' - `, gittest.DefaultObjectHash.Format, relativePath), + `, gittest.DefaultObjectHash.Format, relativePath, git.DefaultRef.String()), relativePath + ".refs": "", }) @@ -747,26 +782,30 @@ custom_hooks_path = '%[2]s/custom_hooks.tar' { desc: "nonexistent repo", setup: func(tb testing.TB) (*gitalypb.Repository, *git.Checksum) { - repo := &gitalypb.Repository{ + _, repoPath, _ := createAndSeedRepository(t, ctx, cfg) + repoChecksum, repoBundle, repoRefs := createBackupArtifacts(t, cfg, repoPath) + + nonexistentRepo := &gitalypb.Repository{ StorageName: "default", RelativePath: gittest.NewRepositoryName(tb), } - relativePath := stripRelativePath(tb, repo) + relativePath := stripRelativePath(tb, nonexistentRepo) testhelper.WriteFiles(tb, backupRoot, map[string]any{ - filepath.Join("manifests", repo.GetStorageName(), repo.GetRelativePath(), "+latest.toml"): fmt.Sprintf(` + filepath.Join("manifests", nonexistentRepo.GetStorageName(), nonexistentRepo.GetRelativePath(), "+latest.toml"): fmt.Sprintf(` object_format = '%[1]s' +head_reference = '%[3]s' [[steps]] bundle_path = '%[2]s.bundle' ref_path = '%[2]s.refs' custom_hooks_path = '%[2]s/custom_hooks.tar' - `, gittest.DefaultObjectHash.Format, relativePath), + `, gittest.DefaultObjectHash.Format, relativePath, git.DefaultRef.String()), relativePath + ".bundle": repoBundle, relativePath + ".refs": repoRefs, }) - return repo, repoChecksum + return nonexistentRepo, repoChecksum }, expectExists: true, }, @@ -775,46 +814,58 @@ custom_hooks_path = '%[2]s/custom_hooks.tar' setup: func(tb testing.TB) (*gitalypb.Repository, *git.Checksum) { const backupID = "abc123" - _, expectedRepoPath := gittest.CreateRepository(t, ctx, cfg) - - repo, _ := gittest.CreateRepository(t, ctx, cfg) + repo, repoPath, head := createAndSeedRepository(t, ctx, cfg) + relativePath := stripRelativePath(tb, repo) - root := gittest.WriteCommit(tb, cfg, expectedRepoPath, - gittest.WithBranch("master"), - ) - master1 := gittest.WriteCommit(tb, cfg, expectedRepoPath, - gittest.WithBranch("master"), - gittest.WithParents(root), + master1 := gittest.WriteCommit(tb, cfg, repoPath, + gittest.WithBranch("main"), + gittest.WithParents(head), ) - other := gittest.WriteCommit(tb, cfg, expectedRepoPath, + other := gittest.WriteCommit(tb, cfg, repoPath, gittest.WithBranch("other"), - gittest.WithParents(root), + gittest.WithParents(head), ) - gittest.Exec(tb, cfg, "-C", expectedRepoPath, "symbolic-ref", "HEAD", "refs/heads/master") - bundle1 := gittest.Exec(tb, cfg, "-C", expectedRepoPath, "bundle", "create", "-", + gittest.Exec(tb, cfg, "-C", repoPath, "symbolic-ref", "HEAD", "refs/heads/main") + bundle1 := gittest.Exec(tb, cfg, "-C", repoPath, "bundle", "create", "-", "HEAD", - "refs/heads/master", + "refs/heads/main", "refs/heads/other", ) - refs1 := gittest.Exec(t, cfg, "-C", expectedRepoPath, "show-ref", "--head") + refs1 := gittest.Exec(t, cfg, "-C", repoPath, "show-ref", "--head") + customHooksPath1 := mustCreateCustomHooksArchive(t, ctx) - master2 := gittest.WriteCommit(tb, cfg, expectedRepoPath, - gittest.WithBranch("master"), + master2 := gittest.WriteCommit(tb, cfg, repoPath, + gittest.WithBranch("main"), gittest.WithParents(master1), ) - bundle2 := gittest.Exec(tb, cfg, "-C", expectedRepoPath, "bundle", "create", "-", + bundle2 := gittest.Exec(tb, cfg, "-C", repoPath, "bundle", "create", "-", "HEAD", "^"+master1.String(), "^"+other.String(), - "refs/heads/master", + "refs/heads/main", "refs/heads/other", ) - refs2 := gittest.Exec(t, cfg, "-C", expectedRepoPath, "show-ref", "--head") + refs2 := gittest.Exec(t, cfg, "-C", repoPath, "show-ref", "--head") + customHooksPath2 := mustCreateCustomHooksArchive(t, ctx, + "another-hook-1.sample", + "another-hook-2.sample") + + checksum := gittest.ChecksumRepo(t, cfg, repoPath) + + // Create some more commits that will be reverted by the restore. + gittest.WriteCommit(tb, cfg, repoPath, + gittest.WithBranch("main"), + gittest.WithParents(master2), + ) + gittest.WriteCommit(tb, cfg, repoPath, + gittest.WithBranch("other-2"), + gittest.WithParents(master2), + ) - relativePath := stripRelativePath(tb, repo) testhelper.WriteFiles(tb, backupRoot, map[string]any{ filepath.Join("manifests", repo.GetStorageName(), repo.GetRelativePath(), "+latest.toml"): fmt.Sprintf(` object_format = '%[1]s' +head_reference = '%[4]s' [[steps]] bundle_path = '%[2]s/%[3]s/001.bundle' @@ -825,21 +876,27 @@ custom_hooks_path = '%[2]s/%[3]s/001.custom_hooks.tar' bundle_path = '%[2]s/%[3]s/002.bundle' ref_path = '%[2]s/%[3]s/002.refs' custom_hooks_path = '%[2]s/%[3]s/002.custom_hooks.tar' - `, gittest.DefaultObjectHash.Format, relativePath, backupID), + `, gittest.DefaultObjectHash.Format, relativePath, backupID, git.DefaultRef.String()), filepath.Join(relativePath, backupID, "001.bundle"): bundle1, filepath.Join(relativePath, backupID, "002.bundle"): bundle2, filepath.Join(relativePath, backupID, "001.refs"): refs1, filepath.Join(relativePath, backupID, "002.refs"): refs2, }) - checksum := new(git.Checksum) - checksum.Add(git.NewReference("HEAD", master2)) - checksum.Add(git.NewReference("refs/heads/master", master2)) - checksum.Add(git.NewReference("refs/heads/other", other)) + testhelper.CopyFile(tb, customHooksPath1, filepath.Join(backupRoot, relativePath, backupID, "001.custom_hooks.tar")) + testhelper.CopyFile(tb, customHooksPath2, filepath.Join(backupRoot, relativePath, backupID, "002.custom_hooks.tar")) return repo, checksum }, expectExists: true, + expectedPaths: []string{ + "custom_hooks/pre-commit.sample", + "custom_hooks/prepare-commit-msg.sample", + "custom_hooks/pre-push.sample", + "custom_hooks/another-hook-1.sample", + "custom_hooks/another-hook-2.sample", + }, + shouldUseResetRefsOptimisation: true, }, } { t.Run(tc.desc, func(t *testing.T) { @@ -851,7 +908,10 @@ custom_hooks_path = '%[2]s/%[3]s/002.custom_hooks.tar' locator, err := backup.ResolveLocator("pointer", sink) require.NoError(t, err) - fsBackup := managerTC.setup(t, sink, locator) + logger := testhelper.NewLogger(t) + hook := testhelper.AddLoggerHook(logger) + + fsBackup := managerTC.setup(t, sink, locator, logger) err = fsBackup.Restore(ctx, &backup.RestoreRequest{ Server: storage.ServerInfo{Address: cfg.SocketPath, Token: cfg.Auth.Token}, Repository: repo, @@ -889,6 +949,13 @@ custom_hooks_path = '%[2]s/%[3]s/002.custom_hooks.tar' require.FileExists(t, filepath.Join(repoPath, p)) } } + + if tc.shouldUseResetRefsOptimisation { + require.Nil(t, hook.LastEntry()) + } else if tc.expectedErrAs == nil { + // If we're not expecting an error, then we should've proceeded with a bundle restore. + require.Equal(t, hook.LastEntry().Message, "unable to reset refs. Proceeding with a normal restore") + } }) } }) @@ -909,22 +976,22 @@ func TestManager_Restore_specific(t *testing.T) { for _, managerTC := range []struct { desc string - setup func(t testing.TB, sink backup.Sink, locator backup.Locator) *backup.Manager + setup func(t testing.TB, sink backup.Sink, locator backup.Locator, logger log.LogrusLogger) *backup.Manager }{ { desc: "RPC manager", - setup: func(tb testing.TB, sink backup.Sink, locator backup.Locator) *backup.Manager { + setup: func(tb testing.TB, sink backup.Sink, locator backup.Locator, logger log.LogrusLogger) *backup.Manager { pool := client.NewPool() tb.Cleanup(func() { testhelper.MustClose(tb, pool) }) - return backup.NewManager(sink, testhelper.SharedLogger(t), locator, pool) + return backup.NewManager(sink, logger, locator, pool) }, }, { desc: "Local manager", - setup: func(tb testing.TB, sink backup.Sink, locator backup.Locator) *backup.Manager { + setup: func(tb testing.TB, sink backup.Sink, locator backup.Locator, logger log.LogrusLogger) *backup.Manager { if testhelper.IsPraefectEnabled() { tb.Skip("local backup manager expects to operate on the local filesystem so cannot operate through praefect") } @@ -935,7 +1002,7 @@ func TestManager_Restore_specific(t *testing.T) { tb.Cleanup(catfileCache.Stop) txManager := transaction.NewTrackingManager() - return backup.NewManagerLocal(sink, testhelper.SharedLogger(t), locator, storageLocator, gitCmdFactory, catfileCache, txManager, repoCounter) + return backup.NewManagerLocal(sink, logger, locator, storageLocator, gitCmdFactory, catfileCache, txManager, repoCounter) }, }, } { @@ -952,23 +1019,17 @@ func TestManager_Restore_specific(t *testing.T) { repoClient := gitalypb.NewRepositoryServiceClient(cc) - _, repoPath := gittest.CreateRepository(t, ctx, cfg) - commitID := gittest.WriteCommit(t, cfg, repoPath, gittest.WithBranch("main")) - gittest.WriteTag(t, cfg, repoPath, "v1.0.0", commitID.Revision()) - repoChecksum := gittest.ChecksumRepo(t, cfg, repoPath) - repoBundle := gittest.BundleRepo(t, cfg, repoPath, "-") - repoRefs := gittest.Exec(t, cfg, "-C", repoPath, "show-ref", "--head") - backupRoot := testhelper.TempDir(t) for _, tc := range []struct { - desc string - setup func(tb testing.TB) (*gitalypb.Repository, *git.Checksum) - alwaysCreate bool - expectExists bool - expectedPaths []string - expectedErrAs error - expectedHeadReference git.ReferenceName + desc string + setup func(tb testing.TB) (*gitalypb.Repository, *git.Checksum) + alwaysCreate bool + expectExists bool + expectedPaths []string + expectedErrAs error + expectedHeadReference git.ReferenceName + shouldUseResetRefsOptimisation bool }{ { desc: "missing backup", @@ -983,69 +1044,84 @@ func TestManager_Restore_specific(t *testing.T) { { desc: "single incremental", setup: func(tb testing.TB) (*gitalypb.Repository, *git.Checksum) { - repo, _ := gittest.CreateRepository(tb, ctx, cfg) + repo, repoPath, _ := createAndSeedRepository(t, ctx, cfg) + repoChecksum, repoBundle, repoRefs := createBackupArtifacts(t, cfg, repoPath) relativePath := stripRelativePath(tb, repo) testhelper.WriteFiles(tb, backupRoot, map[string]any{ filepath.Join("manifests", repo.GetStorageName(), repo.GetRelativePath(), backupID+".toml"): fmt.Sprintf(` object_format = '%[1]s' +head_reference = '%[4]s' [[steps]] bundle_path = '%[2]s/%[3]s/001.bundle' ref_path = '%[2]s/%[3]s/001.refs' custom_hooks_path = '%[2]s/%[3]s/001.custom_hooks.tar' - `, gittest.DefaultObjectHash.Format, relativePath, backupID), + `, gittest.DefaultObjectHash.Format, relativePath, backupID, git.DefaultRef.String()), filepath.Join(relativePath, backupID, "001.bundle"): repoBundle, filepath.Join(relativePath, backupID, "001.refs"): repoRefs, }) return repo, repoChecksum }, - expectExists: true, + expectExists: true, + shouldUseResetRefsOptimisation: true, }, { desc: "many incrementals", setup: func(tb testing.TB) (*gitalypb.Repository, *git.Checksum) { - _, expectedRepoPath := gittest.CreateRepository(tb, ctx, cfg) + repo, repoPath, head := createAndSeedRepository(t, ctx, cfg) - repo, _ := gittest.CreateRepository(tb, ctx, cfg) - - root := gittest.WriteCommit(tb, cfg, expectedRepoPath, - gittest.WithBranch("master"), - ) - master1 := gittest.WriteCommit(tb, cfg, expectedRepoPath, - gittest.WithBranch("master"), - gittest.WithParents(root), + master1 := gittest.WriteCommit(tb, cfg, repoPath, + gittest.WithBranch("main"), + gittest.WithParents(head), ) - other := gittest.WriteCommit(tb, cfg, expectedRepoPath, + other := gittest.WriteCommit(tb, cfg, repoPath, gittest.WithBranch("other"), - gittest.WithParents(root), + gittest.WithParents(head), ) - gittest.Exec(tb, cfg, "-C", expectedRepoPath, "symbolic-ref", "HEAD", "refs/heads/master") - bundle1 := gittest.Exec(tb, cfg, "-C", expectedRepoPath, "bundle", "create", "-", + gittest.Exec(tb, cfg, "-C", repoPath, "symbolic-ref", "HEAD", "refs/heads/main") + bundle1 := gittest.Exec(tb, cfg, "-C", repoPath, "bundle", "create", "-", "HEAD", - "refs/heads/master", + "refs/heads/main", "refs/heads/other", ) - refs1 := gittest.Exec(tb, cfg, "-C", expectedRepoPath, "show-ref", "--head") + refs1 := gittest.Exec(tb, cfg, "-C", repoPath, "show-ref", "--head") + customHooksPath1 := mustCreateCustomHooksArchive(t, ctx) - master2 := gittest.WriteCommit(tb, cfg, expectedRepoPath, - gittest.WithBranch("master"), + master2 := gittest.WriteCommit(tb, cfg, repoPath, + gittest.WithBranch("main"), gittest.WithParents(master1), ) - bundle2 := gittest.Exec(tb, cfg, "-C", expectedRepoPath, "bundle", "create", "-", + bundle2 := gittest.Exec(tb, cfg, "-C", repoPath, "bundle", "create", "-", "HEAD", "^"+master1.String(), "^"+other.String(), - "refs/heads/master", + "refs/heads/main", "refs/heads/other", ) - refs2 := gittest.Exec(tb, cfg, "-C", expectedRepoPath, "show-ref", "--head") + refs2 := gittest.Exec(tb, cfg, "-C", repoPath, "show-ref", "--head") + customHooksPath2 := mustCreateCustomHooksArchive(t, ctx, + "another-hook-1.sample", + "another-hook-2.sample") + + checksum := gittest.ChecksumRepo(t, cfg, repoPath) + + // Create some more commits that will be reverted by the restore. + gittest.WriteCommit(tb, cfg, repoPath, + gittest.WithBranch("main"), + gittest.WithParents(master2), + ) + gittest.WriteCommit(tb, cfg, repoPath, + gittest.WithBranch("other-2"), + gittest.WithParents(master2), + ) relativePath := stripRelativePath(tb, repo) testhelper.WriteFiles(tb, backupRoot, map[string]any{ filepath.Join("manifests", repo.GetStorageName(), repo.GetRelativePath(), backupID+".toml"): fmt.Sprintf(` object_format = '%[1]s' +head_reference = '%[4]s' [[steps]] bundle_path = '%[2]s/%[3]s/001.bundle' @@ -1056,21 +1132,27 @@ custom_hooks_path = '%[2]s/%[3]s/001.custom_hooks.tar' bundle_path = '%[2]s/%[3]s/002.bundle' ref_path = '%[2]s/%[3]s/002.refs' custom_hooks_path = '%[2]s/%[3]s/002.custom_hooks.tar' - `, gittest.DefaultObjectHash.Format, relativePath, backupID), + `, gittest.DefaultObjectHash.Format, relativePath, backupID, git.DefaultRef.String()), filepath.Join(relativePath, backupID, "001.bundle"): bundle1, filepath.Join(relativePath, backupID, "002.bundle"): bundle2, filepath.Join(relativePath, backupID, "001.refs"): refs1, filepath.Join(relativePath, backupID, "002.refs"): refs2, }) - checksum := new(git.Checksum) - checksum.Add(git.NewReference("HEAD", master2)) - checksum.Add(git.NewReference("refs/heads/master", master2)) - checksum.Add(git.NewReference("refs/heads/other", other)) + testhelper.CopyFile(tb, customHooksPath1, filepath.Join(backupRoot, relativePath, backupID, "001.custom_hooks.tar")) + testhelper.CopyFile(tb, customHooksPath2, filepath.Join(backupRoot, relativePath, backupID, "002.custom_hooks.tar")) return repo, checksum }, expectExists: true, + expectedPaths: []string{ + "custom_hooks/pre-commit.sample", + "custom_hooks/prepare-commit-msg.sample", + "custom_hooks/pre-push.sample", + "custom_hooks/another-hook-1.sample", + "custom_hooks/another-hook-2.sample", + }, + shouldUseResetRefsOptimisation: true, }, { desc: "empty backup", @@ -1098,7 +1180,8 @@ custom_hooks_path = 'custom_hooks.tar' { desc: "head reference", setup: func(tb testing.TB) (*gitalypb.Repository, *git.Checksum) { - repo, _ := gittest.CreateRepository(tb, ctx, cfg) + repo, repoPath, head := createAndSeedRepository(t, ctx, cfg) + _, repoBundle, repoRefs := createBackupArtifacts(t, cfg, repoPath) testhelper.WriteFiles(tb, backupRoot, map[string]any{ filepath.Join("manifests", repo.GetStorageName(), repo.GetRelativePath(), backupID+".toml"): fmt.Sprintf( @@ -1118,12 +1201,13 @@ custom_hooks_path = 'custom_hooks.tar' // Negate off the default branch since the manifest is // explicitly setting a different unborn branch that // will not be part of the checksum. - checksum.Add(git.NewReference("HEAD", commitID)) + checksum.Add(git.NewReference("HEAD", head)) return repo, checksum }, - expectExists: true, - expectedHeadReference: "refs/heads/banana", + expectExists: true, + expectedHeadReference: "refs/heads/banana", + shouldUseResetRefsOptimisation: true, }, } { t.Run(tc.desc, func(t *testing.T) { @@ -1135,7 +1219,10 @@ custom_hooks_path = 'custom_hooks.tar' locator, err := backup.ResolveLocator("pointer", sink) require.NoError(t, err) - fsBackup := managerTC.setup(t, sink, locator) + logger := testhelper.NewLogger(t) + hook := testhelper.AddLoggerHook(logger) + + fsBackup := managerTC.setup(t, sink, locator, logger) err = fsBackup.Restore(ctx, &backup.RestoreRequest{ Server: storage.ServerInfo{Address: cfg.SocketPath, Token: cfg.Auth.Token}, Repository: repo, @@ -1181,6 +1268,13 @@ custom_hooks_path = 'custom_hooks.tar' ref := gittest.GetSymbolicRef(t, cfg, repoPath, "HEAD") require.Equal(t, tc.expectedHeadReference, git.ReferenceName(ref.Target)) } + + if tc.shouldUseResetRefsOptimisation { + require.Nil(t, hook.LastEntry()) + } else if tc.expectedErrAs == nil { + // If we're not expecting an error, then we should've proceeded with a bundle restore. + require.Equal(t, hook.LastEntry().Message, "unable to reset refs. Proceeding with a normal restore") + } }) } }) @@ -1264,7 +1358,7 @@ func stripRelativePath(tb testing.TB, repo storage.Repository) string { return strings.TrimSuffix(repo.GetRelativePath(), ".git") } -func mustCreateCustomHooksArchive(t *testing.T, ctx context.Context) string { +func mustCreateCustomHooksArchive(t *testing.T, ctx context.Context, additionalHooks ...string) string { t.Helper() tmpDir := testhelper.TempDir(t) @@ -1276,6 +1370,10 @@ func mustCreateCustomHooksArchive(t *testing.T, ctx context.Context) string { require.NoError(t, os.WriteFile(filepath.Join(hooksDirPath, "prepare-commit-msg.sample"), []byte("bar"), os.ModePerm)) require.NoError(t, os.WriteFile(filepath.Join(hooksDirPath, "pre-push.sample"), []byte("baz"), os.ModePerm)) + for _, hookName := range additionalHooks { + require.NoError(t, os.WriteFile(filepath.Join(hooksDirPath, hookName), []byte("additional hook content"), os.ModePerm)) + } + archivePath := filepath.Join(tmpDir, "custom_hooks.tar") file, err := os.Create(archivePath) require.NoError(t, err) @@ -1285,3 +1383,19 @@ func mustCreateCustomHooksArchive(t *testing.T, ctx context.Context) string { return archivePath } + +func createAndSeedRepository(t *testing.T, ctx context.Context, cfg config.Cfg) (repo *gitalypb.Repository, repoPath string, head git.ObjectID) { + repo, repoPath = gittest.CreateRepository(t, ctx, cfg) + commitID := gittest.WriteCommit(t, cfg, repoPath, gittest.WithBranch("main")) + gittest.WriteTag(t, cfg, repoPath, "v1.0.0", commitID.Revision()) + + return repo, repoPath, commitID +} + +func createBackupArtifacts(t *testing.T, cfg config.Cfg, repoPath string) (repoChecksum *git.Checksum, repoBundle []byte, repoRefs []byte) { + repoChecksum = gittest.ChecksumRepo(t, cfg, repoPath) + repoBundle = gittest.BundleRepo(t, cfg, repoPath, "-") + repoRefs = gittest.Exec(t, cfg, "-C", repoPath, "show-ref", "--head") + + return repoChecksum, repoBundle, repoRefs +} diff --git a/internal/backup/repository_test.go b/internal/backup/repository_test.go index bf1bf0fa35..47cc7f6a06 100644 --- a/internal/backup/repository_test.go +++ b/internal/backup/repository_test.go @@ -32,6 +32,7 @@ func removeHeadReference(refs []git.Reference) []git.Reference { func TestRemoteRepository_ResetRefs(t *testing.T) { cfg := testcfg.Build(t) + testcfg.BuildGitalyHooks(t, cfg) cfg.SocketPath = testserver.RunGitalyServer(t, cfg, setup.RegisterAll) ctx := testhelper.Context(t) diff --git a/internal/gitaly/service/repository/restore_repository_test.go b/internal/gitaly/service/repository/restore_repository_test.go index 8bd55285cb..0b438443b4 100644 --- a/internal/gitaly/service/repository/restore_repository_test.go +++ b/internal/gitaly/service/repository/restore_repository_test.go @@ -142,7 +142,7 @@ func TestRestoreRepository(t *testing.T) { backupID: "", } }, - expectedErr: structerr.NewFailedPrecondition("restore repository: manager: repository skipped: read refs: doesn't exist").WithDetail( + expectedErr: structerr.NewFailedPrecondition("restore repository: manager: restore from bundle: repository skipped: read refs: doesn't exist").WithDetail( &gitalypb.RestoreRepositoryResponse_SkippedError{}, ), }, -- GitLab