Welcome to mirror list, hosted at ThFree Co, Russian Federation.

gitlab.com/gitlab-org/gitaly.git - Unnamed repository; edit this file 'description' to name the repository.
summaryrefslogtreecommitdiff
diff options
context:
space:
mode:
authorQuang-Minh Nguyen <qmnguyen@gitlab.com>2024-01-15 19:18:40 +0300
committerQuang-Minh Nguyen <qmnguyen@gitlab.com>2024-01-24 08:39:28 +0300
commitfc31823056ae6a6f6a229658da806b5e94ec0246 (patch)
tree66077a050d524ab9695ff3b726f31fb9fbe5f5cc
parent2ad90f6613140c7ca097272b8f22e34113787a68 (diff)
Add repacking support to transaction managerqmnguyen0711/add-repacking-support-to-transaction-manager
Gitaly has a sophisticated housekeeping system. That system packs loose objects, re-organizes the on-disk layout of packfiles, prunes unreachable objects, etc. It aims to make the repository optimal, performant, and cost-effective. It's a crucial component for Gitaly. The current housekeeping approach is working on the repository concurrently with the TransactionManager. This is not okay as the TransactionManager is expected to be the single writer in the repository. We'll thus need a different method for repacking objects of a repository to synchronize it with all other access. The WAL manager has a very different way of handling concurrent requests. As a result, the repacking task should adapt the new architecture accordingly. The manager handles a repacking task in three stages: preparation, verification, and applying. When a transaction is committed, the goroutine of the transaction runs repacking preparation. This stage triggers `git-repack(1)` command with different parameters depending on the desired strategy. Afterward, it attaches the list of new files and a list of deleted files to the transaction. This stage can span multiple minutes/hours. While it runs, the manager can accept other update transactions. When the preparation stage finishes, the repacking transaction is submitted to the manager and the verification is performed. This verification is head-of-line blocking for each repository. The manager verifies if the repacking task causes any conflict with other transactions accepted beforehand. There are two types of conflicts: - Another transaction points to new references to pruned objects. - Another transaction includes a change that depends on pruned objects. Both cases require examining the list of committed transactions since the time the repacking task started. The manager collects the reference tips and verifies if they are still accessible from the repository or in any new packfiles produced by other transactions. The dependency check is not supported now. In the future, when Git supports extracting relevant objects of a pruned object, we can resolve conflicts smarter. At present, the manager rejects the repacking task if it finds any conflict. If the task is good to go, the manager appends the WAL log entry. Finally, the corresponding log entry is applied. The manager removes redundant packfiles and links new ones. If there are any concurrent transactions that introduce file changes, their resulting packfiles are located next to the repacked one(s). At this stage, we don't want to modify the housekeeping scheduler. The scheduler decides when and how a housekeeping task should run on a repository. It has different repacking strategies depending on the repository situation. The manager handles those strategies accordingly. There are 4 of them now: - IncrementalWithUnreachable: this strategy packs unreachable objects into a single packfile. In the WAL transaction, all changes are packed by default. So, this strategy is a no-op. - Geometric: this strategy rearranges the list of packfiles according to a geometric progression without taking reachability into account. It doesn't prune objects either. - FullWithUnreachable: this strategy merges all packfiles into a single packfile, simultaneously removing any loose objects. Unreachable objects are then appended to the end of this unified packfile. - FullWithCruft: In traditional housekeeping, the manager gets rid of unreachable objects via full repacking with cruft. It pushes all unreachable objects to a cruft packfile and keeps track of each object mtimes. All unreachable objects exceeding a grace period are cleaned up. The grace period is to ensure the housekeeping doesn't delete a to-be-reachable object accidentally. In WAL, it's feasible to examine the list of applied transactions. As a result, we don't need to take object expiry or cruft pack into account. This operation triggers a normal full repack without cruft packing. We keep the same strategy name for backward compatibility. Those strategies have increasing costs as well as corresponding effects. The lower-cost ones will be triggered more frequently. Only the last strategy involves object pruning. Others are safe for concurrency.
-rw-r--r--internal/gitaly/storage/storagemgr/testhelper_test.go31
-rw-r--r--internal/gitaly/storage/storagemgr/transaction_manager.go604
-rw-r--r--internal/gitaly/storage/storagemgr/transaction_manager_housekeeping_test.go2862
-rw-r--r--internal/gitaly/storage/storagemgr/transaction_manager_test.go12
4 files changed, 3443 insertions, 66 deletions
diff --git a/internal/gitaly/storage/storagemgr/testhelper_test.go b/internal/gitaly/storage/storagemgr/testhelper_test.go
index dcd8bb58c..1d9798e6a 100644
--- a/internal/gitaly/storage/storagemgr/testhelper_test.go
+++ b/internal/gitaly/storage/storagemgr/testhelper_test.go
@@ -77,6 +77,8 @@ type PackfilesState struct {
PooledObjects []git.ObjectID
// HasMultiPackIndex asserts whether there is a multi-pack-index inside the objects/pack directory.
HasMultiPackIndex bool
+ // HasCommitGraphs assert whether the repository has valid commit graphs.
+ HasCommitGraphs bool
}
// PackfileState describes the asserted state of an individual packfile, including its contained objects, index, bitmap.
@@ -336,6 +338,10 @@ func collectPackfilesState(tb testing.TB, repoPath string, cfg config.Cfg, objec
require.NoError(tb, err)
}
+ info, err := stats.CommitGraphInfoForRepository(repoPath)
+ require.NoError(tb, err)
+ state.HasCommitGraphs = info.Exists
+
return state
}
@@ -454,10 +460,12 @@ type testTransactionTag struct {
}
type testTransactionCommits struct {
- First testTransactionCommit
- Second testTransactionCommit
- Third testTransactionCommit
- Diverging testTransactionCommit
+ First testTransactionCommit
+ Second testTransactionCommit
+ Third testTransactionCommit
+ Diverging testTransactionCommit
+ Orphan testTransactionCommit
+ Unreachable testTransactionCommit
}
type testTransactionSetup struct {
@@ -557,6 +565,14 @@ type RunPackRefs struct {
TransactionID int
}
+// RunRepack calls repack housekeeping task on a transaction.
+type RunRepack struct {
+ // TransactionID is the transaction for which the repack task runs.
+ TransactionID int
+ // Config is the desired repacking config for the task.
+ Config housekeeping.RepackObjectsConfig
+}
+
// Commit calls Commit on a transaction.
type Commit struct {
// TransactionID identifies the transaction to commit.
@@ -777,7 +793,7 @@ func runTransactionTest(t *testing.T, ctx context.Context, tc transactionTestCas
transaction, err := transactionManager.Begin(beginCtx, step.RelativePath, step.SnapshottedRelativePaths, step.ReadOnly)
require.ErrorIs(t, err, step.ExpectedError)
if err == nil {
- require.Equal(t, step.ExpectedSnapshotLSN, transaction.SnapshotLSN())
+ require.Equalf(t, step.ExpectedSnapshotLSN, transaction.SnapshotLSN(), "mismatched ExpectedSnapshotLSN")
}
if step.ReadOnly {
@@ -936,6 +952,11 @@ func runTransactionTest(t *testing.T, ctx context.Context, tc transactionTestCas
transaction := openTransactions[step.TransactionID]
transaction.PackRefs()
+ case RunRepack:
+ require.Contains(t, openTransactions, step.TransactionID, "test error: repack housekeeping task aborted on committed before beginning it")
+
+ transaction := openTransactions[step.TransactionID]
+ transaction.Repack(step.Config)
case RepositoryAssertion:
require.Contains(t, openTransactions, step.TransactionID, "test error: transaction's snapshot asserted before beginning it")
transaction := openTransactions[step.TransactionID]
diff --git a/internal/gitaly/storage/storagemgr/transaction_manager.go b/internal/gitaly/storage/storagemgr/transaction_manager.go
index 6cf98ffbe..1bb77f115 100644
--- a/internal/gitaly/storage/storagemgr/transaction_manager.go
+++ b/internal/gitaly/storage/storagemgr/transaction_manager.go
@@ -1,6 +1,7 @@
package storagemgr
import (
+ "bufio"
"bytes"
"container/list"
"context"
@@ -19,6 +20,7 @@ import (
"github.com/dgraph-io/badger/v4"
"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/housekeeping"
"gitlab.com/gitlab-org/gitaly/v16/internal/git/localrepo"
"gitlab.com/gitlab-org/gitaly/v16/internal/git/stats"
@@ -68,6 +70,9 @@ var (
errHousekeepingConflictOtherUpdates = errors.New("housekeeping in the same transaction with other updates")
// errHousekeepingConflictConcurrent is returned when there are another concurrent housekeeping task.
errHousekeepingConflictConcurrent = errors.New("conflict with another concurrent housekeeping task")
+ // errRepackConflictPrunedObject is returned when the repacking task pruned an object that is still used by other
+ // concurrent transactions.
+ errRepackConflictPrunedObject = errors.New("pruned object used by other updates")
// Below errors are used to error out in cases when updates have been staged in a read-only transaction.
errReadOnlyReferenceUpdates = errors.New("reference updates staged in a read-only transaction")
@@ -150,6 +155,7 @@ type repositoryCreation struct {
// such as the cleanup of unneeded files and optimizations for the repository's data structures.
type runHousekeeping struct {
packRefs *runPackRefs
+ repack *runRepack
}
// runPackRefs models refs packing housekeeping task. It packs heads and tags for efficient repository access.
@@ -159,6 +165,24 @@ type runPackRefs struct {
PrunedRefs map[git.ReferenceName]struct{}
}
+// runRepack models repack housekeeping task. We support multiple repacking strategies. At this stage, the outside
+// scheduler determines which strategy to use. The transaction manager is responsible for executing it. In the future,
+// we want to make housekeeping smarter by migrating housekeeping scheduling responsibility to this manager. That work
+// is tracked in https://gitlab.com/gitlab-org/gitaly/-/issues/5709.
+type runRepack struct {
+ // config tells which strategy and baggaged options.
+ config housekeeping.RepackObjectsConfig
+ isFullRepack bool
+ // newFiles contains the list of new packfiles to be applied into the destination repository.
+ newFiles []string
+ // deletedFiles contains the list of packfiles should be removed in the destination repository. The repacking
+ // command runs in the snapshot repository for a significant amount of time. During so, other requests
+ // containing new objects can land and be applied before housekeeping task finishes. So, we keep a snapshot of
+ // packfile structure before and after running the task. When the manager applies the task, it targets those
+ // known files only. Other packfiles can still co-exist along side with the resulting repacked packfile(s).
+ deletedFiles []string
+}
+
// ReferenceUpdates contains references to update. Reference name is used as the key and the value
// is the expected old tip and the desired new tip.
type ReferenceUpdates map[git.ReferenceName]ReferenceUpdate
@@ -624,6 +648,16 @@ func (txn *Transaction) PackRefs() {
}
}
+// Repack sets repacking housekeeping task as a part of the transaction.
+func (txn *Transaction) Repack(config housekeeping.RepackObjectsConfig) {
+ if txn.runHousekeeping == nil {
+ txn.runHousekeeping = &runHousekeeping{}
+ }
+ txn.runHousekeeping.repack = &runRepack{
+ config: config,
+ }
+}
+
// IncludeObject includes the given object and its dependencies in the transaction's logged pack file even
// if the object is unreachable from the references.
func (txn *Transaction) IncludeObject(oid git.ObjectID) {
@@ -1148,6 +1182,9 @@ func (mgr *TransactionManager) prepareHousekeeping(ctx context.Context, transact
if err := mgr.preparePackRefs(ctx, transaction); err != nil {
return err
}
+ if err := mgr.prepareRepacking(ctx, transaction); err != nil {
+ return err
+ }
return nil
}
@@ -1198,7 +1235,7 @@ func (mgr *TransactionManager) preparePackRefs(ctx context.Context, transaction
return structerr.New("exec pack-refs: %w", err).WithMetadata("stderr", stderr.String())
}
- if err := os.Chmod(filepath.Join(filepath.Join(repoPath, "packed-refs")), perm.SharedReadOnlyFile); err != nil {
+ if err := os.Chmod(filepath.Join(repoPath, "packed-refs"), perm.SharedReadOnlyFile); err != nil {
return fmt.Errorf("modifying permission of pack-refs file")
}
@@ -1228,6 +1265,205 @@ func (mgr *TransactionManager) preparePackRefs(ctx context.Context, transaction
return nil
}
+// prepareRepacking runs git-repack(1) command against the snapshot repository using desired repacking strategy. Each
+// strategy has a different cost and effect corresponding to scheduling frequency.
+// - IncrementalWithUnreachable: pack all loose objects into one packfile. This strategy is a no-op because all new
+// objects regardless of their reachablity status are packed by default by the manager.
+// - Geometric: merge all packs together with geometric repacking. This is expensive or cheap depending on which packs
+// get merged. No need for a connectivity check.
+// - FullWithUnreachable: merge all packs into one but keep unreachable objects. This is more expensive but we don't
+// take connectivity into account. This strategy is essential for object pool. As we cannot prune objects in a pool,
+// packing them into one single packfile boosts its performance.
+// - FullWithCruft: Merge all packs into one and prune unreachable objects. It is the most effective, but yet costly
+// strategy. We cannot run this type of task frequently on a large repository. This stategy is handled as a full
+// repacking without cruft because we don't need object expiry.
+// Before the command runs, we capture a snapshot of existing packfiles. After the command finishes, we re-capture the
+// list and extract the list of to-be-updated packfiles. This practice is to prevent repacking task from deleting
+// packfiles of other concurrent updates at the applying phase.
+func (mgr *TransactionManager) prepareRepacking(ctx context.Context, transaction *Transaction) error {
+ if transaction.runHousekeeping.repack == nil {
+ return nil
+ }
+
+ var err error
+ repack := transaction.runHousekeeping.repack
+
+ // Build a working repository pointing to snapshot repository. Housekeeping task can access the repository
+ // without the needs for quarantine.
+ workingRepository := mgr.repositoryFactory.Build(transaction.snapshot.relativePath(transaction.relativePath))
+ repoPath := mgr.getAbsolutePath(workingRepository.GetRelativePath())
+
+ if repack.isFullRepack, err = housekeeping.ValidateRepacking(repack.config); err != nil {
+ return fmt.Errorf("validating repacking: %w", err)
+ }
+
+ if repack.config.Strategy == housekeeping.RepackObjectsStrategyIncrementalWithUnreachable {
+ // Once the transaction manager has been applied and at least one complete repack has occurred, there
+ // should be no loose unreachable objects remaining in the repository. When the transaction manager
+ // processes a change, it consolidates all unreachable objects and objects about to become reachable
+ // into a new packfile, which is then placed in the repository. As a result, unreachable objects may
+ // still exist but are confined to packfiles. These will eventually be cleaned up during a full repack.
+ // In the interim, geometric repacking is utilized to optimize the structure of packfiles for faster
+ // access. Therefore, this operation is effectively a no-op. However, we maintain it for the sake of
+ // backward compatibility with the existing housekeeping scheduler.
+ return nil
+ }
+
+ // Capture the list of packfiles and their baggages before repacking.
+ beforeFiles, err := mgr.collectPackfiles(ctx, repoPath)
+ if err != nil {
+ return fmt.Errorf("collecting existing packfiles: %w", err)
+ }
+
+ switch repack.config.Strategy {
+ case housekeeping.RepackObjectsStrategyGeometric:
+ // Geometric repacking rearranges the list of packfiles according to a geometric progression. This process
+ // does not consider object reachability. Since all unreachable objects remain within small packfiles,
+ // they become included in the newly created packfiles. Geometric repacking does not prune any objects.
+ if err := housekeeping.PerformGeometricRepacking(ctx, workingRepository, repack.config); err != nil {
+ return fmt.Errorf("perform geometric repacking: %w", err)
+ }
+ case housekeeping.RepackObjectsStrategyFullWithUnreachable:
+ // This strategy merges all packfiles into a single packfile, simultaneously removing any loose objects
+ // if present. Unreachable objects are then appended to the end of this unified packfile. Although the
+ // `git-repack(1)` command does not offer an option to specifically pack loose unreachable objects, this
+ // is not an issue because the transaction manager already ensures that unreachable objects are
+ // contained within packfiles. Therefore, this strategy effectively consolidates all packfiles into a
+ // single one. Adopting this strategy is crucial for alternates, as it ensures that we can manage
+ // objects within an object pool without the capability to prune them.
+ if err := housekeeping.PerformFullRepackingWithUnreachable(ctx, workingRepository, repack.config); err != nil {
+ return err
+ }
+ case housekeeping.RepackObjectsStrategyFullWithCruft:
+ // Both of above strategies don't prune unreachable objects. They re-organize the objects between
+ // packfiles. In the traditional housekeeping, the manager gets rid of unreachable objects via full
+ // repacking with cruft. It pushes all unreachable objects to a cruft packfile and keeps track of each
+ // object mtimes. All unreachable objects exceeding a grace period are cleaned up. The grace period is
+ // to ensure the housekeeping doesn't delete a to-be-reachable object accidentally, for example when GC
+ // runs while a concurrent push is being processed.
+ // The transaction manager handles concurrent requests very differently from the original git way. Each
+ // request runs on a snapshot repository and the results are collected in the form of packfiles. Those
+ // packfiles contain resulting reachable and unreachable objects. As a result, we don't need to take
+ // object expiry nor curft pack into account. This operation triggers a normal full repack without
+ // cruft packing.
+ // Afterward, packed unreachable objects are removed. During migration to transaction system, there
+ // might be some loose unreachable objects. They will eventually be packed via either of the above tasks.
+ if err := housekeeping.PerformRepack(ctx, workingRepository, repack.config,
+ // Do a full repack.
+ git.Flag{Name: "-a"},
+ // Don't include objects part of alternate.
+ git.Flag{Name: "-l"},
+ // Delete loose objects made redundant by this repack. This option is essential during migration
+ // to the new transaction system. Afterward, it can be removed.
+ git.Flag{Name: "-d"},
+ ); err != nil {
+ return err
+ }
+
+ if err := housekeeping.WriteCommitGraph(ctx, workingRepository, housekeeping.WriteCommitGraphConfig{ReplaceChain: true}); err != nil {
+ return fmt.Errorf("re-writing commit graph: %w", err)
+ }
+ }
+
+ // Re-capture the list of packfiles and their baggages after repacking.
+ afterFiles, err := mgr.collectPackfiles(ctx, repoPath)
+ if err != nil {
+ return fmt.Errorf("collecting new packfiles: %w", err)
+ }
+
+ for file := range beforeFiles {
+ // We delete the files only if it's missing from the before set.
+ if _, exist := afterFiles[file]; !exist {
+ repack.deletedFiles = append(repack.deletedFiles, file)
+ }
+ }
+
+ for file := range afterFiles {
+ // Similarly, we don't need to link existing packfiles.
+ if _, exist := beforeFiles[file]; !exist {
+ repack.newFiles = append(repack.newFiles, file)
+ }
+ }
+
+ // Copy new files to lsn directory.
+ for _, file := range repack.newFiles {
+ if err := os.Link(
+ filepath.Join(filepath.Join(filepath.Join(repoPath, "objects", "pack"), file)),
+ filepath.Join(transaction.walFilesPath(), file),
+ ); err != nil {
+ return fmt.Errorf("copying packfiles to WAL directory: %w", err)
+ }
+ }
+
+ // Copy the whole commit graphs over.
+ if repack.config.Strategy == housekeeping.RepackObjectsStrategyFullWithCruft && (len(repack.newFiles) > 0 || len(repack.deletedFiles) > 0) {
+ commitGraphsDir := filepath.Join(repoPath, "objects", "info", "commit-graphs")
+ if graphEntries, err := os.ReadDir(commitGraphsDir); err != nil {
+ if !errors.Is(err, os.ErrNotExist) {
+ return err
+ }
+ } else if len(graphEntries) > 0 {
+ walGraphsDir := filepath.Join(transaction.walFilesPath(), "commit-graphs")
+ if err := os.MkdirAll(walGraphsDir, perm.PrivateDir); err != nil {
+ return fmt.Errorf("creating commit-graphs dir in WAL dir: %w", err)
+ }
+ for _, entry := range graphEntries {
+ if err := os.Link(
+ filepath.Join(commitGraphsDir, entry.Name()),
+ filepath.Join(walGraphsDir, entry.Name()),
+ ); err != nil {
+ return fmt.Errorf("linking commit-graphs entry to WAL dir: %w", err)
+ }
+ }
+ }
+ }
+
+ if err := safe.NewSyncer().Sync(transaction.walFilesPath()); err != nil {
+ return fmt.Errorf("sync: %w", err)
+ }
+
+ return nil
+}
+
+// packfileExtensions contains the packfile extension and its dependencies. They will be collected after running
+// repacking command.
+var packfileExtensions = map[string]struct{}{
+ "multi-pack-index": {},
+ ".pack": {},
+ ".idx": {},
+ ".rev": {},
+ ".mtimes": {},
+ ".bitmap": {},
+}
+
+// collectPackfiles collects the list of packfiles and their luggage files.
+func (mgr *TransactionManager) collectPackfiles(ctx context.Context, repoPath string) (map[string]struct{}, error) {
+ files, err := os.ReadDir(filepath.Join(repoPath, "objects", "pack"))
+ if err != nil {
+ // The repository has not been packed before. Nothing to collect.
+ if errors.Is(err, os.ErrNotExist) {
+ return nil, nil
+ }
+ return nil, fmt.Errorf("reading objects/pack dir: %w", err)
+ }
+
+ // Filter packfiles and relevant files.
+ collectedFiles := make(map[string]struct{})
+ for _, file := range files {
+ // objects/pack directory should not include any sub-directory. We can simply ignore them.
+ if file.IsDir() {
+ continue
+ }
+ for extension := range packfileExtensions {
+ if strings.HasSuffix(file.Name(), extension) {
+ collectedFiles[file.Name()] = struct{}{}
+ }
+ }
+ }
+
+ return collectedFiles, nil
+}
+
// unwrapExpectedError unwraps expected errors that may occur and returns them directly to the caller.
func unwrapExpectedError(err error) error {
// The manager controls its own execution context and it is canceled only when Stop is called.
@@ -1830,8 +2066,14 @@ func (mgr *TransactionManager) verifyHousekeeping(ctx context.Context, transacti
return nil, fmt.Errorf("verifying pack refs: %w", err)
}
+ repackEntry, err := mgr.verifyRepacking(mgr.ctx, transaction)
+ if err != nil {
+ return nil, fmt.Errorf("verifying repacking: %w", err)
+ }
+
return &gitalypb.LogEntry_Housekeeping{
PackRefs: packRefsEntry,
+ Repack: repackEntry,
}, nil
}
@@ -1892,6 +2134,150 @@ func (mgr *TransactionManager) verifyPackRefs(ctx context.Context, transaction *
}, nil
}
+// The function verifyRepacking confirms that the repacking process can integrate a new collection of packfiles.
+// Repacking should generally be risk-free as it systematically arranges packfiles and removes old, unused objects. The
+// standard housekeeping procedure includes a grace period, in days, to protect objects that might be in use by
+// simultaneous operations. However, the transaction manager deals with concurrent requests more effectively. By
+// reviewing the list of transactions since the repacking began, we can identify potential conflicts involving
+// concurrent processes.
+// We need to watch out for two types of conflicts:
+// 1. Overlapping transactions could reference objects that have been pruned. Extracting these pruned objects from the
+// repacking process isn't straightforward. The repacking task must confirm that these referenced objects are still
+// accessible in the repacked repository. This is done using the `git-cat-file --batch-check` command. Simultaneously,
+// it's possible for these transactions to refer to new objects created by other concurrent transactions. So, we need to
+// setup a working directory and apply changed packfiles for checking.
+// 2. There might be transactions in progress that rely on dependencies that have been pruned. This type of conflict
+// can't be fully checked until Git v2.43 is implemented in Gitaly, as detailed in the GitLab epic
+// (https://gitlab.com/groups/gitlab-org/-/epics/11242).
+// It's important to note that this verification is specific to the `RepackObjectsStrategyFullWithCruft` strategy. Other
+// strategies focus solely on reorganizing packfiles and do not remove objects.
+func (mgr *TransactionManager) verifyRepacking(ctx context.Context, transaction *Transaction) (_ *gitalypb.LogEntry_Housekeeping_Repack, finalErr error) {
+ repack := transaction.runHousekeeping.repack
+ if repack == nil {
+ return nil, nil
+ }
+ // Other strategies re-organize packfiles without pruning unreachable objects. No need to run following
+ // expensive verification.
+ if repack.config.Strategy != housekeeping.RepackObjectsStrategyFullWithCruft {
+ return &gitalypb.LogEntry_Housekeeping_Repack{
+ NewFiles: repack.newFiles,
+ DeletedFiles: repack.deletedFiles,
+ }, nil
+ }
+
+ // Setup a working repository including the snapshot repository and all changes of concurrent transactions.
+ relativePath := transaction.snapshotRepository.GetRelativePath()
+ snapshot, err := newSnapshot(ctx,
+ mgr.storagePath,
+ filepath.Join(transaction.stagingDirectory, "staging"),
+ []string{relativePath},
+ )
+ if err != nil {
+ return nil, fmt.Errorf("new snapshot: %w", err)
+ }
+
+ workingRepository := mgr.repositoryFactory.Build(snapshot.relativePath(relativePath))
+ workingRepositoryPath, err := workingRepository.Path()
+ if err != nil {
+ return nil, fmt.Errorf("getting working repository path: %w", err)
+ }
+
+ objectHash, err := workingRepository.ObjectHash(ctx)
+ if err != nil {
+ return nil, fmt.Errorf("detecting object hash: %w", err)
+ }
+
+ referenceTips := map[string]struct{}{}
+ elm := mgr.committedEntries.Front()
+ for elm != nil {
+ committed := elm.Value.(*committedEntry)
+ if committed.lsn > transaction.snapshotLSN && committed.entry.RelativePath == transaction.relativePath {
+ // Collect reference tips. All of them should exist in the resulting packfile or new concurrent
+ // packfiles while repacking is running.
+ for _, txn := range committed.entry.GetReferenceTransactions() {
+ for _, change := range txn.GetChanges() {
+ oid := change.GetNewOid()
+ if !objectHash.IsZeroOID(git.ObjectID(oid)) {
+ referenceTips[string(oid)] = struct{}{}
+ }
+ }
+ }
+
+ // Linking the changes to the working directory if it does not match deleted files.
+ packPrefix := committed.entry.GetPackPrefix()
+ if packPrefix != "" {
+ shouldApply := true
+ for _, deletedFile := range repack.deletedFiles {
+ if packPrefix == deletedFile {
+ shouldApply = false
+ break
+ }
+ }
+ if shouldApply {
+ if err := mgr.applyPackFileToRepository(ctx, committed.lsn, committed.entry, workingRepositoryPath); err != nil {
+ return nil, fmt.Errorf("applying packfiles to working repository: %w", err)
+ }
+ }
+ }
+ }
+ elm = elm.Next()
+ }
+
+ if len(referenceTips) == 0 {
+ return &gitalypb.LogEntry_Housekeeping_Repack{
+ NewFiles: repack.newFiles,
+ DeletedFiles: repack.deletedFiles,
+ }, nil
+ }
+
+ // Although we have a wrapper for caching `git-cat-file` process, this command targets the snapshot repository
+ // that the repacking task depends on. This task might run for minutes to hours. It's unlikely there is any
+ // concurrent operation sharing this process. So, no need to cache it. An alternative solution is to read and
+ // parse the index files of the working directory. That approach avoids spawning `git-cat-file(1)`. In contrast,
+ // it requires storing the list of objects in memory.
+ var stderr bytes.Buffer
+ ctx, cancel := context.WithCancel(ctx)
+ catFileCmd, err := workingRepository.Exec(ctx, git.Command{
+ Name: "cat-file",
+ Flags: []git.Option{
+ git.Flag{Name: "--batch-check"},
+ git.Flag{Name: "-Z"},
+ },
+ }, git.WithSetupStdin(), git.WithSetupStdout(), git.WithStderr(&stderr))
+ if err != nil {
+ cancel()
+ return nil, structerr.New("spawning cat-file command: %w", err).WithMetadata("stderr", stderr.String())
+ }
+ defer func() {
+ cancel()
+ if err := catFileCmd.Wait(); finalErr == nil && err != nil && !errors.Is(err, context.Canceled) {
+ finalErr = structerr.New("waiting for cat-file command: %w", err).WithMetadata("stderr", stderr.String())
+ }
+ }()
+
+ stdout := bufio.NewReader(catFileCmd)
+ for oid := range referenceTips {
+ // Verify if the reference tip is reachable from the resulting single packfile.
+ if _, err := catFileCmd.Write([]byte(fmt.Sprintf("%s\000", oid))); err != nil {
+ return nil, fmt.Errorf("writing cat-file command: %w", err)
+ }
+ if _, err := catfile.ParseObjectInfo(objectHash, stdout, true); err != nil {
+ if errors.As(err, &catfile.NotFoundError{}) {
+ return nil, errRepackConflictPrunedObject
+ }
+ return nil, fmt.Errorf("reading object info: %w", err)
+ }
+ }
+ if _, err := catFileCmd.Write([]byte("flush\000")); err != nil {
+ return nil, fmt.Errorf("writing flush: %w", err)
+ }
+
+ return &gitalypb.LogEntry_Housekeeping_Repack{
+ NewFiles: repack.newFiles,
+ DeletedFiles: repack.deletedFiles,
+ }, nil
+}
+
// applyDefaultBranchUpdate applies the default branch update to the repository from the log entry.
func (mgr *TransactionManager) applyDefaultBranchUpdate(ctx context.Context, logEntry *gitalypb.LogEntry) error {
if logEntry.DefaultBranchUpdate == nil {
@@ -2311,7 +2697,11 @@ func (mgr *TransactionManager) applyRepositoryDeletion(ctx context.Context, logE
// has an associated pack file. This is done by hard linking the pack and index from the
// log into the repository's object directory.
func (mgr *TransactionManager) applyPackFile(ctx context.Context, lsn LSN, logEntry *gitalypb.LogEntry) error {
- packDirectory := filepath.Join(mgr.getAbsolutePath(logEntry.RelativePath), "objects", "pack")
+ return mgr.applyPackFileToRepository(ctx, lsn, logEntry, mgr.getAbsolutePath(logEntry.RelativePath))
+}
+
+func (mgr *TransactionManager) applyPackFileToRepository(ctx context.Context, lsn LSN, logEntry *gitalypb.LogEntry, repoPath string) error {
+ packDirectory := filepath.Join(repoPath, "objects", "pack")
for _, fileExtension := range []string{
".pack",
".idx",
@@ -2378,81 +2768,173 @@ func (mgr *TransactionManager) applyHousekeeping(ctx context.Context, lsn LSN, l
if logEntry.Housekeeping == nil {
return nil
}
+
+ if err := mgr.applyPackRefs(ctx, lsn, logEntry); err != nil {
+ return fmt.Errorf("applying pack refs: %w", err)
+ }
+
+ if err := mgr.applyRepacking(ctx, lsn, logEntry); err != nil {
+ return fmt.Errorf("applying repacking: %w", err)
+ }
+
+ return nil
+}
+
+func (mgr *TransactionManager) applyPackRefs(ctx context.Context, lsn LSN, logEntry *gitalypb.LogEntry) error {
+ if logEntry.Housekeeping.PackRefs == nil {
+ return nil
+ }
+
repositoryPath := mgr.getAbsolutePath(logEntry.RelativePath)
- if logEntry.Housekeeping.PackRefs != nil {
- // Remove packed-refs lock. While we shouldn't be producing any new stale locks, it makes sense to have
- // this for historic state until we're certain none of the repositories contain stale locks anymore.
- // This clean up is not needed afterward.
- if err := mgr.removePackedRefsLocks(ctx, repositoryPath); err != nil {
- return fmt.Errorf("applying pack-refs: %w", err)
+ // Remove packed-refs lock. While we shouldn't be producing any new stale locks, it makes sense to have
+ // this for historic state until we're certain none of the repositories contain stale locks anymore.
+ // This clean up is not needed afterward.
+ if err := mgr.removePackedRefsLocks(ctx, repositoryPath); err != nil {
+ return fmt.Errorf("applying pack-refs: %w", err)
+ }
+
+ packedRefsPath := filepath.Join(repositoryPath, "packed-refs")
+ // Replace the packed-refs file.
+ if err := os.Remove(packedRefsPath); err != nil {
+ if !errors.Is(err, os.ErrNotExist) {
+ return fmt.Errorf("removing existing pack-refs: %w", err)
}
+ }
+ if err := os.Link(
+ filepath.Join(walFilesPathForLSN(mgr.stateDirectory, lsn), "packed-refs"),
+ packedRefsPath,
+ ); err != nil {
+ return fmt.Errorf("linking new packed-refs: %w", err)
+ }
- packedRefsPath := filepath.Join(repositoryPath, "packed-refs")
- // Replace the packed-refs file.
- if err := os.Remove(packedRefsPath); err != nil {
+ modifiedDirs := map[string]struct{}{}
+ // Prune loose references. The log entry carries the list of fully qualified references to prune.
+ for _, ref := range logEntry.Housekeeping.PackRefs.PrunedRefs {
+ path := filepath.Join(repositoryPath, string(ref))
+ if err := os.Remove(path); err != nil {
if !errors.Is(err, os.ErrNotExist) {
- return fmt.Errorf("removing existing pack-refs: %w", err)
+ return structerr.New("pruning loose reference: %w", err).WithMetadata("ref", path)
}
}
- if err := os.Link(
- filepath.Join(walFilesPathForLSN(mgr.stateDirectory, lsn), "packed-refs"),
- packedRefsPath,
- ); err != nil {
- return fmt.Errorf("linking new packed-refs: %w", err)
- }
+ modifiedDirs[filepath.Dir(path)] = struct{}{}
+ }
- modifiedDirs := map[string]struct{}{}
- // Prune loose references. The log entry carries the list of fully qualified references to prune.
- for _, ref := range logEntry.Housekeeping.PackRefs.PrunedRefs {
- path := filepath.Join(repositoryPath, string(ref))
- if err := os.Remove(path); err != nil {
- if !errors.Is(err, os.ErrNotExist) {
- return structerr.New("pruning loose reference: %w", err).WithMetadata("ref", path)
+ syncer := safe.NewSyncer()
+ // Traverse all modified dirs back to the root "refs" dir of the repository. Remove any empty directory
+ // along the way. It prevents leaving empty dirs around after a loose ref is pruned. `git-pack-refs`
+ // command does dir removal for us, but in staginge repository during preparation stage. In the actual
+ // repository, we need to do it ourselves.
+ rootRefDir := filepath.Join(repositoryPath, "refs")
+ for dir := range modifiedDirs {
+ for dir != rootRefDir {
+ if isEmpty, err := isDirEmpty(dir); err != nil {
+ // If a dir does not exist, it properly means a directory may already be deleted by a
+ // previous interrupted attempt on applying the log entry. We simply ignore the error
+ // and move up the directory hierarchy.
+ if errors.Is(err, fs.ErrNotExist) {
+ dir = filepath.Dir(dir)
+ continue
+ } else {
+ return fmt.Errorf("checking empty ref dir: %w", err)
}
+ } else if !isEmpty {
+ break
}
- modifiedDirs[filepath.Dir(path)] = struct{}{}
- }
-
- syncer := safe.NewSyncer()
- // Traverse all modified dirs back to the root "refs" dir of the repository. Remove any empty directory
- // along the way. It prevents leaving empty dirs around after a loose ref is pruned. `git-pack-refs`
- // command does dir removal for us, but in staginge repository during preparation stage. In the actual
- // repository, we need to do it ourselves.
- rootRefDir := filepath.Join(repositoryPath, "refs")
- for dir := range modifiedDirs {
- for dir != rootRefDir {
- if isEmpty, err := isDirEmpty(dir); err != nil {
- // If a dir does not exist, it properly means a directory may already be deleted by a
- // previous interrupted attempt on applying the log entry. We simply ignore the error
- // and move up the directory hierarchy.
- if errors.Is(err, fs.ErrNotExist) {
- dir = filepath.Dir(dir)
- continue
- } else {
- return fmt.Errorf("checking empty ref dir: %w", err)
- }
- } else if !isEmpty {
- break
- }
- if err := os.Remove(dir); err != nil {
- return fmt.Errorf("removing empty ref dir: %w", err)
- }
- dir = filepath.Dir(dir)
+ if err := os.Remove(dir); err != nil {
+ return fmt.Errorf("removing empty ref dir: %w", err)
+ }
+ dir = filepath.Dir(dir)
+ }
+ // If there is any empty dir along the way, it's removed and dir pointer moves up until the dir
+ // is not empty or reaching the root dir. That one should be fsynced to flush the dir removal.
+ // If there is no empty dir, it stays at the dir of pruned refs, which also needs a flush.
+ if err := syncer.Sync(dir); err != nil {
+ return fmt.Errorf("sync dir: %w", err)
+ }
+ }
+
+ // Sync the root of the repository to flush packed-refs replacement.
+ if err := syncer.SyncParent(packedRefsPath); err != nil {
+ return fmt.Errorf("sync parent: %w", err)
+ }
+ return nil
+}
+
+// applyRepacking applies the new packfile set and removed known pruned packfiles. New packfiles created by concurrent
+// changes are kept intact.
+func (mgr *TransactionManager) applyRepacking(ctx context.Context, lsn LSN, logEntry *gitalypb.LogEntry) error {
+ if logEntry.Housekeeping.Repack == nil {
+ return nil
+ }
+
+ repack := logEntry.Housekeeping.Repack
+ repoPath := mgr.getAbsolutePath(logEntry.RelativePath)
+
+ for _, file := range repack.NewFiles {
+ if err := os.Link(
+ filepath.Join(walFilesPathForLSN(mgr.stateDirectory, lsn), file),
+ filepath.Join(repoPath, "objects", "pack", file),
+ ); err != nil {
+ // A new resulting packfile might exist if the log entry is re-applied after a crash.
+ if !errors.Is(err, os.ErrExist) {
+ return fmt.Errorf("linking new packfile: %w", err)
}
- // If there is any empty dir along the way, it's removed and dir pointer moves up until the dir
- // is not empty or reaching the root dir. That one should be fsynced to flush the dir removal.
- // If there is no empty dir, it stays at the dir of pruned refs, which also needs a flush.
- if err := syncer.Sync(dir); err != nil {
- return fmt.Errorf("sync dir: %w", err)
+ }
+ }
+
+ // Clean up and apply commit graphs.
+ walGraphsDir := filepath.Join(walFilesPathForLSN(mgr.stateDirectory, lsn), "commit-graphs")
+ if graphEntries, err := os.ReadDir(walGraphsDir); err != nil {
+ if !errors.Is(err, os.ErrNotExist) {
+ return err
+ }
+ } else if len(graphEntries) > 0 {
+ commitGraphsDir := filepath.Join(repoPath, "objects", "info", "commit-graphs")
+ if err := os.RemoveAll(commitGraphsDir); err != nil {
+ return fmt.Errorf("reseting commit-graphs dir: %w", err)
+ }
+ if err := os.MkdirAll(commitGraphsDir, perm.PrivateDir); err != nil {
+ return fmt.Errorf("creating commit-graphs dir: %w", err)
+ }
+ for _, entry := range graphEntries {
+ if err := os.Link(
+ filepath.Join(walGraphsDir, entry.Name()),
+ filepath.Join(commitGraphsDir, entry.Name()),
+ ); err != nil {
+ return fmt.Errorf("linking commit-graphs entry: %w", err)
}
}
+ }
- // Sync the root of the repository to flush packed-refs replacement.
- if err := syncer.SyncParent(packedRefsPath); err != nil {
- return fmt.Errorf("sync parent: %w", err)
+ for _, file := range repack.DeletedFiles {
+ if err := os.Remove(filepath.Join(repoPath, "objects", "pack", file)); err != nil {
+ // Repacking task is the only operation that touch an on-disk packfile. Other operations should
+ // only create new packfiles. However, after a crash, a pre-existing packfile might be cleaned up
+ // beforehand.
+ if !errors.Is(err, os.ErrNotExist) {
+ return fmt.Errorf("clean up repacked packfile: %w", err)
+ }
}
}
+
+ // During migration to the new transaction system, loose objects might still exist here and there. So, this task
+ // needs to clean up redundant loose objects. After the target repository runs repacking for the first time,
+ // there shouldn't be any further loose objects. All of them exist in packfiles. Afterward, this command will
+ // exist instantly. We can remove this run after the transaction system is fully applied.
+ repo := mgr.repositoryFactory.Build(logEntry.RelativePath)
+ var stderr bytes.Buffer
+ if err := repo.ExecAndWait(ctx, git.Command{
+ Name: "prune-packed",
+ Flags: []git.Option{git.Flag{Name: "--quiet"}},
+ }, git.WithStderr(&stderr)); err != nil {
+ return structerr.New("exec prune-packed: %w", err).WithMetadata("stderr", stderr.String())
+ }
+
+ if err := safe.NewSyncer().Sync(filepath.Join(repoPath, "objects")); err != nil {
+ return fmt.Errorf("sync recursive: %w", err)
+ }
+
return nil
}
diff --git a/internal/gitaly/storage/storagemgr/transaction_manager_housekeeping_test.go b/internal/gitaly/storage/storagemgr/transaction_manager_housekeeping_test.go
index d776185a7..ef3d1a2b1 100644
--- a/internal/gitaly/storage/storagemgr/transaction_manager_housekeeping_test.go
+++ b/internal/gitaly/storage/storagemgr/transaction_manager_housekeeping_test.go
@@ -11,6 +11,7 @@ import (
"github.com/stretchr/testify/require"
"gitlab.com/gitlab-org/gitaly/v16/internal/git"
"gitlab.com/gitlab-org/gitaly/v16/internal/git/gittest"
+ "gitlab.com/gitlab-org/gitaly/v16/internal/git/housekeeping"
"gitlab.com/gitlab-org/gitaly/v16/internal/gitaly/config"
"gitlab.com/gitlab-org/gitaly/v16/internal/helper/perm"
"gitlab.com/gitlab-org/gitaly/v16/internal/testhelper"
@@ -1092,3 +1093,2864 @@ func generateHousekeepingPackRefsTests(t *testing.T, ctx context.Context, testPa
},
}
}
+
+// A shortcut to return a digest hash correspondingly to the current testing object hash format. Names of
+// packfiles have digest hashes. The following tests examine on-disk packfiles of WAL log entry's luggage and
+// the destination repository. We could use an incremental names for attached packfiles of the log entry,
+// eventually, they will be applied into the repository. The actual on-disk packfiles should match the their
+// IDs, which are specified in the *.idx files. So, it's essential to include the digest hash in the tests
+// although it's annoying to switch between different hash formats.
+func hash(tb testing.TB, sha1 string, sha256 string) string {
+ return gittest.ObjectHashDependent(tb, map[string]string{
+ git.ObjectHashSHA1.Format: sha1,
+ git.ObjectHashSHA256.Format: sha256,
+ })
+}
+
+type walDirectoryState struct {
+ lsn LSN
+ includePackfiles []string
+ includeMultiIndexes []string
+ includeCommitGraphs []string
+ includeObjects []git.ObjectID
+}
+
+func generateDirectoryState(cfg config.Cfg, stats []*walDirectoryState) testhelper.DirectoryState {
+ state := testhelper.DirectoryState{}
+ if len(stats) == 0 {
+ return state
+ }
+ state["/"] = testhelper.DirectoryEntry{Mode: fs.ModeDir | perm.PrivateDir}
+ state["/wal"] = testhelper.DirectoryEntry{Mode: fs.ModeDir | perm.PrivateDir}
+ for _, stat := range stats {
+ walDir := fmt.Sprintf("/wal/%d", stat.lsn)
+ state[walDir] = testhelper.DirectoryEntry{Mode: fs.ModeDir | perm.PrivateDir}
+ for _, packfile := range stat.includePackfiles {
+ state[filepath.Join(walDir, packfile+".pack")] = anyDirectoryEntry(cfg)
+ state[filepath.Join(walDir, packfile+".idx")] = anyDirectoryEntry(cfg)
+ state[filepath.Join(walDir, packfile+".rev")] = anyDirectoryEntry(cfg)
+ }
+ for _, index := range stat.includeMultiIndexes {
+ state[filepath.Join(walDir, "multi-pack-index")] = anyDirectoryEntryWithPerm(cfg, perm.SharedFile)
+ state[filepath.Join(walDir, index+".bitmap")] = anyDirectoryEntry(cfg)
+ }
+ for _, graph := range stat.includeCommitGraphs {
+ state[filepath.Join(walDir, "commit-graphs")] = testhelper.DirectoryEntry{Mode: fs.ModeDir | perm.PrivateDir}
+ state[filepath.Join(walDir, "commit-graphs", "commit-graph-chain")] = anyDirectoryEntry(cfg)
+ state[filepath.Join(walDir, "commit-graphs", graph+".graph")] = anyDirectoryEntry(cfg)
+ }
+ if len(stat.includeObjects) != 0 {
+ state[filepath.Join(walDir, "objects.idx")] = indexFileDirectoryEntry(cfg)
+ state[filepath.Join(walDir, "objects.rev")] = reverseIndexFileDirectoryEntry(cfg)
+ state[filepath.Join(walDir, "objects.pack")] = packFileDirectoryEntry(cfg, stat.includeObjects)
+ }
+ }
+ return state
+}
+
+// generateHousekeepingRepackingStrategyTests returns a set of tests which run repacking with different strategies and
+// settings.
+func generateHousekeepingRepackingStrategyTests(t *testing.T, ctx context.Context, testPartitionID partitionID, relativePath string) []transactionTestCase {
+ customSetup := func(t *testing.T, ctx context.Context, testPartitionID partitionID, relativePath string) testTransactionSetup {
+ setup := setupTest(t, ctx, testPartitionID, relativePath)
+ gittest.WriteRef(t, setup.Config, setup.RepositoryPath, "refs/heads/main", setup.Commits.Third.OID)
+ gittest.WriteRef(t, setup.Config, setup.RepositoryPath, "refs/heads/branch", setup.Commits.Diverging.OID)
+ setup.Commits.Unreachable = testTransactionCommit{
+ OID: gittest.WriteCommit(t, setup.Config, setup.RepositoryPath, gittest.WithParents(setup.Commits.Second.OID), gittest.WithMessage("unreachable commit")),
+ }
+ setup.Commits.Orphan = testTransactionCommit{
+ OID: gittest.WriteCommit(t, setup.Config, setup.RepositoryPath, gittest.WithParents(), gittest.WithMessage("orphan commit")),
+ }
+ // Create an initial packfile, but keep loose objects intact.
+ // gittest.Exec(t, setup.Config, "-C", setup.RepositoryPath, "repack", "-ad")
+ return setup
+ }
+ setup := customSetup(t, ctx, testPartitionID, relativePath)
+
+ defaultReferences := map[git.ReferenceName]git.ObjectID{
+ "refs/heads/main": setup.Commits.Third.OID,
+ "refs/heads/branch": setup.Commits.Diverging.OID,
+ }
+ defaultReachableObjects := []git.ObjectID{
+ gittest.DefaultObjectHash.EmptyTreeOID,
+ setup.Commits.First.OID,
+ setup.Commits.Second.OID,
+ setup.Commits.Third.OID,
+ setup.Commits.Diverging.OID,
+ }
+ return []transactionTestCase{
+ {
+ desc: "run repacking (IncrementalWithUnreachable)",
+ customSetup: customSetup,
+ steps: steps{
+ StartManager{
+ ModifyStorage: func(tb testing.TB, cfg config.Cfg, storagePath string) {
+ repoPath := filepath.Join(storagePath, setup.RelativePath)
+ gittest.Exec(tb, cfg, "-C", repoPath, "repack", "-ad")
+ },
+ },
+ Begin{
+ TransactionID: 1,
+ RelativePath: setup.RelativePath,
+ },
+ RunRepack{
+ TransactionID: 1,
+ Config: housekeeping.RepackObjectsConfig{
+ Strategy: housekeeping.RepackObjectsStrategyIncrementalWithUnreachable,
+ },
+ },
+ Commit{
+ TransactionID: 1,
+ },
+ },
+ expectedState: StateAssertion{
+ Database: DatabaseState{
+ string(keyAppliedLSN(setup.PartitionID)): LSN(1).toProto(),
+ },
+ Repositories: RepositoryStates{
+ setup.RelativePath: {
+ DefaultBranch: "refs/heads/main",
+ References: &ReferencesState{LooseReferences: defaultReferences},
+ Packfiles: &PackfilesState{
+ // Loose objects stay intact.
+ LooseObjects: []git.ObjectID{
+ setup.Commits.Orphan.OID,
+ setup.Commits.Unreachable.OID,
+ },
+ Packfiles: map[string]*PackfileState{
+ hash(t,
+ "pack-5f624d294fda1b8df86f1c286c6a66757b44126e",
+ "pack-c57ed22f16c0a35f04febe26eac0fe8974b2b4ab3469d1ece0bc2983588ad44e",
+ ): {
+ Objects: defaultReachableObjects,
+ HasBitmap: true,
+ HasReverseIndex: true,
+ },
+ },
+ HasMultiPackIndex: false,
+ HasCommitGraphs: false,
+ },
+ },
+ },
+ Directory: testhelper.DirectoryState{
+ "/": {Mode: fs.ModeDir | perm.PrivateDir},
+ "/wal": {Mode: fs.ModeDir | perm.PrivateDir},
+ "/wal/1": {Mode: fs.ModeDir | perm.PrivateDir},
+ },
+ },
+ },
+ {
+ desc: "run repacking (FullWithUnreachable) on a repository with an existing packfile",
+ customSetup: customSetup,
+ steps: steps{
+ StartManager{
+ ModifyStorage: func(tb testing.TB, cfg config.Cfg, storagePath string) {
+ repoPath := filepath.Join(storagePath, setup.RelativePath)
+ gittest.Exec(tb, cfg, "-C", repoPath, "repack", "-ad")
+ },
+ },
+ Begin{
+ TransactionID: 1,
+ RelativePath: setup.RelativePath,
+ },
+ RunRepack{
+ TransactionID: 1,
+ Config: housekeeping.RepackObjectsConfig{
+ Strategy: housekeeping.RepackObjectsStrategyFullWithUnreachable,
+ WriteBitmap: true,
+ WriteMultiPackIndex: true,
+ },
+ },
+ Commit{
+ TransactionID: 1,
+ },
+ },
+ expectedState: StateAssertion{
+ Database: DatabaseState{
+ string(keyAppliedLSN(setup.PartitionID)): LSN(1).toProto(),
+ },
+ Repositories: RepositoryStates{
+ setup.RelativePath: {
+ DefaultBranch: "refs/heads/main",
+ References: &ReferencesState{LooseReferences: defaultReferences},
+ Packfiles: &PackfilesState{
+ // Unreachable objects are packed.
+ LooseObjects: nil,
+ Packfiles: map[string]*PackfileState{
+ hash(t,
+ "pack-689b1fa746246c50a8b0f3469a06c7ae68af9926",
+ "pack-3506da99c69e8bbb4e3122636a486ffcc3506f08d24426823a2a394a7fb16b94",
+ ): {
+ Objects: append(defaultReachableObjects,
+ setup.Commits.Orphan.OID,
+ setup.Commits.Unreachable.OID,
+ ),
+ HasBitmap: false,
+ HasReverseIndex: true,
+ },
+ },
+ HasMultiPackIndex: true,
+ HasCommitGraphs: false,
+ },
+ },
+ },
+ Directory: generateDirectoryState(setup.Config, []*walDirectoryState{
+ {
+ lsn: 1,
+ includePackfiles: []string{hash(t,
+ "pack-689b1fa746246c50a8b0f3469a06c7ae68af9926",
+ "pack-3506da99c69e8bbb4e3122636a486ffcc3506f08d24426823a2a394a7fb16b94",
+ )},
+ includeMultiIndexes: []string{hash(t,
+ "multi-pack-index-dece3dfef114aa668c61339e0d4eb081af62ce68",
+ "multi-pack-index-bf9ee4098624aeb3fae4990d943443f5759d6d63c8cca686b19fb48e3c6a6f25",
+ )},
+ },
+ }),
+ },
+ },
+ {
+ desc: "run repacking (FullWithUnreachable) on a repository without any packfile",
+ customSetup: customSetup,
+ steps: steps{
+ StartManager{},
+ Begin{
+ TransactionID: 1,
+ RelativePath: setup.RelativePath,
+ },
+ RunRepack{
+ TransactionID: 1,
+ Config: housekeeping.RepackObjectsConfig{
+ Strategy: housekeeping.RepackObjectsStrategyFullWithUnreachable,
+ WriteBitmap: true,
+ WriteMultiPackIndex: true,
+ },
+ },
+ Commit{
+ TransactionID: 1,
+ },
+ },
+ expectedState: StateAssertion{
+ Database: DatabaseState{
+ string(keyAppliedLSN(setup.PartitionID)): LSN(1).toProto(),
+ },
+ Repositories: RepositoryStates{
+ setup.RelativePath: {
+ DefaultBranch: "refs/heads/main",
+ References: &ReferencesState{LooseReferences: defaultReferences},
+ Packfiles: &PackfilesState{
+ // Interestingly, loose unreachable objects stay untouched!
+ LooseObjects: []git.ObjectID{
+ setup.Commits.Orphan.OID,
+ setup.Commits.Unreachable.OID,
+ },
+ Packfiles: map[string]*PackfileState{
+ hash(t,
+ "pack-a81cd79eb9f32ce0afbdc15dec51c7141029e54c",
+ "pack-ce649b013f4191c500c7c4de5fe407120314c83354944e5639bf1a33a2c94110",
+ ): {
+ Objects: defaultReachableObjects,
+ HasBitmap: false,
+ HasReverseIndex: true,
+ },
+ },
+ HasMultiPackIndex: true,
+ HasCommitGraphs: false,
+ },
+ },
+ },
+ Directory: generateDirectoryState(setup.Config, []*walDirectoryState{
+ {
+ lsn: 1,
+ includePackfiles: []string{hash(t,
+ "pack-a81cd79eb9f32ce0afbdc15dec51c7141029e54c",
+ "pack-ce649b013f4191c500c7c4de5fe407120314c83354944e5639bf1a33a2c94110",
+ )},
+ includeMultiIndexes: []string{hash(t,
+ "multi-pack-index-f881063fbb14e481b5be5619df02c9874dbe5d3b",
+ "multi-pack-index-67d1f13534c85393277dc006444eee9b6670b6f1554faa43e051fa9402efa3a8",
+ )},
+ },
+ }),
+ },
+ },
+ {
+ desc: "run repacking (Geometric) on a repository without any packfile",
+ customSetup: customSetup,
+ steps: steps{
+ StartManager{},
+ Begin{
+ TransactionID: 1,
+ RelativePath: setup.RelativePath,
+ },
+ RunRepack{
+ TransactionID: 1,
+ Config: housekeeping.RepackObjectsConfig{
+ Strategy: housekeeping.RepackObjectsStrategyGeometric,
+ WriteBitmap: true,
+ WriteMultiPackIndex: true,
+ },
+ },
+ Commit{
+ TransactionID: 1,
+ },
+ },
+ expectedState: StateAssertion{
+ Database: DatabaseState{
+ string(keyAppliedLSN(setup.PartitionID)): LSN(1).toProto(),
+ },
+ Repositories: RepositoryStates{
+ setup.RelativePath: {
+ DefaultBranch: "refs/heads/main",
+ References: &ReferencesState{LooseReferences: defaultReferences},
+ Packfiles: &PackfilesState{
+ LooseObjects: nil,
+ Packfiles: map[string]*PackfileState{
+ hash(t,
+ "pack-e1b234fb89567714fc382281c7f89a363f4ac115",
+ "pack-d7214dae50142c99e75bf21d679b8cc14bc5d82cdb84dc23f39120101a6ed5e9",
+ ): {
+ Objects: append(defaultReachableObjects,
+ setup.Commits.Orphan.OID,
+ setup.Commits.Unreachable.OID,
+ ),
+ HasBitmap: false,
+ HasReverseIndex: true,
+ },
+ },
+ HasMultiPackIndex: true,
+ HasCommitGraphs: false,
+ },
+ },
+ },
+ Directory: generateDirectoryState(setup.Config, []*walDirectoryState{
+ {
+ lsn: 1,
+ includePackfiles: []string{hash(t,
+ "pack-e1b234fb89567714fc382281c7f89a363f4ac115",
+ "pack-d7214dae50142c99e75bf21d679b8cc14bc5d82cdb84dc23f39120101a6ed5e9",
+ )},
+ includeMultiIndexes: []string{hash(t,
+ "multi-pack-index-70fc88df37859b5f9c0d68a6b4ed42e9a6d3819e",
+ "multi-pack-index-d55aca104a7164aa65e984e53ebe2633c515eba475a68dccc982156fefaf9c51",
+ )},
+ },
+ }),
+ },
+ },
+ {
+ desc: "run repacking (Geometric) on a repository having an existing packfile",
+ customSetup: customSetup,
+ steps: steps{
+ StartManager{
+ ModifyStorage: func(tb testing.TB, cfg config.Cfg, storagePath string) {
+ repoPath := filepath.Join(storagePath, setup.RelativePath)
+ gittest.Exec(tb, cfg, "-C", repoPath, "repack", "-ad")
+ },
+ },
+ Begin{
+ TransactionID: 1,
+ RelativePath: setup.RelativePath,
+ },
+ RunRepack{
+ TransactionID: 1,
+ Config: housekeeping.RepackObjectsConfig{
+ Strategy: housekeeping.RepackObjectsStrategyGeometric,
+ WriteBitmap: true,
+ WriteMultiPackIndex: true,
+ },
+ },
+ Commit{
+ TransactionID: 1,
+ },
+ },
+ expectedState: StateAssertion{
+ Database: DatabaseState{
+ string(keyAppliedLSN(setup.PartitionID)): LSN(1).toProto(),
+ },
+ Repositories: RepositoryStates{
+ setup.RelativePath: {
+ DefaultBranch: "refs/heads/main",
+ References: &ReferencesState{LooseReferences: defaultReferences},
+ Packfiles: &PackfilesState{
+ LooseObjects: nil,
+ Packfiles: map[string]*PackfileState{
+ // Initial packfile.
+ hash(t,
+ "pack-5f624d294fda1b8df86f1c286c6a66757b44126e",
+ "pack-c57ed22f16c0a35f04febe26eac0fe8974b2b4ab3469d1ece0bc2983588ad44e",
+ ): {
+ Objects: defaultReachableObjects,
+ HasBitmap: false,
+ HasReverseIndex: true,
+ },
+ // New packfile that contains unreachable objects. This
+ // is a co-incident, it follows the geometric
+ // progression.
+ hash(t,
+ "pack-f20a6e68adae9088db85f994838091d53fbaf608",
+ "pack-aa6d40f5f019492a7cc11291ab68666ae7ac2a23e66762905581c44523bb12bd",
+ ): {
+ Objects: []git.ObjectID{
+ setup.Commits.Orphan.OID,
+ setup.Commits.Unreachable.OID,
+ },
+ HasBitmap: false,
+ HasReverseIndex: true,
+ },
+ },
+ HasMultiPackIndex: true,
+ HasCommitGraphs: false,
+ },
+ },
+ },
+ Directory: generateDirectoryState(setup.Config, []*walDirectoryState{
+ {
+ lsn: 1,
+ includePackfiles: []string{hash(t,
+ "pack-f20a6e68adae9088db85f994838091d53fbaf608",
+ "pack-aa6d40f5f019492a7cc11291ab68666ae7ac2a23e66762905581c44523bb12bd",
+ )},
+ includeMultiIndexes: []string{hash(t,
+ "multi-pack-index-8b9315908033879678e7a6d7ff16d8cf3f419181",
+ "multi-pack-index-c2be71e50a69e0706c7818e608144ad07eca4425a4d6446d4b4f1a667f756500",
+ )},
+ },
+ }),
+ },
+ },
+ {
+ desc: "run repacking (FullWithCruft) on a repository having all loose objects",
+ customSetup: customSetup,
+ steps: steps{
+ StartManager{},
+ Begin{
+ TransactionID: 1,
+ RelativePath: setup.RelativePath,
+ },
+ RunRepack{
+ TransactionID: 1,
+ Config: housekeeping.RepackObjectsConfig{
+ Strategy: housekeeping.RepackObjectsStrategyFullWithCruft,
+ WriteBitmap: true,
+ WriteMultiPackIndex: true,
+ },
+ },
+ Commit{
+ TransactionID: 1,
+ },
+ },
+ expectedState: StateAssertion{
+ Database: DatabaseState{
+ string(keyAppliedLSN(setup.PartitionID)): LSN(1).toProto(),
+ },
+ Repositories: RepositoryStates{
+ setup.RelativePath: {
+ DefaultBranch: "refs/heads/main",
+ References: &ReferencesState{LooseReferences: defaultReferences},
+ Packfiles: &PackfilesState{
+ // Interestingly, loose unreachable objects stay untouched!
+ LooseObjects: []git.ObjectID{
+ setup.Commits.Orphan.OID,
+ setup.Commits.Unreachable.OID,
+ },
+ Packfiles: map[string]*PackfileState{
+ hash(t,
+ "pack-a81cd79eb9f32ce0afbdc15dec51c7141029e54c",
+ "pack-ce649b013f4191c500c7c4de5fe407120314c83354944e5639bf1a33a2c94110",
+ ): {
+ Objects: defaultReachableObjects,
+ HasBitmap: false,
+ HasReverseIndex: true,
+ },
+ },
+ HasMultiPackIndex: true,
+ HasCommitGraphs: true,
+ },
+ },
+ },
+ Directory: generateDirectoryState(setup.Config, []*walDirectoryState{
+ {
+ lsn: 1,
+ includePackfiles: []string{hash(t,
+ "pack-a81cd79eb9f32ce0afbdc15dec51c7141029e54c",
+ "pack-ce649b013f4191c500c7c4de5fe407120314c83354944e5639bf1a33a2c94110",
+ )},
+ includeMultiIndexes: []string{hash(t,
+ "multi-pack-index-f881063fbb14e481b5be5619df02c9874dbe5d3b",
+ "multi-pack-index-67d1f13534c85393277dc006444eee9b6670b6f1554faa43e051fa9402efa3a8",
+ )},
+ includeCommitGraphs: []string{hash(t,
+ "graph-5eddc89b8217451ecd51182f91ddf6f58b20f0f7",
+ "graph-d7a6f93863d026b02376bf869ab4fa23a7cd6bdbc013543741352b574cc19606",
+ )},
+ },
+ }),
+ },
+ },
+ {
+ desc: "run repacking (FullWithCruft) on a repository whose objects are packed",
+ customSetup: customSetup,
+ steps: steps{
+ StartManager{
+ ModifyStorage: func(tb testing.TB, cfg config.Cfg, storagePath string) {
+ repoPath := filepath.Join(storagePath, setup.RelativePath)
+ gittest.Exec(tb, cfg, "-C", repoPath, "repack", "-adl")
+ gittest.Exec(tb, cfg, "-C", repoPath, "repack", "-adl", "--keep-unreachable")
+ },
+ },
+ Begin{
+ TransactionID: 1,
+ RelativePath: setup.RelativePath,
+ },
+ RunRepack{
+ TransactionID: 1,
+ Config: housekeeping.RepackObjectsConfig{
+ Strategy: housekeeping.RepackObjectsStrategyFullWithCruft,
+ WriteBitmap: true,
+ WriteMultiPackIndex: true,
+ },
+ },
+ Commit{
+ TransactionID: 1,
+ },
+ },
+ expectedState: StateAssertion{
+ Database: DatabaseState{
+ string(keyAppliedLSN(setup.PartitionID)): LSN(1).toProto(),
+ },
+ Repositories: RepositoryStates{
+ setup.RelativePath: {
+ DefaultBranch: "refs/heads/main",
+ References: &ReferencesState{LooseReferences: defaultReferences},
+ Packfiles: &PackfilesState{
+ // Unreachable objects are pruned.
+ LooseObjects: nil,
+ Packfiles: map[string]*PackfileState{
+ hash(t,
+ "pack-a81cd79eb9f32ce0afbdc15dec51c7141029e54c",
+ "pack-ce649b013f4191c500c7c4de5fe407120314c83354944e5639bf1a33a2c94110",
+ ): {
+ Objects: defaultReachableObjects,
+ HasBitmap: false,
+ HasReverseIndex: true,
+ },
+ },
+ HasMultiPackIndex: true,
+ HasCommitGraphs: true,
+ },
+ },
+ },
+ Directory: generateDirectoryState(setup.Config, []*walDirectoryState{
+ {
+ lsn: 1,
+ includePackfiles: []string{hash(t,
+ "pack-a81cd79eb9f32ce0afbdc15dec51c7141029e54c",
+ "pack-ce649b013f4191c500c7c4de5fe407120314c83354944e5639bf1a33a2c94110",
+ )},
+ includeMultiIndexes: []string{hash(t,
+ "multi-pack-index-f881063fbb14e481b5be5619df02c9874dbe5d3b",
+ "multi-pack-index-67d1f13534c85393277dc006444eee9b6670b6f1554faa43e051fa9402efa3a8",
+ )},
+ includeCommitGraphs: []string{hash(t,
+ "graph-5eddc89b8217451ecd51182f91ddf6f58b20f0f7",
+ "graph-d7a6f93863d026b02376bf869ab4fa23a7cd6bdbc013543741352b574cc19606",
+ )},
+ },
+ }),
+ },
+ },
+ {
+ desc: "run repacking (FullWithCruft) on a repository having both packfile and loose unreachable objects",
+ customSetup: customSetup,
+ steps: steps{
+ StartManager{
+ ModifyStorage: func(tb testing.TB, cfg config.Cfg, storagePath string) {
+ repoPath := filepath.Join(storagePath, setup.RelativePath)
+ gittest.Exec(tb, cfg, "-C", repoPath, "repack", "-adl")
+ },
+ },
+ Begin{
+ TransactionID: 1,
+ RelativePath: setup.RelativePath,
+ },
+ RunRepack{
+ TransactionID: 1,
+ Config: housekeeping.RepackObjectsConfig{
+ Strategy: housekeeping.RepackObjectsStrategyFullWithCruft,
+ WriteBitmap: true,
+ WriteMultiPackIndex: true,
+ },
+ },
+ Commit{
+ TransactionID: 1,
+ },
+ },
+ expectedState: StateAssertion{
+ Database: DatabaseState{
+ string(keyAppliedLSN(setup.PartitionID)): LSN(1).toProto(),
+ },
+ Repositories: RepositoryStates{
+ setup.RelativePath: {
+ DefaultBranch: "refs/heads/main",
+ References: &ReferencesState{LooseReferences: defaultReferences},
+ Packfiles: &PackfilesState{
+ // Interestingly, loose unreachable objects stay untouched!
+ LooseObjects: []git.ObjectID{
+ setup.Commits.Orphan.OID,
+ setup.Commits.Unreachable.OID,
+ },
+ Packfiles: map[string]*PackfileState{
+ hash(t,
+ "pack-a81cd79eb9f32ce0afbdc15dec51c7141029e54c",
+ "pack-ce649b013f4191c500c7c4de5fe407120314c83354944e5639bf1a33a2c94110",
+ ): {
+ Objects: defaultReachableObjects,
+ HasBitmap: false,
+ HasReverseIndex: true,
+ },
+ },
+ HasMultiPackIndex: true,
+ HasCommitGraphs: true,
+ },
+ },
+ },
+ Directory: generateDirectoryState(setup.Config, []*walDirectoryState{
+ {
+ lsn: 1,
+ includePackfiles: []string{hash(t,
+ "pack-a81cd79eb9f32ce0afbdc15dec51c7141029e54c",
+ "pack-ce649b013f4191c500c7c4de5fe407120314c83354944e5639bf1a33a2c94110",
+ )},
+ includeMultiIndexes: []string{hash(t,
+ "multi-pack-index-f881063fbb14e481b5be5619df02c9874dbe5d3b",
+ "multi-pack-index-67d1f13534c85393277dc006444eee9b6670b6f1554faa43e051fa9402efa3a8",
+ )},
+ includeCommitGraphs: []string{hash(t,
+ "graph-5eddc89b8217451ecd51182f91ddf6f58b20f0f7",
+ "graph-d7a6f93863d026b02376bf869ab4fa23a7cd6bdbc013543741352b574cc19606",
+ )},
+ },
+ }),
+ },
+ },
+ {
+ desc: "run repacking without bitmap and multi-pack-index",
+ customSetup: customSetup,
+ steps: steps{
+ StartManager{
+ ModifyStorage: func(tb testing.TB, cfg config.Cfg, storagePath string) {
+ repoPath := filepath.Join(storagePath, setup.RelativePath)
+ gittest.Exec(tb, cfg, "-c", "repack.writeBitmaps=false", "-C", repoPath, "repack", "-ad")
+ },
+ },
+ Begin{
+ TransactionID: 1,
+ RelativePath: setup.RelativePath,
+ },
+ RunRepack{
+ TransactionID: 1,
+ Config: housekeeping.RepackObjectsConfig{
+ Strategy: housekeeping.RepackObjectsStrategyGeometric,
+ WriteBitmap: false,
+ WriteMultiPackIndex: false,
+ },
+ },
+ Commit{
+ TransactionID: 1,
+ },
+ },
+ expectedState: StateAssertion{
+ Database: DatabaseState{
+ string(keyAppliedLSN(setup.PartitionID)): LSN(1).toProto(),
+ },
+ Repositories: RepositoryStates{
+ setup.RelativePath: {
+ DefaultBranch: "refs/heads/main",
+ References: &ReferencesState{LooseReferences: defaultReferences},
+ Packfiles: &PackfilesState{
+ LooseObjects: nil,
+ Packfiles: map[string]*PackfileState{
+ hash(t,
+ "pack-5f624d294fda1b8df86f1c286c6a66757b44126e",
+ "pack-c57ed22f16c0a35f04febe26eac0fe8974b2b4ab3469d1ece0bc2983588ad44e",
+ ): {
+ Objects: defaultReachableObjects,
+ HasBitmap: false,
+ HasReverseIndex: true,
+ },
+ hash(t,
+ "pack-f20a6e68adae9088db85f994838091d53fbaf608",
+ "pack-aa6d40f5f019492a7cc11291ab68666ae7ac2a23e66762905581c44523bb12bd",
+ ): {
+ Objects: []git.ObjectID{
+ setup.Commits.Orphan.OID,
+ setup.Commits.Unreachable.OID,
+ },
+ HasBitmap: false,
+ HasReverseIndex: true,
+ },
+ },
+ HasMultiPackIndex: false,
+ HasCommitGraphs: false,
+ },
+ },
+ },
+ Directory: generateDirectoryState(setup.Config, []*walDirectoryState{
+ {
+ lsn: 1,
+ includePackfiles: []string{hash(t,
+ "pack-f20a6e68adae9088db85f994838091d53fbaf608",
+ "pack-aa6d40f5f019492a7cc11291ab68666ae7ac2a23e66762905581c44523bb12bd",
+ )},
+ },
+ }),
+ },
+ },
+ {
+ desc: "run repacking (Geometric) on a repository having existing commit-graphs",
+ customSetup: customSetup,
+ steps: steps{
+ StartManager{
+ ModifyStorage: func(tb testing.TB, cfg config.Cfg, storagePath string) {
+ repoPath := filepath.Join(storagePath, setup.RelativePath)
+ gittest.Exec(tb, cfg, "-c", "repack.writeBitmaps=false", "-C", repoPath, "repack", "-ad")
+ gittest.Exec(tb, cfg, "-C", repoPath, "commit-graph", "write", "--reachable", "--changed-paths", "--size-multiple=4", "--split=replace")
+ },
+ },
+ Begin{
+ TransactionID: 1,
+ RelativePath: setup.RelativePath,
+ },
+ RunRepack{
+ TransactionID: 1,
+ Config: housekeeping.RepackObjectsConfig{
+ Strategy: housekeeping.RepackObjectsStrategyGeometric,
+ },
+ },
+ Commit{
+ TransactionID: 1,
+ },
+ },
+ expectedState: StateAssertion{
+ Database: DatabaseState{
+ string(keyAppliedLSN(setup.PartitionID)): LSN(1).toProto(),
+ },
+ Repositories: RepositoryStates{
+ setup.RelativePath: {
+ DefaultBranch: "refs/heads/main",
+ References: &ReferencesState{LooseReferences: defaultReferences},
+ Packfiles: &PackfilesState{
+ LooseObjects: nil,
+ Packfiles: map[string]*PackfileState{
+ hash(t,
+ "pack-5f624d294fda1b8df86f1c286c6a66757b44126e",
+ "pack-c57ed22f16c0a35f04febe26eac0fe8974b2b4ab3469d1ece0bc2983588ad44e",
+ ): {
+ Objects: defaultReachableObjects,
+ HasBitmap: false,
+ HasReverseIndex: true,
+ },
+ hash(t,
+ "pack-f20a6e68adae9088db85f994838091d53fbaf608",
+ "pack-aa6d40f5f019492a7cc11291ab68666ae7ac2a23e66762905581c44523bb12bd",
+ ): {
+ Objects: []git.ObjectID{
+ setup.Commits.Orphan.OID,
+ setup.Commits.Unreachable.OID,
+ },
+ HasBitmap: false,
+ HasReverseIndex: true,
+ },
+ },
+ HasMultiPackIndex: false,
+ HasCommitGraphs: true,
+ },
+ },
+ },
+ Directory: generateDirectoryState(setup.Config, []*walDirectoryState{
+ {
+ lsn: 1,
+ includePackfiles: []string{hash(t,
+ "pack-f20a6e68adae9088db85f994838091d53fbaf608",
+ "pack-aa6d40f5f019492a7cc11291ab68666ae7ac2a23e66762905581c44523bb12bd",
+ )},
+ },
+ }),
+ },
+ },
+ {
+ desc: "run repacking (FullWithUnreachable) on a repository having existing commit-graphs",
+ customSetup: customSetup,
+ steps: steps{
+ StartManager{
+ ModifyStorage: func(tb testing.TB, cfg config.Cfg, storagePath string) {
+ repoPath := filepath.Join(storagePath, setup.RelativePath)
+ gittest.Exec(tb, cfg, "-c", "repack.writeBitmaps=false", "-C", repoPath, "repack", "-ad")
+ gittest.Exec(tb, cfg, "-C", repoPath, "commit-graph", "write", "--reachable", "--changed-paths", "--size-multiple=4", "--split=replace")
+ },
+ },
+ Begin{
+ TransactionID: 1,
+ RelativePath: setup.RelativePath,
+ },
+ RunRepack{
+ TransactionID: 1,
+ Config: housekeeping.RepackObjectsConfig{
+ Strategy: housekeeping.RepackObjectsStrategyFullWithUnreachable,
+ },
+ },
+ Commit{
+ TransactionID: 1,
+ },
+ },
+ expectedState: StateAssertion{
+ Database: DatabaseState{
+ string(keyAppliedLSN(setup.PartitionID)): LSN(1).toProto(),
+ },
+ Repositories: RepositoryStates{
+ setup.RelativePath: {
+ DefaultBranch: "refs/heads/main",
+ References: &ReferencesState{LooseReferences: defaultReferences},
+ Packfiles: &PackfilesState{
+ LooseObjects: nil,
+ Packfiles: map[string]*PackfileState{
+ hash(t,
+ "pack-689b1fa746246c50a8b0f3469a06c7ae68af9926",
+ "pack-3506da99c69e8bbb4e3122636a486ffcc3506f08d24426823a2a394a7fb16b94",
+ ): {
+ Objects: append(defaultReachableObjects,
+ setup.Commits.Orphan.OID,
+ setup.Commits.Unreachable.OID,
+ ),
+ HasBitmap: false,
+ HasReverseIndex: true,
+ },
+ },
+ HasMultiPackIndex: false,
+ HasCommitGraphs: true,
+ },
+ },
+ },
+ Directory: generateDirectoryState(setup.Config, []*walDirectoryState{
+ {
+ lsn: 1,
+ includePackfiles: []string{hash(t,
+ "pack-689b1fa746246c50a8b0f3469a06c7ae68af9926",
+ "pack-3506da99c69e8bbb4e3122636a486ffcc3506f08d24426823a2a394a7fb16b94",
+ )},
+ },
+ }),
+ },
+ },
+ {
+ desc: "run repacking (FullWithCruft) on a repository having existing commit-graphs",
+ customSetup: customSetup,
+ steps: steps{
+ StartManager{
+ ModifyStorage: func(tb testing.TB, cfg config.Cfg, storagePath string) {
+ repoPath := filepath.Join(storagePath, setup.RelativePath)
+ gittest.Exec(tb, cfg, "-C", repoPath, "repack", "-adl")
+ gittest.Exec(tb, cfg, "-C", repoPath, "commit-graph", "write", "--reachable", "--changed-paths", "--size-multiple=4", "--split=replace")
+ },
+ },
+ Begin{
+ TransactionID: 1,
+ RelativePath: setup.RelativePath,
+ },
+ RunRepack{
+ TransactionID: 1,
+ Config: housekeeping.RepackObjectsConfig{
+ Strategy: housekeeping.RepackObjectsStrategyFullWithCruft,
+ WriteBitmap: true,
+ WriteMultiPackIndex: true,
+ },
+ },
+ Commit{
+ TransactionID: 1,
+ },
+ },
+ expectedState: StateAssertion{
+ Database: DatabaseState{
+ string(keyAppliedLSN(setup.PartitionID)): LSN(1).toProto(),
+ },
+ Repositories: RepositoryStates{
+ setup.RelativePath: {
+ DefaultBranch: "refs/heads/main",
+ References: &ReferencesState{LooseReferences: defaultReferences},
+ Packfiles: &PackfilesState{
+ // Interestingly, loose unreachable objects stay untouched!
+ LooseObjects: []git.ObjectID{
+ setup.Commits.Orphan.OID,
+ setup.Commits.Unreachable.OID,
+ },
+ Packfiles: map[string]*PackfileState{
+ hash(t,
+ "pack-a81cd79eb9f32ce0afbdc15dec51c7141029e54c",
+ "pack-ce649b013f4191c500c7c4de5fe407120314c83354944e5639bf1a33a2c94110",
+ ): {
+ Objects: defaultReachableObjects,
+ HasBitmap: false,
+ HasReverseIndex: true,
+ },
+ },
+ HasMultiPackIndex: true,
+ HasCommitGraphs: true,
+ },
+ },
+ },
+ Directory: generateDirectoryState(setup.Config, []*walDirectoryState{
+ {
+ lsn: 1,
+ includePackfiles: []string{hash(t,
+ "pack-a81cd79eb9f32ce0afbdc15dec51c7141029e54c",
+ "pack-ce649b013f4191c500c7c4de5fe407120314c83354944e5639bf1a33a2c94110",
+ )},
+ includeMultiIndexes: []string{hash(t,
+ "multi-pack-index-f881063fbb14e481b5be5619df02c9874dbe5d3b",
+ "multi-pack-index-67d1f13534c85393277dc006444eee9b6670b6f1554faa43e051fa9402efa3a8",
+ )},
+ includeCommitGraphs: []string{hash(t,
+ "graph-5eddc89b8217451ecd51182f91ddf6f58b20f0f7",
+ "graph-d7a6f93863d026b02376bf869ab4fa23a7cd6bdbc013543741352b574cc19606",
+ )},
+ },
+ }),
+ },
+ },
+ {
+ desc: "run repacking twice with the same setting",
+ customSetup: customSetup,
+ steps: steps{
+ StartManager{
+ ModifyStorage: func(tb testing.TB, cfg config.Cfg, storagePath string) {
+ repoPath := filepath.Join(storagePath, setup.RelativePath)
+ gittest.Exec(tb, cfg, "-C", repoPath, "repack", "-adl")
+ },
+ },
+ Begin{
+ TransactionID: 1,
+ RelativePath: setup.RelativePath,
+ },
+ RunRepack{
+ TransactionID: 1,
+ Config: housekeeping.RepackObjectsConfig{
+ Strategy: housekeeping.RepackObjectsStrategyFullWithUnreachable,
+ WriteBitmap: true,
+ WriteMultiPackIndex: true,
+ },
+ },
+ Commit{
+ TransactionID: 1,
+ },
+ Begin{
+ TransactionID: 2,
+ RelativePath: setup.RelativePath,
+ ExpectedSnapshotLSN: 1,
+ },
+ RunRepack{
+ TransactionID: 2,
+ Config: housekeeping.RepackObjectsConfig{
+ Strategy: housekeeping.RepackObjectsStrategyFullWithUnreachable,
+ WriteBitmap: true,
+ WriteMultiPackIndex: true,
+ },
+ },
+ Commit{
+ TransactionID: 2,
+ },
+ },
+ expectedState: StateAssertion{
+ Database: DatabaseState{
+ string(keyAppliedLSN(setup.PartitionID)): LSN(2).toProto(),
+ },
+ Repositories: RepositoryStates{
+ setup.RelativePath: {
+ DefaultBranch: "refs/heads/main",
+ References: &ReferencesState{LooseReferences: defaultReferences},
+ Packfiles: &PackfilesState{
+ LooseObjects: nil,
+ Packfiles: map[string]*PackfileState{
+ hash(t,
+ "pack-689b1fa746246c50a8b0f3469a06c7ae68af9926",
+ "pack-3506da99c69e8bbb4e3122636a486ffcc3506f08d24426823a2a394a7fb16b94",
+ ): {
+ Objects: append(defaultReachableObjects,
+ setup.Commits.Orphan.OID,
+ setup.Commits.Unreachable.OID,
+ ),
+ HasBitmap: false,
+ HasReverseIndex: true,
+ },
+ },
+ HasMultiPackIndex: true,
+ HasCommitGraphs: false,
+ },
+ },
+ },
+ Directory: generateDirectoryState(setup.Config, []*walDirectoryState{
+ {
+ lsn: 1,
+ includePackfiles: []string{hash(t,
+ "pack-689b1fa746246c50a8b0f3469a06c7ae68af9926",
+ "pack-3506da99c69e8bbb4e3122636a486ffcc3506f08d24426823a2a394a7fb16b94",
+ )},
+ includeMultiIndexes: []string{hash(t,
+ "multi-pack-index-dece3dfef114aa668c61339e0d4eb081af62ce68",
+ "multi-pack-index-bf9ee4098624aeb3fae4990d943443f5759d6d63c8cca686b19fb48e3c6a6f25",
+ )},
+ },
+ {
+ lsn: 2,
+ },
+ }),
+ },
+ },
+ {
+ desc: "run repacking in the same transaction including other changes",
+ customSetup: customSetup,
+ steps: steps{
+ StartManager{},
+ Begin{
+ TransactionID: 1,
+ RelativePath: setup.RelativePath,
+ },
+ RunRepack{
+ TransactionID: 1,
+ Config: housekeeping.RepackObjectsConfig{
+ Strategy: housekeeping.RepackObjectsStrategyFullWithUnreachable,
+ WriteBitmap: true,
+ WriteMultiPackIndex: true,
+ },
+ },
+ Commit{
+ TransactionID: 1,
+ ReferenceUpdates: ReferenceUpdates{
+ "refs/heads/main": {OldOID: setup.Commits.First.OID, NewOID: setup.Commits.Second.OID},
+ },
+ ExpectedError: errHousekeepingConflictOtherUpdates,
+ },
+ },
+ expectedState: StateAssertion{
+ Database: DatabaseState{},
+ Repositories: RepositoryStates{
+ setup.RelativePath: {
+ DefaultBranch: "refs/heads/main",
+ References: &ReferencesState{LooseReferences: defaultReferences},
+ Packfiles: &PackfilesState{
+ LooseObjects: append(defaultReachableObjects, setup.Commits.Unreachable.OID, setup.Commits.Orphan.OID),
+ Packfiles: map[string]*PackfileState{},
+ HasMultiPackIndex: false,
+ HasCommitGraphs: false,
+ },
+ },
+ },
+ Directory: testhelper.DirectoryState{
+ "/": {Mode: fs.ModeDir | perm.PrivateDir},
+ "/wal": {Mode: fs.ModeDir | perm.PrivateDir},
+ },
+ },
+ },
+ }
+}
+
+// generateHousekeepingRepackingConcurrentTests returns a set of tests which run repacking before, after, or alongside
+// with other transactions.
+func generateHousekeepingRepackingConcurrentTests(t *testing.T, ctx context.Context, setup testTransactionSetup) []transactionTestCase {
+ return []transactionTestCase{
+ {
+ desc: "run repacking on an empty repository",
+ steps: steps{
+ Prune{},
+ StartManager{},
+ Begin{
+ TransactionID: 1,
+ RelativePath: setup.RelativePath,
+ },
+ RunRepack{
+ TransactionID: 1,
+ Config: housekeeping.RepackObjectsConfig{
+ Strategy: housekeeping.RepackObjectsStrategyFullWithCruft,
+ },
+ },
+ Commit{
+ TransactionID: 1,
+ },
+ },
+ expectedState: StateAssertion{
+ Database: DatabaseState{
+ string(keyAppliedLSN(setup.PartitionID)): LSN(1).toProto(),
+ },
+ Directory: generateDirectoryState(setup.Config, []*walDirectoryState{
+ {lsn: 1},
+ }),
+ Repositories: RepositoryStates{
+ setup.RelativePath: {
+ DefaultBranch: "refs/heads/main",
+ References: &ReferencesState{
+ LooseReferences: map[git.ReferenceName]git.ObjectID{},
+ },
+ Packfiles: &PackfilesState{
+ Packfiles: map[string]*PackfileState{},
+ HasCommitGraphs: false,
+ },
+ },
+ },
+ },
+ },
+ {
+ desc: "run repacking after some changes including both reachable and unreachable objects",
+ steps: steps{
+ Prune{},
+ StartManager{},
+ Begin{
+ TransactionID: 1,
+ RelativePath: setup.RelativePath,
+ },
+ Commit{
+ TransactionID: 1,
+ ReferenceUpdates: ReferenceUpdates{
+ "refs/heads/main": {OldOID: setup.ObjectHash.ZeroOID, NewOID: setup.Commits.Second.OID},
+ },
+ QuarantinedPacks: [][]byte{
+ setup.Commits.First.Pack,
+ setup.Commits.Second.Pack,
+ setup.Commits.Diverging.Pack, // This commit is not reachable
+ },
+ IncludeObjects: []git.ObjectID{setup.Commits.Diverging.OID},
+ },
+ Begin{
+ TransactionID: 2,
+ RelativePath: setup.RelativePath,
+ ExpectedSnapshotLSN: 1,
+ },
+ RunRepack{
+ TransactionID: 2,
+ Config: housekeeping.RepackObjectsConfig{
+ Strategy: housekeeping.RepackObjectsStrategyFullWithCruft,
+ },
+ },
+ Commit{
+ TransactionID: 2,
+ },
+ },
+ expectedState: StateAssertion{
+ Database: DatabaseState{
+ string(keyAppliedLSN(setup.PartitionID)): LSN(2).toProto(),
+ },
+ Directory: generateDirectoryState(setup.Config, []*walDirectoryState{
+ {
+ lsn: 1,
+ includeObjects: []git.ObjectID{
+ setup.ObjectHash.EmptyTreeOID,
+ setup.Commits.First.OID,
+ setup.Commits.Second.OID,
+ setup.Commits.Diverging.OID,
+ },
+ },
+ {
+ lsn: 2,
+ includePackfiles: []string{hash(t,
+ "pack-98be7bb46e97ddbe7e3093e0cc5bca60f37f9b09",
+ "pack-53630df54431a48f6d87f1bbe0d054327f8eb1964f813de1821d15bc5dcb1621",
+ )},
+ includeCommitGraphs: []string{hash(t,
+ "graph-8cd59940f998ecb90f7935b6b7adc8df46d9174e",
+ "graph-8cbbd20f75bc45c5337718fe4ab8498e1ce7524e87d6bd7fc9581bc08c119562",
+ )},
+ },
+ }),
+ Repositories: RepositoryStates{
+ setup.RelativePath: {
+ DefaultBranch: "refs/heads/main",
+ References: &ReferencesState{
+ LooseReferences: map[git.ReferenceName]git.ObjectID{
+ "refs/heads/main": setup.Commits.Second.OID,
+ },
+ },
+ Packfiles: &PackfilesState{
+ Packfiles: map[string]*PackfileState{
+ hash(t,
+ "pack-98be7bb46e97ddbe7e3093e0cc5bca60f37f9b09",
+ "pack-53630df54431a48f6d87f1bbe0d054327f8eb1964f813de1821d15bc5dcb1621",
+ ): {
+ // Diverging commit is gone.
+ Objects: []git.ObjectID{
+ gittest.DefaultObjectHash.EmptyTreeOID,
+ setup.Commits.First.OID,
+ setup.Commits.Second.OID,
+ },
+ HasReverseIndex: true,
+ },
+ },
+ HasCommitGraphs: true,
+ },
+ },
+ },
+ },
+ },
+ {
+ desc: "run repacking before another transaction that produce new packfiles",
+ steps: steps{
+ Prune{},
+ StartManager{},
+ Begin{
+ TransactionID: 1,
+ RelativePath: setup.RelativePath,
+ },
+ Commit{
+ TransactionID: 1,
+ ReferenceUpdates: ReferenceUpdates{
+ "refs/heads/main": {OldOID: setup.ObjectHash.ZeroOID, NewOID: setup.Commits.Second.OID},
+ },
+ QuarantinedPacks: [][]byte{
+ setup.Commits.First.Pack,
+ setup.Commits.Second.Pack,
+ setup.Commits.Diverging.Pack, // This commit is not reachable
+ },
+ IncludeObjects: []git.ObjectID{setup.Commits.Diverging.OID},
+ },
+ Begin{
+ TransactionID: 2,
+ RelativePath: setup.RelativePath,
+ ExpectedSnapshotLSN: 1,
+ },
+ RunRepack{
+ TransactionID: 2,
+ Config: housekeeping.RepackObjectsConfig{
+ Strategy: housekeeping.RepackObjectsStrategyFullWithCruft,
+ },
+ },
+ Commit{
+ TransactionID: 2,
+ },
+ Begin{
+ TransactionID: 3,
+ RelativePath: setup.RelativePath,
+ ExpectedSnapshotLSN: 2,
+ },
+ Commit{
+ TransactionID: 3,
+ ReferenceUpdates: ReferenceUpdates{
+ "refs/heads/main": {OldOID: setup.Commits.Second.OID, NewOID: setup.Commits.Third.OID},
+ },
+ QuarantinedPacks: [][]byte{
+ setup.Commits.Third.Pack,
+ },
+ },
+ },
+ expectedState: StateAssertion{
+ Database: DatabaseState{
+ string(keyAppliedLSN(setup.PartitionID)): LSN(3).toProto(),
+ },
+ Directory: generateDirectoryState(setup.Config, []*walDirectoryState{
+ {
+ lsn: 1,
+ includeObjects: []git.ObjectID{
+ setup.ObjectHash.EmptyTreeOID,
+ setup.Commits.First.OID,
+ setup.Commits.Second.OID,
+ setup.Commits.Diverging.OID,
+ },
+ },
+ {
+ lsn: 2,
+ includePackfiles: []string{hash(t,
+ "pack-98be7bb46e97ddbe7e3093e0cc5bca60f37f9b09",
+ "pack-53630df54431a48f6d87f1bbe0d054327f8eb1964f813de1821d15bc5dcb1621",
+ )},
+ includeCommitGraphs: []string{hash(t,
+ "graph-8cd59940f998ecb90f7935b6b7adc8df46d9174e",
+ "graph-8cbbd20f75bc45c5337718fe4ab8498e1ce7524e87d6bd7fc9581bc08c119562",
+ )},
+ },
+ {
+ lsn: 3,
+ includeObjects: []git.ObjectID{
+ setup.Commits.Third.OID,
+ },
+ },
+ }),
+ Repositories: RepositoryStates{
+ setup.RelativePath: {
+ DefaultBranch: "refs/heads/main",
+ References: &ReferencesState{
+ LooseReferences: map[git.ReferenceName]git.ObjectID{
+ "refs/heads/main": setup.Commits.Third.OID,
+ },
+ },
+ Packfiles: &PackfilesState{
+ Packfiles: map[string]*PackfileState{
+ hash(t,
+ "pack-98be7bb46e97ddbe7e3093e0cc5bca60f37f9b09",
+ "pack-53630df54431a48f6d87f1bbe0d054327f8eb1964f813de1821d15bc5dcb1621",
+ ): {
+ // Diverging commit is gone.
+ Objects: []git.ObjectID{
+ gittest.DefaultObjectHash.EmptyTreeOID,
+ setup.Commits.First.OID,
+ setup.Commits.Second.OID,
+ },
+ HasReverseIndex: true,
+ },
+ hash(t,
+ "pack-529ec37accbc126425efe69abdf91153411532a6",
+ "pack-895b4eade6c459f47a382a0d637ef1ce34a661c76f003c7d7a38a7420e3afc69",
+ ): {
+ Objects: []git.ObjectID{
+ setup.Commits.Third.OID,
+ },
+ HasReverseIndex: true,
+ },
+ },
+ HasCommitGraphs: true,
+ },
+ },
+ },
+ },
+ },
+ {
+ desc: "run repacking concurrently with another transaction that produce new packfiles",
+ steps: steps{
+ Prune{},
+ StartManager{},
+ Begin{
+ TransactionID: 1,
+ RelativePath: setup.RelativePath,
+ },
+ Commit{
+ TransactionID: 1,
+ ReferenceUpdates: ReferenceUpdates{
+ "refs/heads/main": {OldOID: setup.ObjectHash.ZeroOID, NewOID: setup.Commits.Second.OID},
+ },
+ QuarantinedPacks: [][]byte{
+ setup.Commits.First.Pack,
+ setup.Commits.Second.Pack,
+ setup.Commits.Diverging.Pack,
+ },
+ IncludeObjects: []git.ObjectID{setup.Commits.Diverging.OID},
+ },
+ Begin{
+ TransactionID: 2,
+ RelativePath: setup.RelativePath,
+ ExpectedSnapshotLSN: 1,
+ },
+ Begin{
+ TransactionID: 3,
+ RelativePath: setup.RelativePath,
+ ExpectedSnapshotLSN: 1,
+ },
+ RunRepack{
+ TransactionID: 2,
+ Config: housekeeping.RepackObjectsConfig{
+ Strategy: housekeeping.RepackObjectsStrategyFullWithCruft,
+ },
+ },
+ Commit{
+ TransactionID: 3,
+ ReferenceUpdates: ReferenceUpdates{
+ "refs/heads/main": {OldOID: setup.Commits.Second.OID, NewOID: setup.Commits.Third.OID},
+ },
+ QuarantinedPacks: [][]byte{
+ setup.Commits.Third.Pack,
+ },
+ },
+ Commit{
+ TransactionID: 2,
+ },
+ },
+ expectedState: StateAssertion{
+ Database: DatabaseState{
+ string(keyAppliedLSN(setup.PartitionID)): LSN(3).toProto(),
+ },
+ Directory: generateDirectoryState(setup.Config, []*walDirectoryState{
+ {
+ lsn: 1,
+ includeObjects: []git.ObjectID{
+ setup.ObjectHash.EmptyTreeOID,
+ setup.Commits.First.OID,
+ setup.Commits.Second.OID,
+ setup.Commits.Diverging.OID,
+ },
+ },
+ {
+ lsn: 2,
+ includeObjects: []git.ObjectID{
+ setup.Commits.Third.OID,
+ },
+ },
+ {
+ lsn: 3,
+ includePackfiles: []string{hash(t,
+ "pack-98be7bb46e97ddbe7e3093e0cc5bca60f37f9b09",
+ "pack-53630df54431a48f6d87f1bbe0d054327f8eb1964f813de1821d15bc5dcb1621",
+ )},
+ includeCommitGraphs: []string{hash(t,
+ "graph-8cd59940f998ecb90f7935b6b7adc8df46d9174e",
+ "graph-8cbbd20f75bc45c5337718fe4ab8498e1ce7524e87d6bd7fc9581bc08c119562",
+ )},
+ },
+ }),
+ Repositories: RepositoryStates{
+ setup.RelativePath: {
+ DefaultBranch: "refs/heads/main",
+ References: &ReferencesState{
+ LooseReferences: map[git.ReferenceName]git.ObjectID{
+ "refs/heads/main": setup.Commits.Third.OID,
+ },
+ },
+ Packfiles: &PackfilesState{
+ Packfiles: map[string]*PackfileState{
+ hash(t,
+ "pack-98be7bb46e97ddbe7e3093e0cc5bca60f37f9b09",
+ "pack-53630df54431a48f6d87f1bbe0d054327f8eb1964f813de1821d15bc5dcb1621",
+ ): {
+ Objects: []git.ObjectID{
+ gittest.DefaultObjectHash.EmptyTreeOID,
+ setup.Commits.First.OID,
+ setup.Commits.Second.OID,
+ },
+ HasReverseIndex: true,
+ },
+ hash(t,
+ "pack-529ec37accbc126425efe69abdf91153411532a6",
+ "pack-895b4eade6c459f47a382a0d637ef1ce34a661c76f003c7d7a38a7420e3afc69",
+ ): {
+ Objects: []git.ObjectID{
+ setup.Commits.Third.OID,
+ },
+ HasReverseIndex: true,
+ },
+ },
+ HasCommitGraphs: true,
+ },
+ },
+ },
+ },
+ },
+ {
+ desc: "run repacking concurrently with another transaction that points to a survived object",
+ steps: steps{
+ Prune{},
+ StartManager{},
+ Begin{
+ TransactionID: 1,
+ RelativePath: setup.RelativePath,
+ },
+ Commit{
+ TransactionID: 1,
+ ReferenceUpdates: ReferenceUpdates{
+ "refs/heads/main": {OldOID: setup.ObjectHash.ZeroOID, NewOID: setup.Commits.Second.OID},
+ },
+ QuarantinedPacks: [][]byte{
+ setup.Commits.First.Pack,
+ setup.Commits.Second.Pack,
+ setup.Commits.Diverging.Pack,
+ },
+ IncludeObjects: []git.ObjectID{setup.Commits.Diverging.OID},
+ },
+ Begin{
+ TransactionID: 2,
+ RelativePath: setup.RelativePath,
+ ExpectedSnapshotLSN: 1,
+ },
+ Begin{
+ TransactionID: 3,
+ RelativePath: setup.RelativePath,
+ ExpectedSnapshotLSN: 1,
+ },
+ RunRepack{
+ TransactionID: 2,
+ Config: housekeeping.RepackObjectsConfig{
+ Strategy: housekeeping.RepackObjectsStrategyFullWithCruft,
+ },
+ },
+ Commit{
+ TransactionID: 3,
+ ReferenceUpdates: ReferenceUpdates{
+ "refs/heads/main": {OldOID: setup.Commits.Second.OID, NewOID: setup.Commits.First.OID},
+ },
+ },
+ Commit{
+ TransactionID: 2,
+ },
+ },
+ expectedState: StateAssertion{
+ Database: DatabaseState{
+ string(keyAppliedLSN(setup.PartitionID)): LSN(3).toProto(),
+ },
+ Directory: generateDirectoryState(setup.Config, []*walDirectoryState{
+ {
+ lsn: 1,
+ includeObjects: []git.ObjectID{
+ setup.ObjectHash.EmptyTreeOID,
+ setup.Commits.First.OID,
+ setup.Commits.Second.OID,
+ setup.Commits.Diverging.OID,
+ },
+ },
+ {
+ lsn: 3,
+ includePackfiles: []string{hash(t,
+ "pack-98be7bb46e97ddbe7e3093e0cc5bca60f37f9b09",
+ "pack-53630df54431a48f6d87f1bbe0d054327f8eb1964f813de1821d15bc5dcb1621",
+ )},
+ includeCommitGraphs: []string{hash(t,
+ "graph-8cd59940f998ecb90f7935b6b7adc8df46d9174e",
+ "graph-8cbbd20f75bc45c5337718fe4ab8498e1ce7524e87d6bd7fc9581bc08c119562",
+ )},
+ },
+ }),
+ Repositories: RepositoryStates{
+ setup.RelativePath: {
+ DefaultBranch: "refs/heads/main",
+ References: &ReferencesState{
+ LooseReferences: map[git.ReferenceName]git.ObjectID{
+ "refs/heads/main": setup.Commits.First.OID,
+ },
+ },
+ Packfiles: &PackfilesState{
+ Packfiles: map[string]*PackfileState{
+ hash(t,
+ "pack-98be7bb46e97ddbe7e3093e0cc5bca60f37f9b09",
+ "pack-53630df54431a48f6d87f1bbe0d054327f8eb1964f813de1821d15bc5dcb1621",
+ ): {
+ Objects: []git.ObjectID{
+ gittest.DefaultObjectHash.EmptyTreeOID,
+ setup.Commits.First.OID,
+ setup.Commits.Second.OID,
+ },
+ HasReverseIndex: true,
+ },
+ },
+ HasCommitGraphs: true,
+ },
+ },
+ },
+ },
+ },
+ {
+ desc: "run repacking that spans through multiple transactions",
+ steps: steps{
+ Prune{},
+ StartManager{},
+ Begin{
+ TransactionID: 1,
+ RelativePath: setup.RelativePath,
+ },
+ Commit{
+ TransactionID: 1,
+ ReferenceUpdates: ReferenceUpdates{
+ "refs/heads/main": {OldOID: setup.ObjectHash.ZeroOID, NewOID: setup.Commits.First.OID},
+ },
+ QuarantinedPacks: [][]byte{
+ setup.Commits.First.Pack,
+ },
+ },
+ Begin{
+ TransactionID: 2,
+ RelativePath: setup.RelativePath,
+ ExpectedSnapshotLSN: 1,
+ },
+ RunRepack{
+ TransactionID: 2,
+ Config: housekeeping.RepackObjectsConfig{
+ Strategy: housekeeping.RepackObjectsStrategyFullWithCruft,
+ },
+ },
+ Begin{
+ TransactionID: 3,
+ RelativePath: setup.RelativePath,
+ ExpectedSnapshotLSN: 1,
+ },
+ Commit{
+ TransactionID: 3,
+ ReferenceUpdates: ReferenceUpdates{
+ "refs/heads/main": {OldOID: setup.Commits.First.OID, NewOID: setup.Commits.Second.OID},
+ },
+ QuarantinedPacks: [][]byte{
+ setup.Commits.Second.Pack,
+ },
+ },
+ Begin{
+ TransactionID: 4,
+ RelativePath: setup.RelativePath,
+ ExpectedSnapshotLSN: 2,
+ },
+ Commit{
+ TransactionID: 4,
+ ReferenceUpdates: ReferenceUpdates{
+ "refs/heads/main": {OldOID: setup.Commits.Second.OID, NewOID: setup.Commits.Third.OID},
+ },
+ QuarantinedPacks: [][]byte{
+ setup.Commits.Third.Pack,
+ },
+ },
+ Begin{
+ TransactionID: 5,
+ RelativePath: setup.RelativePath,
+ ExpectedSnapshotLSN: 3,
+ },
+ Commit{
+ TransactionID: 5,
+ ReferenceUpdates: ReferenceUpdates{
+ "refs/heads/main": {OldOID: setup.Commits.Third.OID, NewOID: setup.Commits.Diverging.OID},
+ },
+ QuarantinedPacks: [][]byte{
+ setup.Commits.Diverging.Pack,
+ },
+ },
+ Commit{
+ TransactionID: 2,
+ },
+ },
+ expectedState: StateAssertion{
+ Database: DatabaseState{
+ string(keyAppliedLSN(setup.PartitionID)): LSN(5).toProto(),
+ },
+ Directory: generateDirectoryState(setup.Config, []*walDirectoryState{
+ {
+ lsn: 1,
+ includeObjects: []git.ObjectID{
+ setup.ObjectHash.EmptyTreeOID,
+ setup.Commits.First.OID,
+ },
+ },
+ {
+ lsn: 2,
+ includeObjects: []git.ObjectID{
+ setup.Commits.Second.OID,
+ },
+ },
+ {
+ lsn: 3,
+ includeObjects: []git.ObjectID{
+ setup.Commits.Third.OID,
+ },
+ },
+ {
+ lsn: 4,
+ includeObjects: []git.ObjectID{
+ setup.Commits.Diverging.OID,
+ },
+ },
+ {
+ lsn: 5,
+ },
+ }),
+ Repositories: RepositoryStates{
+ setup.RelativePath: {
+ DefaultBranch: "refs/heads/main",
+ References: &ReferencesState{
+ LooseReferences: map[git.ReferenceName]git.ObjectID{
+ "refs/heads/main": setup.Commits.Diverging.OID,
+ },
+ },
+ Packfiles: &PackfilesState{
+ Packfiles: map[string]*PackfileState{
+ hash(t,
+ "pack-452292f7e0c6bcca1b42c53aaac4537416b5dbb9",
+ "pack-735ad245db57a16c41525c9101c42594d090c7021b51aa12d9104a4eea4223c5",
+ ): {
+ Objects: []git.ObjectID{
+ setup.ObjectHash.EmptyTreeOID,
+ setup.Commits.First.OID,
+ },
+ HasReverseIndex: true,
+ },
+ hash(t,
+ "pack-407172c9edc9b3cef89f8fb341262155b6b401ae",
+ "pack-8c9a31ee3c6493a1883f96fe629925b6f94c00d810eb6c80d5e2502fba646d3a",
+ ): {
+ Objects: []git.ObjectID{
+ setup.Commits.Diverging.OID,
+ },
+ HasReverseIndex: true,
+ },
+ hash(t,
+ "pack-529ec37accbc126425efe69abdf91153411532a6",
+ "pack-895b4eade6c459f47a382a0d637ef1ce34a661c76f003c7d7a38a7420e3afc69",
+ ): {
+ Objects: []git.ObjectID{
+ setup.Commits.Third.OID,
+ },
+ HasReverseIndex: true,
+ },
+ hash(t,
+ "pack-5a422f6c469963ffa026bf15cfd151751fba6e5f",
+ "pack-3ed0b733b17d82f87a350b856f7fd6d6a781d85c5d8d36fad64b459124444f11",
+ ): {
+ Objects: []git.ObjectID{
+ setup.Commits.Second.OID,
+ },
+ HasReverseIndex: true,
+ },
+ },
+ },
+ },
+ },
+ },
+ },
+ {
+ desc: "run repacking (FullWithUnreachable) concurrently with another transaction pointing new reference to packed objects",
+ steps: steps{
+ Prune{},
+ StartManager{},
+ Begin{
+ TransactionID: 1,
+ RelativePath: setup.RelativePath,
+ },
+ Commit{
+ TransactionID: 1,
+ ReferenceUpdates: ReferenceUpdates{
+ "refs/heads/main": {OldOID: setup.ObjectHash.ZeroOID, NewOID: setup.Commits.First.OID},
+ "refs/heads/branch": {OldOID: setup.ObjectHash.ZeroOID, NewOID: setup.Commits.Second.OID},
+ },
+ QuarantinedPacks: [][]byte{
+ setup.Commits.First.Pack,
+ setup.Commits.Second.Pack,
+ },
+ },
+ Begin{
+ TransactionID: 2,
+ RelativePath: setup.RelativePath,
+ ExpectedSnapshotLSN: 1,
+ },
+ Commit{
+ TransactionID: 2,
+ ReferenceUpdates: ReferenceUpdates{
+ "refs/heads/branch": {OldOID: setup.Commits.Second.OID, NewOID: setup.ObjectHash.ZeroOID},
+ },
+ },
+ Begin{
+ TransactionID: 3,
+ RelativePath: setup.RelativePath,
+ ExpectedSnapshotLSN: 2,
+ },
+ Begin{
+ TransactionID: 4,
+ RelativePath: setup.RelativePath,
+ ExpectedSnapshotLSN: 2,
+ },
+ RunRepack{
+ TransactionID: 3,
+ Config: housekeeping.RepackObjectsConfig{
+ Strategy: housekeeping.RepackObjectsStrategyFullWithUnreachable,
+ },
+ },
+ Commit{
+ TransactionID: 4,
+ ReferenceUpdates: ReferenceUpdates{
+ "refs/heads/main": {OldOID: setup.Commits.First.OID, NewOID: setup.Commits.Second.OID},
+ },
+ },
+ Commit{
+ TransactionID: 3,
+ },
+ },
+ expectedState: StateAssertion{
+ Database: DatabaseState{
+ string(keyAppliedLSN(setup.PartitionID)): LSN(4).toProto(),
+ },
+ Directory: generateDirectoryState(setup.Config, []*walDirectoryState{
+ {
+ lsn: 1,
+ includeObjects: []git.ObjectID{
+ setup.ObjectHash.EmptyTreeOID,
+ setup.Commits.First.OID,
+ setup.Commits.Second.OID,
+ },
+ },
+ {
+ lsn: 4,
+ includePackfiles: []string{hash(t,
+ "pack-df5e3e230b167b4ce31a30f389e0f1908ae40f2b",
+ "pack-6a4d9d6b54438754effb555adec435cd9031a01cba7515bdf8b73a0e2714c6ff",
+ )},
+ },
+ }),
+ Repositories: RepositoryStates{
+ setup.RelativePath: {
+ DefaultBranch: "refs/heads/main",
+ References: &ReferencesState{
+ LooseReferences: map[git.ReferenceName]git.ObjectID{
+ "refs/heads/main": setup.Commits.Second.OID,
+ },
+ },
+ Packfiles: &PackfilesState{
+ Packfiles: map[string]*PackfileState{
+ hash(t,
+ "pack-df5e3e230b167b4ce31a30f389e0f1908ae40f2b",
+ "pack-6a4d9d6b54438754effb555adec435cd9031a01cba7515bdf8b73a0e2714c6ff",
+ ): {
+ Objects: []git.ObjectID{
+ gittest.DefaultObjectHash.EmptyTreeOID,
+ setup.Commits.First.OID,
+ setup.Commits.Second.OID,
+ },
+ HasReverseIndex: true,
+ },
+ },
+ HasCommitGraphs: false,
+ },
+ },
+ },
+ },
+ },
+ {
+ desc: "run repacking (Geometric) concurrently with another transaction pointing new reference to packed objects",
+ steps: steps{
+ Prune{},
+ StartManager{},
+ Begin{
+ TransactionID: 1,
+ RelativePath: setup.RelativePath,
+ },
+ Commit{
+ TransactionID: 1,
+ ReferenceUpdates: ReferenceUpdates{
+ "refs/heads/main": {OldOID: setup.ObjectHash.ZeroOID, NewOID: setup.Commits.First.OID},
+ "refs/heads/branch": {OldOID: setup.ObjectHash.ZeroOID, NewOID: setup.Commits.Second.OID},
+ },
+ QuarantinedPacks: [][]byte{
+ setup.Commits.First.Pack,
+ setup.Commits.Second.Pack,
+ },
+ },
+ Begin{
+ TransactionID: 2,
+ RelativePath: setup.RelativePath,
+ ExpectedSnapshotLSN: 1,
+ },
+ Commit{
+ TransactionID: 2,
+ ReferenceUpdates: ReferenceUpdates{
+ "refs/heads/branch": {OldOID: setup.Commits.Second.OID, NewOID: setup.ObjectHash.ZeroOID},
+ },
+ },
+ Begin{
+ TransactionID: 3,
+ RelativePath: setup.RelativePath,
+ ExpectedSnapshotLSN: 2,
+ },
+ Begin{
+ TransactionID: 4,
+ RelativePath: setup.RelativePath,
+ ExpectedSnapshotLSN: 2,
+ },
+ RunRepack{
+ TransactionID: 3,
+ Config: housekeeping.RepackObjectsConfig{
+ Strategy: housekeeping.RepackObjectsStrategyGeometric,
+ },
+ },
+ Commit{
+ TransactionID: 4,
+ ReferenceUpdates: ReferenceUpdates{
+ "refs/heads/main": {OldOID: setup.Commits.First.OID, NewOID: setup.Commits.Second.OID},
+ },
+ },
+ Commit{
+ TransactionID: 3,
+ },
+ },
+ expectedState: StateAssertion{
+ Database: DatabaseState{
+ string(keyAppliedLSN(setup.PartitionID)): LSN(4).toProto(),
+ },
+ Directory: generateDirectoryState(setup.Config, []*walDirectoryState{
+ {
+ lsn: 1,
+ includeObjects: []git.ObjectID{
+ setup.ObjectHash.EmptyTreeOID,
+ setup.Commits.First.OID,
+ setup.Commits.Second.OID,
+ },
+ },
+ {
+ // No new packfiles
+ lsn: 4,
+ },
+ }),
+ Repositories: RepositoryStates{
+ setup.RelativePath: {
+ DefaultBranch: "refs/heads/main",
+ References: &ReferencesState{
+ LooseReferences: map[git.ReferenceName]git.ObjectID{
+ "refs/heads/main": setup.Commits.Second.OID,
+ },
+ },
+ Packfiles: &PackfilesState{
+ Packfiles: map[string]*PackfileState{
+ hash(t,
+ "pack-4274682fcb6a4dbb1a59ba7dd8577402e61ccbd2",
+ "pack-8ebabff3c37210ed37c4343255992f62a2ce113f7fb11f757de3bca157379d40",
+ ): {
+ Objects: []git.ObjectID{
+ gittest.DefaultObjectHash.EmptyTreeOID,
+ setup.Commits.First.OID,
+ setup.Commits.Second.OID,
+ },
+ HasReverseIndex: true,
+ },
+ },
+ HasCommitGraphs: false,
+ },
+ },
+ },
+ },
+ },
+ {
+ desc: "run repacking (FullWithCruft) concurrently with another transaction pointing new reference to pruned objects",
+ steps: steps{
+ Prune{},
+ StartManager{},
+ Begin{
+ TransactionID: 1,
+ RelativePath: setup.RelativePath,
+ },
+ Commit{
+ TransactionID: 1,
+ ReferenceUpdates: ReferenceUpdates{
+ "refs/heads/main": {OldOID: setup.ObjectHash.ZeroOID, NewOID: setup.Commits.First.OID},
+ "refs/heads/branch": {OldOID: setup.ObjectHash.ZeroOID, NewOID: setup.Commits.Second.OID},
+ },
+ QuarantinedPacks: [][]byte{
+ setup.Commits.First.Pack,
+ setup.Commits.Second.Pack,
+ },
+ },
+ Begin{
+ TransactionID: 2,
+ RelativePath: setup.RelativePath,
+ ExpectedSnapshotLSN: 1,
+ },
+ Commit{
+ TransactionID: 2,
+ ReferenceUpdates: ReferenceUpdates{
+ "refs/heads/branch": {OldOID: setup.Commits.Second.OID, NewOID: setup.ObjectHash.ZeroOID},
+ },
+ },
+ Begin{
+ TransactionID: 3,
+ RelativePath: setup.RelativePath,
+ ExpectedSnapshotLSN: 2,
+ },
+ Begin{
+ TransactionID: 4,
+ RelativePath: setup.RelativePath,
+ ExpectedSnapshotLSN: 2,
+ },
+ RunRepack{
+ TransactionID: 3,
+ Config: housekeeping.RepackObjectsConfig{
+ Strategy: housekeeping.RepackObjectsStrategyFullWithCruft,
+ },
+ },
+ Commit{
+ TransactionID: 4,
+ ReferenceUpdates: ReferenceUpdates{
+ "refs/heads/main": {OldOID: setup.Commits.First.OID, NewOID: setup.Commits.Second.OID},
+ },
+ },
+ Commit{
+ TransactionID: 3,
+ ExpectedError: errRepackConflictPrunedObject,
+ },
+ },
+ expectedState: StateAssertion{
+ Database: DatabaseState{
+ string(keyAppliedLSN(setup.PartitionID)): LSN(3).toProto(),
+ },
+ Directory: generateDirectoryState(setup.Config, []*walDirectoryState{
+ {
+ lsn: 1,
+ includeObjects: []git.ObjectID{
+ setup.ObjectHash.EmptyTreeOID,
+ setup.Commits.First.OID,
+ setup.Commits.Second.OID,
+ },
+ },
+ }),
+ Repositories: RepositoryStates{
+ setup.RelativePath: {
+ DefaultBranch: "refs/heads/main",
+ References: &ReferencesState{
+ LooseReferences: map[git.ReferenceName]git.ObjectID{
+ "refs/heads/main": setup.Commits.Second.OID,
+ },
+ },
+ Packfiles: &PackfilesState{
+ Packfiles: map[string]*PackfileState{
+ hash(t,
+ "pack-4274682fcb6a4dbb1a59ba7dd8577402e61ccbd2",
+ "pack-8ebabff3c37210ed37c4343255992f62a2ce113f7fb11f757de3bca157379d40",
+ ): {
+ Objects: []git.ObjectID{
+ gittest.DefaultObjectHash.EmptyTreeOID,
+ setup.Commits.First.OID,
+ setup.Commits.Second.OID,
+ },
+ HasReverseIndex: true,
+ },
+ },
+ HasCommitGraphs: false,
+ },
+ },
+ },
+ },
+ },
+ {
+ desc: "run repacking (FullWithUnreachable) on an alternate member",
+ steps: steps{
+ RemoveRepository{},
+ StartManager{},
+ Begin{
+ TransactionID: 1,
+ RelativePath: "pool",
+ },
+ CreateRepository{
+ TransactionID: 1,
+ References: map[git.ReferenceName]git.ObjectID{
+ "refs/heads/main": setup.Commits.First.OID,
+ },
+ Packs: [][]byte{setup.Commits.First.Pack},
+ },
+ Commit{
+ TransactionID: 1,
+ },
+ Begin{
+ TransactionID: 2,
+ RelativePath: "member",
+ ExpectedSnapshotLSN: 1,
+ },
+ CreateRepository{
+ TransactionID: 2,
+ Alternate: "../../pool/objects",
+ },
+ Commit{
+ TransactionID: 2,
+ },
+ Begin{
+ TransactionID: 3,
+ RelativePath: "member",
+ ExpectedSnapshotLSN: 2,
+ },
+ Commit{
+ TransactionID: 3,
+ ReferenceUpdates: ReferenceUpdates{
+ "refs/heads/branch": {OldOID: setup.ObjectHash.ZeroOID, NewOID: setup.Commits.Third.OID},
+ },
+ QuarantinedPacks: [][]byte{
+ setup.Commits.Second.Pack,
+ setup.Commits.Third.Pack,
+ },
+ },
+ Begin{
+ TransactionID: 4,
+ RelativePath: "member",
+ ExpectedSnapshotLSN: 3,
+ },
+ Commit{
+ TransactionID: 4,
+ ReferenceUpdates: ReferenceUpdates{
+ "refs/heads/branch": {OldOID: setup.Commits.Third.OID, NewOID: setup.Commits.Second.OID},
+ },
+ },
+ Begin{
+ TransactionID: 5,
+ RelativePath: "pool",
+ ExpectedSnapshotLSN: 4,
+ },
+ Commit{
+ TransactionID: 5,
+ QuarantinedPacks: [][]byte{
+ setup.Commits.Second.Pack,
+ },
+ IncludeObjects: []git.ObjectID{setup.Commits.Second.OID},
+ },
+ Begin{
+ TransactionID: 6,
+ RelativePath: "member",
+ ExpectedSnapshotLSN: 5,
+ },
+ RunRepack{
+ TransactionID: 6,
+ Config: housekeeping.RepackObjectsConfig{
+ Strategy: housekeeping.RepackObjectsStrategyFullWithUnreachable,
+ },
+ },
+ Commit{
+ TransactionID: 6,
+ },
+ },
+ expectedState: StateAssertion{
+ Database: DatabaseState{
+ string(keyAppliedLSN(setup.PartitionID)): LSN(6).toProto(),
+ },
+ Repositories: RepositoryStates{
+ "pool": {
+ Packfiles: &PackfilesState{
+ LooseObjects: nil,
+ // First commit and its tree object.
+ Packfiles: map[string]*PackfileState{
+ hash(t,
+ "pack-452292f7e0c6bcca1b42c53aaac4537416b5dbb9",
+ "pack-735ad245db57a16c41525c9101c42594d090c7021b51aa12d9104a4eea4223c5",
+ ): {
+ Objects: []git.ObjectID{
+ setup.ObjectHash.EmptyTreeOID,
+ setup.Commits.First.OID,
+ },
+ HasReverseIndex: true,
+ },
+ hash(t,
+ "pack-5a422f6c469963ffa026bf15cfd151751fba6e5f",
+ "pack-3ed0b733b17d82f87a350b856f7fd6d6a781d85c5d8d36fad64b459124444f11",
+ ): {
+ Objects: []git.ObjectID{
+ setup.Commits.Second.OID,
+ },
+ HasReverseIndex: true,
+ },
+ },
+ HasMultiPackIndex: false,
+ HasCommitGraphs: false,
+ },
+ DefaultBranch: "refs/heads/main",
+ References: &ReferencesState{
+ LooseReferences: map[git.ReferenceName]git.ObjectID{
+ "refs/heads/main": setup.Commits.First.OID,
+ },
+ },
+ },
+ "member": {
+ Packfiles: &PackfilesState{
+ LooseObjects: nil,
+ Packfiles: map[string]*PackfileState{
+ // Packfile containing second commit (reachable) and
+ // third commit (unreachable). Redundant objects in
+ // quarantined packs are removed.
+ hash(t,
+ "pack-529ec37accbc126425efe69abdf91153411532a6",
+ "pack-895b4eade6c459f47a382a0d637ef1ce34a661c76f003c7d7a38a7420e3afc69",
+ ): {
+ Objects: []git.ObjectID{
+ setup.Commits.Third.OID,
+ },
+ HasReverseIndex: true,
+ },
+ },
+ PooledObjects: []git.ObjectID{
+ setup.ObjectHash.EmptyTreeOID,
+ setup.Commits.First.OID,
+ // Both member and pool have second commit. It's
+ // deduplicated and the member inherits it from the
+ // pool.
+ setup.Commits.Second.OID,
+ },
+ HasMultiPackIndex: false,
+ HasCommitGraphs: false,
+ },
+ References: &ReferencesState{
+ LooseReferences: map[git.ReferenceName]git.ObjectID{
+ "refs/heads/branch": setup.Commits.Second.OID,
+ },
+ },
+ Alternate: "../../pool/objects",
+ },
+ },
+ Directory: generateDirectoryState(setup.Config, []*walDirectoryState{
+ {
+ lsn: 1,
+ includeObjects: []git.ObjectID{
+ setup.ObjectHash.EmptyTreeOID,
+ setup.Commits.First.OID,
+ },
+ },
+ {
+ lsn: 3,
+ includeObjects: []git.ObjectID{
+ setup.ObjectHash.EmptyTreeOID,
+ setup.Commits.First.OID,
+ setup.Commits.Second.OID,
+ setup.Commits.Third.OID,
+ },
+ },
+ {
+ lsn: 5,
+ includeObjects: []git.ObjectID{
+ setup.Commits.Second.OID,
+ },
+ },
+ {
+ lsn: 6,
+ includePackfiles: []string{hash(t,
+ "pack-529ec37accbc126425efe69abdf91153411532a6",
+ "pack-895b4eade6c459f47a382a0d637ef1ce34a661c76f003c7d7a38a7420e3afc69",
+ )},
+ },
+ }),
+ },
+ },
+ {
+ desc: "run repacking (FullWithUnreachable) on an alternate pool",
+ steps: steps{
+ RemoveRepository{},
+ StartManager{},
+ Begin{
+ TransactionID: 1,
+ RelativePath: "pool",
+ },
+ CreateRepository{
+ TransactionID: 1,
+ References: map[git.ReferenceName]git.ObjectID{
+ "refs/heads/main": setup.Commits.First.OID,
+ },
+ Packs: [][]byte{setup.Commits.First.Pack},
+ },
+ Commit{
+ TransactionID: 1,
+ },
+ Begin{
+ TransactionID: 2,
+ RelativePath: "member",
+ ExpectedSnapshotLSN: 1,
+ },
+ CreateRepository{
+ TransactionID: 2,
+ Alternate: "../../pool/objects",
+ },
+ Commit{
+ TransactionID: 2,
+ },
+ Begin{
+ TransactionID: 3,
+ RelativePath: "pool",
+ ExpectedSnapshotLSN: 2,
+ },
+ Commit{
+ TransactionID: 3,
+ ReferenceUpdates: ReferenceUpdates{
+ "refs/heads/branch": {OldOID: setup.ObjectHash.ZeroOID, NewOID: setup.Commits.Third.OID},
+ },
+ QuarantinedPacks: [][]byte{
+ setup.Commits.Second.Pack,
+ setup.Commits.Third.Pack,
+ },
+ },
+ Begin{
+ TransactionID: 4,
+ RelativePath: "pool",
+ ExpectedSnapshotLSN: 3,
+ },
+ Commit{
+ TransactionID: 4,
+ ReferenceUpdates: ReferenceUpdates{
+ "refs/heads/branch": {OldOID: setup.Commits.Third.OID, NewOID: setup.Commits.Second.OID},
+ },
+ },
+ Begin{
+ TransactionID: 5,
+ RelativePath: "pool",
+ ExpectedSnapshotLSN: 4,
+ },
+ RunRepack{
+ TransactionID: 5,
+ Config: housekeeping.RepackObjectsConfig{
+ Strategy: housekeeping.RepackObjectsStrategyFullWithUnreachable,
+ },
+ },
+ Commit{
+ TransactionID: 5,
+ },
+ },
+ expectedState: StateAssertion{
+ Database: DatabaseState{
+ string(keyAppliedLSN(setup.PartitionID)): LSN(5).toProto(),
+ },
+ Repositories: RepositoryStates{
+ "pool": {
+ Packfiles: &PackfilesState{
+ LooseObjects: nil,
+ Packfiles: map[string]*PackfileState{
+ hash(t,
+ "pack-144f9890c312a4cf5e66895bf721606d0f691083",
+ "pack-1d8b96ae9cc5301db6024e5d87974c960da6c017a9cf1bbed52bf8fe51e085de",
+ ): {
+ Objects: []git.ObjectID{
+ setup.ObjectHash.EmptyTreeOID,
+ setup.Commits.First.OID,
+ setup.Commits.Second.OID,
+ setup.Commits.Third.OID,
+ },
+ HasReverseIndex: true,
+ },
+ },
+ HasMultiPackIndex: false,
+ HasCommitGraphs: false,
+ },
+ DefaultBranch: "refs/heads/main",
+ References: &ReferencesState{
+ LooseReferences: map[git.ReferenceName]git.ObjectID{
+ "refs/heads/main": setup.Commits.First.OID,
+ "refs/heads/branch": setup.Commits.Second.OID,
+ },
+ },
+ },
+ "member": {
+ Packfiles: &PackfilesState{
+ LooseObjects: nil,
+ Packfiles: make(map[string]*PackfileState),
+ // All objects are accessible in member.
+ PooledObjects: []git.ObjectID{
+ setup.ObjectHash.EmptyTreeOID,
+ setup.Commits.First.OID,
+ setup.Commits.Second.OID,
+ setup.Commits.Third.OID,
+ },
+ HasMultiPackIndex: false,
+ HasCommitGraphs: false,
+ },
+ References: nil,
+ Alternate: "../../pool/objects",
+ },
+ },
+ Directory: generateDirectoryState(setup.Config, []*walDirectoryState{
+ {
+ lsn: 1,
+ includeObjects: []git.ObjectID{
+ setup.ObjectHash.EmptyTreeOID,
+ setup.Commits.First.OID,
+ },
+ },
+ {
+ lsn: 3,
+ includeObjects: []git.ObjectID{
+ setup.Commits.Second.OID,
+ setup.Commits.Third.OID,
+ },
+ },
+ {
+ lsn: 5,
+ includePackfiles: []string{hash(t,
+ "pack-144f9890c312a4cf5e66895bf721606d0f691083",
+ "pack-1d8b96ae9cc5301db6024e5d87974c960da6c017a9cf1bbed52bf8fe51e085de",
+ )},
+ },
+ }),
+ },
+ },
+ {
+ desc: "run repacking (Geometric) on an alternate member",
+ steps: steps{
+ RemoveRepository{},
+ StartManager{},
+ Begin{
+ TransactionID: 1,
+ RelativePath: "pool",
+ },
+ CreateRepository{
+ TransactionID: 1,
+ References: map[git.ReferenceName]git.ObjectID{
+ "refs/heads/main": setup.Commits.First.OID,
+ },
+ Packs: [][]byte{setup.Commits.First.Pack},
+ },
+ Commit{
+ TransactionID: 1,
+ },
+ Begin{
+ TransactionID: 2,
+ RelativePath: "member",
+ ExpectedSnapshotLSN: 1,
+ },
+ CreateRepository{
+ TransactionID: 2,
+ Alternate: "../../pool/objects",
+ },
+ Commit{
+ TransactionID: 2,
+ },
+ Begin{
+ TransactionID: 3,
+ RelativePath: "member",
+ ExpectedSnapshotLSN: 2,
+ },
+ Commit{
+ TransactionID: 3,
+ ReferenceUpdates: ReferenceUpdates{
+ "refs/heads/branch": {OldOID: setup.ObjectHash.ZeroOID, NewOID: setup.Commits.Second.OID},
+ },
+ QuarantinedPacks: [][]byte{
+ setup.Commits.Second.Pack,
+ },
+ },
+ Begin{
+ TransactionID: 4,
+ RelativePath: "member",
+ ExpectedSnapshotLSN: 3,
+ },
+ Commit{
+ TransactionID: 4,
+ ReferenceUpdates: ReferenceUpdates{
+ "refs/heads/branch": {OldOID: setup.Commits.Second.OID, NewOID: setup.Commits.Third.OID},
+ },
+ QuarantinedPacks: [][]byte{
+ setup.Commits.Third.Pack,
+ },
+ },
+ Begin{
+ TransactionID: 5,
+ RelativePath: "pool",
+ ExpectedSnapshotLSN: 4,
+ },
+ Commit{
+ TransactionID: 5,
+ QuarantinedPacks: [][]byte{
+ setup.Commits.Second.Pack,
+ },
+ IncludeObjects: []git.ObjectID{setup.Commits.Second.OID},
+ },
+ Begin{
+ TransactionID: 6,
+ RelativePath: "member",
+ ExpectedSnapshotLSN: 5,
+ },
+ RunRepack{
+ TransactionID: 6,
+ Config: housekeeping.RepackObjectsConfig{
+ Strategy: housekeeping.RepackObjectsStrategyGeometric,
+ },
+ },
+ Commit{
+ TransactionID: 6,
+ },
+ },
+ expectedState: StateAssertion{
+ Database: DatabaseState{
+ string(keyAppliedLSN(setup.PartitionID)): LSN(6).toProto(),
+ },
+ Repositories: RepositoryStates{
+ "pool": {
+ Packfiles: &PackfilesState{
+ LooseObjects: nil,
+ Packfiles: map[string]*PackfileState{
+ hash(t,
+ "pack-452292f7e0c6bcca1b42c53aaac4537416b5dbb9",
+ "pack-735ad245db57a16c41525c9101c42594d090c7021b51aa12d9104a4eea4223c5",
+ ): {
+ Objects: []git.ObjectID{
+ setup.ObjectHash.EmptyTreeOID,
+ setup.Commits.First.OID,
+ },
+ HasReverseIndex: true,
+ },
+ hash(t,
+ "pack-5a422f6c469963ffa026bf15cfd151751fba6e5f",
+ "pack-3ed0b733b17d82f87a350b856f7fd6d6a781d85c5d8d36fad64b459124444f11",
+ ): {
+ Objects: []git.ObjectID{
+ setup.Commits.Second.OID,
+ },
+ HasReverseIndex: true,
+ },
+ },
+ HasMultiPackIndex: false,
+ HasCommitGraphs: false,
+ },
+ DefaultBranch: "refs/heads/main",
+ References: &ReferencesState{
+ LooseReferences: map[git.ReferenceName]git.ObjectID{
+ "refs/heads/main": setup.Commits.First.OID,
+ },
+ },
+ },
+ "member": {
+ Packfiles: &PackfilesState{
+ LooseObjects: nil,
+ Packfiles: map[string]*PackfileState{
+ // This packfile matches the quarantined pack of
+ // transaction 3. Geometric repacking does not
+ // deduplicate second commit.
+ hash(t,
+ "pack-4274682fcb6a4dbb1a59ba7dd8577402e61ccbd2",
+ "pack-8ebabff3c37210ed37c4343255992f62a2ce113f7fb11f757de3bca157379d40",
+ ): {
+ Objects: []git.ObjectID{
+ setup.ObjectHash.EmptyTreeOID,
+ setup.Commits.First.OID,
+ setup.Commits.Second.OID,
+ },
+ HasReverseIndex: true,
+ },
+ // This packfile matches the quarantined pack of
+ // transaction 4.
+ hash(t,
+ "pack-529ec37accbc126425efe69abdf91153411532a6",
+ "pack-895b4eade6c459f47a382a0d637ef1ce34a661c76f003c7d7a38a7420e3afc69",
+ ): {
+ Objects: []git.ObjectID{
+ setup.Commits.Third.OID,
+ },
+ HasReverseIndex: true,
+ },
+ },
+ HasMultiPackIndex: false,
+ HasCommitGraphs: false,
+ },
+ References: &ReferencesState{
+ LooseReferences: map[git.ReferenceName]git.ObjectID{
+ "refs/heads/branch": setup.Commits.Third.OID,
+ },
+ },
+ Alternate: "../../pool/objects",
+ },
+ },
+ Directory: generateDirectoryState(setup.Config, []*walDirectoryState{
+ {
+ lsn: 1,
+ includeObjects: []git.ObjectID{
+ setup.ObjectHash.EmptyTreeOID,
+ setup.Commits.First.OID,
+ },
+ },
+ {
+ lsn: 3,
+ includeObjects: []git.ObjectID{
+ setup.ObjectHash.EmptyTreeOID,
+ setup.Commits.First.OID,
+ setup.Commits.Second.OID,
+ },
+ },
+ {
+ lsn: 4,
+ includeObjects: []git.ObjectID{
+ setup.Commits.Third.OID,
+ },
+ },
+ {
+ lsn: 5,
+ includeObjects: []git.ObjectID{
+ setup.Commits.Second.OID,
+ },
+ },
+ {
+ // As they form geometric progression, the repacking task keeps them intact.
+ lsn: 6,
+ },
+ }),
+ },
+ },
+ {
+ desc: "run repacking (FullWithCruft) on an alternate member",
+ steps: steps{
+ RemoveRepository{},
+ StartManager{},
+ Begin{
+ TransactionID: 1,
+ RelativePath: "pool",
+ },
+ CreateRepository{
+ TransactionID: 1,
+ References: map[git.ReferenceName]git.ObjectID{
+ "refs/heads/main": setup.Commits.First.OID,
+ },
+ Packs: [][]byte{setup.Commits.First.Pack},
+ },
+ Commit{
+ TransactionID: 1,
+ },
+ Begin{
+ TransactionID: 2,
+ RelativePath: "member",
+ ExpectedSnapshotLSN: 1,
+ },
+ CreateRepository{
+ TransactionID: 2,
+ Alternate: "../../pool/objects",
+ },
+ Commit{
+ TransactionID: 2,
+ },
+ Begin{
+ TransactionID: 3,
+ RelativePath: "member",
+ ExpectedSnapshotLSN: 2,
+ },
+ Commit{
+ TransactionID: 3,
+ ReferenceUpdates: ReferenceUpdates{
+ "refs/heads/branch-1": {OldOID: setup.ObjectHash.ZeroOID, NewOID: setup.Commits.Second.OID},
+ "refs/heads/branch-2": {OldOID: setup.ObjectHash.ZeroOID, NewOID: setup.Commits.Third.OID},
+ },
+ QuarantinedPacks: [][]byte{
+ setup.Commits.Second.Pack,
+ setup.Commits.Third.Pack,
+ setup.Commits.Diverging.Pack,
+ },
+ IncludeObjects: []git.ObjectID{setup.Commits.Diverging.OID},
+ },
+ Begin{
+ TransactionID: 4,
+ RelativePath: "pool",
+ ExpectedSnapshotLSN: 3,
+ },
+ Commit{
+ TransactionID: 4,
+ QuarantinedPacks: [][]byte{
+ setup.Commits.Second.Pack,
+ },
+ IncludeObjects: []git.ObjectID{setup.Commits.Second.OID},
+ },
+ Begin{
+ TransactionID: 5,
+ RelativePath: "member",
+ ExpectedSnapshotLSN: 4,
+ },
+ RunRepack{
+ TransactionID: 5,
+ Config: housekeeping.RepackObjectsConfig{
+ Strategy: housekeeping.RepackObjectsStrategyFullWithCruft,
+ },
+ },
+ Commit{
+ TransactionID: 5,
+ },
+ },
+ expectedState: StateAssertion{
+ Database: DatabaseState{
+ string(keyAppliedLSN(setup.PartitionID)): LSN(5).toProto(),
+ },
+ Repositories: RepositoryStates{
+ "pool": {
+ Packfiles: &PackfilesState{
+ Packfiles: map[string]*PackfileState{
+ hash(t,
+ "pack-452292f7e0c6bcca1b42c53aaac4537416b5dbb9",
+ "pack-735ad245db57a16c41525c9101c42594d090c7021b51aa12d9104a4eea4223c5",
+ ): {
+ Objects: []git.ObjectID{
+ setup.ObjectHash.EmptyTreeOID,
+ setup.Commits.First.OID,
+ },
+ HasReverseIndex: true,
+ },
+ hash(t,
+ "pack-5a422f6c469963ffa026bf15cfd151751fba6e5f",
+ "pack-3ed0b733b17d82f87a350b856f7fd6d6a781d85c5d8d36fad64b459124444f11",
+ ): {
+ Objects: []git.ObjectID{
+ setup.Commits.Second.OID,
+ },
+ HasReverseIndex: true,
+ },
+ },
+ HasMultiPackIndex: false,
+ HasCommitGraphs: false,
+ },
+ DefaultBranch: "refs/heads/main",
+ References: &ReferencesState{
+ LooseReferences: map[git.ReferenceName]git.ObjectID{
+ "refs/heads/main": setup.Commits.First.OID,
+ },
+ },
+ },
+ "member": {
+ Packfiles: &PackfilesState{
+ Packfiles: map[string]*PackfileState{
+ hash(t,
+ "pack-529ec37accbc126425efe69abdf91153411532a6",
+ "pack-895b4eade6c459f47a382a0d637ef1ce34a661c76f003c7d7a38a7420e3afc69",
+ ): {
+ Objects: []git.ObjectID{
+ // Diverging commit is pruned.
+ setup.Commits.Third.OID,
+ },
+ HasReverseIndex: true,
+ },
+ },
+ PooledObjects: []git.ObjectID{
+ setup.ObjectHash.EmptyTreeOID,
+ setup.Commits.First.OID,
+ // Second commit is deduplicated.
+ setup.Commits.Second.OID,
+ },
+ HasMultiPackIndex: false,
+ HasCommitGraphs: true,
+ },
+ References: &ReferencesState{
+ LooseReferences: map[git.ReferenceName]git.ObjectID{
+ "refs/heads/branch-1": setup.Commits.Second.OID,
+ "refs/heads/branch-2": setup.Commits.Third.OID,
+ },
+ },
+ Alternate: "../../pool/objects",
+ },
+ },
+ Directory: generateDirectoryState(setup.Config, []*walDirectoryState{
+ {
+ lsn: 1,
+ includeObjects: []git.ObjectID{
+ setup.ObjectHash.EmptyTreeOID,
+ setup.Commits.First.OID,
+ },
+ },
+ {
+ lsn: 3,
+ includeObjects: []git.ObjectID{
+ setup.ObjectHash.EmptyTreeOID,
+ setup.Commits.First.OID,
+ setup.Commits.Second.OID,
+ setup.Commits.Third.OID,
+ setup.Commits.Diverging.OID,
+ },
+ },
+ {
+ lsn: 4,
+ includeObjects: []git.ObjectID{
+ setup.Commits.Second.OID,
+ },
+ },
+ {
+ lsn: 5,
+ includePackfiles: []string{hash(t,
+ "pack-529ec37accbc126425efe69abdf91153411532a6",
+ "pack-895b4eade6c459f47a382a0d637ef1ce34a661c76f003c7d7a38a7420e3afc69",
+ )},
+ includeCommitGraphs: []string{hash(t,
+ "graph-5cd3399b1657ded0a67d1dc3f9fef739fd648116",
+ "graph-cc6c67a3c5e19b7b15b1f1551363a9d163fd14b13f11b6aeedcc5a9f40ffb590",
+ )},
+ },
+ }),
+ },
+ },
+ {
+ desc: "run repacking concurrently with other repacking task",
+ steps: steps{
+ StartManager{
+ ModifyStorage: func(tb testing.TB, cfg config.Cfg, storagePath string) {
+ repoPath := filepath.Join(storagePath, setup.RelativePath)
+ gittest.Exec(tb, cfg, "-C", repoPath, "repack", "-ad")
+ },
+ },
+ Begin{
+ TransactionID: 1,
+ RelativePath: setup.RelativePath,
+ },
+ Begin{
+ TransactionID: 2,
+ RelativePath: setup.RelativePath,
+ },
+ RunRepack{
+ TransactionID: 1,
+ Config: housekeeping.RepackObjectsConfig{
+ Strategy: housekeeping.RepackObjectsStrategyFullWithUnreachable,
+ },
+ },
+ RunRepack{
+ TransactionID: 2,
+ Config: housekeeping.RepackObjectsConfig{
+ Strategy: housekeeping.RepackObjectsStrategyFullWithUnreachable,
+ },
+ },
+ Commit{
+ TransactionID: 2,
+ },
+ Commit{
+ TransactionID: 1,
+ ExpectedError: errHousekeepingConflictConcurrent,
+ },
+ },
+ expectedState: StateAssertion{
+ Database: DatabaseState{
+ string(keyAppliedLSN(setup.PartitionID)): LSN(1).toProto(),
+ },
+ Repositories: RepositoryStates{
+ setup.RelativePath: {
+ DefaultBranch: "refs/heads/main",
+ Packfiles: &PackfilesState{
+ // Unreachable objects are packed.
+ LooseObjects: []git.ObjectID{
+ setup.ObjectHash.EmptyTreeOID,
+ setup.Commits.First.OID,
+ setup.Commits.Second.OID,
+ setup.Commits.Third.OID,
+ setup.Commits.Diverging.OID,
+ },
+ Packfiles: map[string]*PackfileState{},
+ },
+ },
+ },
+ Directory: testhelper.DirectoryState{
+ "/": {Mode: fs.ModeDir | perm.PrivateDir},
+ "/wal": {Mode: fs.ModeDir | perm.PrivateDir},
+ "/wal/1": {Mode: fs.ModeDir | perm.PrivateDir},
+ },
+ },
+ },
+ {
+ desc: "run repacking concurrently with other housekeeping task",
+ steps: steps{
+ StartManager{
+ ModifyStorage: func(tb testing.TB, cfg config.Cfg, storagePath string) {
+ repoPath := filepath.Join(storagePath, setup.RelativePath)
+ gittest.Exec(tb, cfg, "-C", repoPath, "repack", "-ad")
+ },
+ },
+ Begin{
+ TransactionID: 1,
+ RelativePath: setup.RelativePath,
+ },
+ Begin{
+ TransactionID: 2,
+ RelativePath: setup.RelativePath,
+ },
+ RunRepack{
+ TransactionID: 1,
+ Config: housekeeping.RepackObjectsConfig{
+ Strategy: housekeeping.RepackObjectsStrategyFullWithUnreachable,
+ },
+ },
+ RunPackRefs{
+ TransactionID: 2,
+ },
+ Commit{
+ TransactionID: 2,
+ },
+ Commit{
+ TransactionID: 1,
+ ExpectedError: errHousekeepingConflictConcurrent,
+ },
+ },
+ expectedState: StateAssertion{
+ Database: DatabaseState{
+ string(keyAppliedLSN(setup.PartitionID)): LSN(1).toProto(),
+ },
+ Repositories: RepositoryStates{
+ setup.RelativePath: {
+ DefaultBranch: "refs/heads/main",
+ Packfiles: &PackfilesState{
+ // Unreachable objects are packed.
+ LooseObjects: []git.ObjectID{
+ setup.ObjectHash.EmptyTreeOID,
+ setup.Commits.First.OID,
+ setup.Commits.Second.OID,
+ setup.Commits.Third.OID,
+ setup.Commits.Diverging.OID,
+ },
+ Packfiles: map[string]*PackfileState{},
+ },
+ },
+ },
+ Directory: testhelper.DirectoryState{
+ "/": {Mode: fs.ModeDir | perm.PrivateDir},
+ "/wal": {Mode: fs.ModeDir | perm.PrivateDir},
+ "/wal/1": {Mode: fs.ModeDir | perm.PrivateDir},
+ "/wal/1/packed-refs": anyDirectoryEntry(setup.Config),
+ },
+ },
+ },
+ }
+}
diff --git a/internal/gitaly/storage/storagemgr/transaction_manager_test.go b/internal/gitaly/storage/storagemgr/transaction_manager_test.go
index cdc8aa655..d868ee76d 100644
--- a/internal/gitaly/storage/storagemgr/transaction_manager_test.go
+++ b/internal/gitaly/storage/storagemgr/transaction_manager_test.go
@@ -92,6 +92,16 @@ func anyDirectoryEntry(cfg config.Cfg) testhelper.DirectoryEntry {
}
}
+// anyDirectoryEntryWithPerm returns a DirectoryEntry that checks for the existence of a file having a particular perm.
+// The content will be asserted in the later state.
+func anyDirectoryEntryWithPerm(cfg config.Cfg, perm os.FileMode) testhelper.DirectoryEntry {
+ return testhelper.DirectoryEntry{
+ Mode: perm,
+ Content: "",
+ ParseContent: func(testing.TB, string, []byte) any { return "" },
+ }
+}
+
// packFileDirectoryEntry returns a DirectoryEntry that parses content as a pack file and asserts that the
// set of objects in the pack file matches the expected objects.
func packFileDirectoryEntry(cfg config.Cfg, expectedObjects []git.ObjectID) testhelper.DirectoryEntry {
@@ -263,6 +273,8 @@ func TestTransactionManager(t *testing.T) {
generateAlternateTests(t, setup),
generateCustomHooksTests(t, setup),
generateHousekeepingPackRefsTests(t, ctx, testPartitionID, relativePath),
+ generateHousekeepingRepackingStrategyTests(t, ctx, testPartitionID, relativePath),
+ generateHousekeepingRepackingConcurrentTests(t, ctx, setup),
}
for _, subCases := range subTests {
testCases = append(testCases, subCases...)