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:
-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...)