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:
authorSami Hiltunen <shiltunen@gitlab.com>2023-05-30 13:57:12 +0300
committerSami Hiltunen <shiltunen@gitlab.com>2023-05-30 13:57:12 +0300
commit67a5d6727d2ee6b4c50fcee9c3b9be5a8b528e42 (patch)
tree1d624dcea3268c97679f0ee68032aaca6b0daacb
parent58851fe18b0291ed7b11c8821c7ae548c2a96337 (diff)
parent55febba5ddd189c48c72fab92258984f9e201f26 (diff)
Merge branch '4736-clean-up-stale-locks-when-applying-wal-in-transactionmanager' into 'master'
gitaly: Cleanup stale reflocks in `prepareReferenceTransaction` Closes #4736 See merge request https://gitlab.com/gitlab-org/gitaly/-/merge_requests/5758 Merged-by: Sami Hiltunen <shiltunen@gitlab.com> Approved-by: Sami Hiltunen <shiltunen@gitlab.com> Reviewed-by: Sami Hiltunen <shiltunen@gitlab.com> Reviewed-by: karthik nayak <knayak@gitlab.com> Co-authored-by: Karthik Nayak <knayak@gitlab.com>
-rw-r--r--internal/git/housekeeping/manager.go14
-rw-r--r--internal/gitaly/partition_manager.go16
-rw-r--r--internal/gitaly/partition_manager_test.go9
-rw-r--r--internal/gitaly/transaction_manager.go100
-rw-r--r--internal/gitaly/transaction_manager_hook_test.go31
-rw-r--r--internal/gitaly/transaction_manager_test.go45
-rw-r--r--internal/testhelper/testserver/gitaly.go1
7 files changed, 153 insertions, 63 deletions
diff --git a/internal/git/housekeeping/manager.go b/internal/git/housekeeping/manager.go
index 173ad06c2..d73152182 100644
--- a/internal/git/housekeeping/manager.go
+++ b/internal/git/housekeeping/manager.go
@@ -19,6 +19,8 @@ type Manager interface {
// OptimizeRepository optimizes the repository's data structures such that it can be more
// efficiently served.
OptimizeRepository(context.Context, *localrepo.Repo, ...OptimizeRepositoryOption) error
+ // AddPackRefsInhibitor allows clients to block housekeeping from running git-pack-refs(1).
+ AddPackRefsInhibitor(ctx context.Context, repoPath string) (bool, func(), error)
}
// repositoryState holds the housekeeping state for individual repositories. This structure can be
@@ -142,7 +144,10 @@ func (s *repositoryStates) addPackRefsInhibitor(ctx context.Context, repoPath st
select {
case <-ctx.Done():
return false, nil, ctx.Err()
- case <-state.packRefsDone:
+ case <-packRefsDone:
+ // We don't use state.packRefsDone, cause there is possibility that it is set
+ // to `nil` by the cleanup function after running `git-pack-refs(1)`.
+ //
// We obtain a lock and continue the loop here to avoid a race wherein another
// goroutine has invoked git-pack-refs(1). By continuing the loop and checking
// the value of packRefsDone, we can avoid that scenario.
@@ -311,3 +316,10 @@ func (m *RepositoryManager) Collect(metrics chan<- prometheus.Metric) {
m.dataStructureSize.Collect(metrics)
m.dataStructureTimeSinceLastOptimization.Collect(metrics)
}
+
+// AddPackRefsInhibitor exposes the internal function addPackRefsInhibitor on the
+// RepositoryManager level. This can then be used by other clients to block housekeeping
+// from running git-pack-refs(1).
+func (m *RepositoryManager) AddPackRefsInhibitor(ctx context.Context, repoPath string) (successful bool, _ func(), err error) {
+ return m.repositoryStates.addPackRefsInhibitor(ctx, repoPath)
+}
diff --git a/internal/gitaly/partition_manager.go b/internal/gitaly/partition_manager.go
index 14a115a28..deaeb84e5 100644
--- a/internal/gitaly/partition_manager.go
+++ b/internal/gitaly/partition_manager.go
@@ -12,6 +12,7 @@ import (
"github.com/dgraph-io/badger/v3"
"github.com/sirupsen/logrus"
"gitlab.com/gitlab-org/gitaly/v16/internal/git"
+ "gitlab.com/gitlab-org/gitaly/v16/internal/git/housekeeping"
"gitlab.com/gitlab-org/gitaly/v16/internal/git/localrepo"
repo "gitlab.com/gitlab-org/gitaly/v16/internal/git/repository"
"gitlab.com/gitlab-org/gitaly/v16/internal/gitaly/config"
@@ -30,6 +31,8 @@ type PartitionManager struct {
storages map[string]*storageManager
// commandFactory is passed as a dependency to the constructed TransactionManagers.
commandFactory git.CommandFactory
+ // housekeepingManager access to the housekeeping.Manager.
+ housekeepingManager housekeeping.Manager
}
// storageManager represents a single storage.
@@ -111,7 +114,13 @@ func (ptn *partition) stop() {
}
// NewPartitionManager returns a new PartitionManager.
-func NewPartitionManager(configuredStorages []config.Storage, cmdFactory git.CommandFactory, localRepoFactory localrepo.Factory, logger logrus.FieldLogger) (*PartitionManager, error) {
+func NewPartitionManager(
+ configuredStorages []config.Storage,
+ cmdFactory git.CommandFactory,
+ housekeepingManager housekeeping.Manager,
+ localRepoFactory localrepo.Factory,
+ logger logrus.FieldLogger,
+) (*PartitionManager, error) {
storages := make(map[string]*storageManager, len(configuredStorages))
for _, storage := range configuredStorages {
repoFactory, err := localRepoFactory.ScopeByStorage(storage.Name)
@@ -154,7 +163,7 @@ func NewPartitionManager(configuredStorages []config.Storage, cmdFactory git.Com
}
}
- return &PartitionManager{storages: storages, commandFactory: cmdFactory}, nil
+ return &PartitionManager{storages: storages, commandFactory: cmdFactory, housekeepingManager: housekeepingManager}, nil
}
func stagingDirectoryPath(storagePath string) string {
@@ -194,7 +203,8 @@ func (pm *PartitionManager) Begin(ctx context.Context, repo repo.GitRepo) (*Tran
return nil, fmt.Errorf("create staging directory: %w", err)
}
- mgr := NewTransactionManager(storageMgr.database, storageMgr.path, relativePath, stagingDir, storageMgr.repoFactory, pm.commandFactory, storageMgr.transactionFinalizerFactory(ptn))
+ mgr := NewTransactionManager(storageMgr.database, storageMgr.path, relativePath, stagingDir, pm.commandFactory, pm.housekeepingManager, storageMgr.repoFactory, storageMgr.transactionFinalizerFactory(ptn))
+
ptn.transactionManager = mgr
storageMgr.partitions[relativePath] = ptn
diff --git a/internal/gitaly/partition_manager_test.go b/internal/gitaly/partition_manager_test.go
index 0c8056474..623a34b14 100644
--- a/internal/gitaly/partition_manager_test.go
+++ b/internal/gitaly/partition_manager_test.go
@@ -12,9 +12,12 @@ import (
"gitlab.com/gitlab-org/gitaly/v16/internal/git"
"gitlab.com/gitlab-org/gitaly/v16/internal/git/catfile"
"gitlab.com/gitlab-org/gitaly/v16/internal/git/gittest"
+ "gitlab.com/gitlab-org/gitaly/v16/internal/git/housekeeping"
"gitlab.com/gitlab-org/gitaly/v16/internal/git/localrepo"
repo "gitlab.com/gitlab-org/gitaly/v16/internal/git/repository"
"gitlab.com/gitlab-org/gitaly/v16/internal/gitaly/config"
+ "gitlab.com/gitlab-org/gitaly/v16/internal/gitaly/transaction"
+ "gitlab.com/gitlab-org/gitaly/v16/internal/grpc/backchannel"
"gitlab.com/gitlab-org/gitaly/v16/internal/helper/perm"
"gitlab.com/gitlab-org/gitaly/v16/internal/structerr"
"gitlab.com/gitlab-org/gitaly/v16/internal/testhelper"
@@ -563,8 +566,12 @@ func TestPartitionManager(t *testing.T) {
)
}
- partitionManager, err := NewPartitionManager(cfg.Storages, cmdFactory, localRepoFactory, logrus.StandardLogger())
+ txManager := transaction.NewManager(cfg, backchannel.NewRegistry())
+ housekeepingManager := housekeeping.NewManager(cfg.Prometheus, txManager)
+
+ partitionManager, err := NewPartitionManager(cfg.Storages, cmdFactory, housekeepingManager, localRepoFactory, logrus.StandardLogger())
require.NoError(t, err)
+
defer func() {
partitionManager.Stop()
for _, storage := range cfg.Storages {
diff --git a/internal/gitaly/transaction_manager.go b/internal/gitaly/transaction_manager.go
index 9a159569b..82f2d6983 100644
--- a/internal/gitaly/transaction_manager.go
+++ b/internal/gitaly/transaction_manager.go
@@ -18,10 +18,10 @@ import (
"github.com/dgraph-io/badger/v3"
"gitlab.com/gitlab-org/gitaly/v16/internal/git"
+ "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/updateref"
"gitlab.com/gitlab-org/gitaly/v16/internal/gitaly/repoutil"
- "gitlab.com/gitlab-org/gitaly/v16/internal/gitaly/transaction"
"gitlab.com/gitlab-org/gitaly/v16/internal/helper/perm"
"gitlab.com/gitlab-org/gitaly/v16/internal/safe"
"gitlab.com/gitlab-org/gitaly/v16/internal/structerr"
@@ -158,7 +158,7 @@ type Transaction struct {
includesPack bool
// stagingRepository is a repository that is used to stage the transaction. If there are quarantined
// objects, it has the quarantine applied so the objects are available for verification and packing.
- stagingRepository repository
+ stagingRepository *localrepo.Repo
// Snapshot contains the details of the Transaction's read snapshot.
snapshot Snapshot
@@ -424,7 +424,7 @@ type TransactionManager struct {
// never been created, or if it has been deleted.
repositoryExists bool
// repository is the repository this TransactionManager is acting on.
- repository repository
+ repository *localrepo.Repo
// repositoryPath is the path to the repository this TransactionManager is acting on.
repositoryPath string
// relativePath is the repository's relative path inside the storage.
@@ -456,6 +456,8 @@ type TransactionManager struct {
appliedLogIndex LogIndex
// customHookIndex stores the log index of the latest committed custom custom hooks in the repository.
customHookIndex LogIndex
+ // housekeepingManager access to the housekeeping.Manager.
+ housekeepingManager housekeeping.Manager
// transactionFinalizer executes when a transaction is completed.
transactionFinalizer func()
@@ -466,19 +468,17 @@ type TransactionManager struct {
awaitingTransactions map[LogIndex]resultChannel
}
-// repository is the localrepo interface used by TransactionManager.
-type repository interface {
- git.RepositoryExecutor
- ResolveRevision(context.Context, git.Revision) (git.ObjectID, error)
- SetDefaultBranch(ctx context.Context, txManager transaction.Manager, reference git.ReferenceName) error
- UnpackObjects(context.Context, io.Reader) error
- Quarantine(string) (*localrepo.Repo, error)
- WalkUnreachableObjects(context.Context, io.Reader, io.Writer) error
- PackObjects(context.Context, io.Reader, io.Writer) error
-}
-
// NewTransactionManager returns a new TransactionManager for the given repository.
-func NewTransactionManager(db *badger.DB, storagePath, relativePath, stagingDir string, repositoryFactory localrepo.StorageScopedFactory, cmdFactory git.CommandFactory, transactionFinalizer func()) *TransactionManager {
+func NewTransactionManager(
+ db *badger.DB,
+ storagePath,
+ relativePath,
+ stagingDir string,
+ cmdFactory git.CommandFactory,
+ housekeepingManager housekeeping.Manager,
+ repositoryFactory localrepo.StorageScopedFactory,
+ transactionFinalizer func(),
+) *TransactionManager {
ctx, cancel := context.WithCancel(context.Background())
return &TransactionManager{
ctx: ctx,
@@ -495,6 +495,7 @@ func NewTransactionManager(db *badger.DB, storagePath, relativePath, stagingDir
initialized: make(chan struct{}),
applyNotifications: make(map[LogIndex]chan struct{}),
stagingDirectory: stagingDir,
+ housekeepingManager: housekeepingManager,
transactionFinalizer: transactionFinalizer,
awaitingTransactions: make(map[LogIndex]resultChannel),
}
@@ -1100,7 +1101,7 @@ func verifyReferencePrefix(referenceName git.ReferenceName) error {
// the updates will go through when they are being applied in the log. This also catches any invalid reference names
// and file/directory conflicts with Git's loose reference storage which can occur with references like
// 'refs/heads/parent' and 'refs/heads/parent/child'.
-func (mgr *TransactionManager) verifyReferencesWithGit(ctx context.Context, referenceUpdates []*gitalypb.LogEntry_ReferenceUpdate, stagingRepository repository) error {
+func (mgr *TransactionManager) verifyReferencesWithGit(ctx context.Context, referenceUpdates []*gitalypb.LogEntry_ReferenceUpdate, stagingRepository *localrepo.Repo) error {
updater, err := mgr.prepareReferenceTransaction(ctx, referenceUpdates, stagingRepository)
if err != nil {
return fmt.Errorf("prepare reference transaction: %w", err)
@@ -1155,27 +1156,68 @@ func (mgr *TransactionManager) applyDefaultBranchUpdate(ctx context.Context, def
// prepareReferenceTransaction prepares a reference transaction with `git update-ref`. It leaves committing
// or aborting up to the caller. Either should be called to clean up the process. The process is cleaned up
// if an error is returned.
-func (mgr *TransactionManager) prepareReferenceTransaction(ctx context.Context, referenceUpdates []*gitalypb.LogEntry_ReferenceUpdate, repository repository) (*updateref.Updater, error) {
- updater, err := updateref.New(ctx, repository, updateref.WithDisabledTransactions())
- if err != nil {
- return nil, fmt.Errorf("new: %w", err)
+func (mgr *TransactionManager) prepareReferenceTransaction(ctx context.Context, referenceUpdates []*gitalypb.LogEntry_ReferenceUpdate, repository *localrepo.Repo) (*updateref.Updater, error) {
+ // This section runs git-update-ref(1), but could fail due to existing
+ // reference locks. So we create a function which can be called again
+ // post cleanup of stale reference locks.
+ updateFunc := func() (*updateref.Updater, error) {
+ updater, err := updateref.New(ctx, repository, updateref.WithDisabledTransactions())
+ if err != nil {
+ return nil, fmt.Errorf("new: %w", err)
+ }
+
+ if err := updater.Start(); err != nil {
+ return nil, fmt.Errorf("start: %w", err)
+ }
+
+ for _, referenceUpdate := range referenceUpdates {
+ if err := updater.Update(git.ReferenceName(referenceUpdate.ReferenceName), git.ObjectID(referenceUpdate.NewOid), ""); err != nil {
+ return nil, fmt.Errorf("update %q: %w", referenceUpdate.ReferenceName, err)
+ }
+ }
+
+ if err := updater.Prepare(); err != nil {
+ return nil, fmt.Errorf("prepare: %w", err)
+ }
+
+ return updater, nil
}
- if err := updater.Start(); err != nil {
- return nil, fmt.Errorf("start: %w", err)
+ // If git-update-ref(1) runs without issues, our work here is done.
+ updater, err := updateFunc()
+ if err == nil {
+ return updater, nil
}
- for _, referenceUpdate := range referenceUpdates {
- if err := updater.Update(git.ReferenceName(referenceUpdate.ReferenceName), git.ObjectID(referenceUpdate.NewOid), ""); err != nil {
- return nil, fmt.Errorf("update %q: %w", referenceUpdate.ReferenceName, err)
+ // If we get an error due to existing stale reference locks, we should clear it up
+ // and retry running git-update-ref(1).
+ var updateRefError *updateref.AlreadyLockedError
+ if errors.As(err, &updateRefError) {
+ // Before clearing stale reference locks, we add should ensure that housekeeping doesn't
+ // run git-pack-refs(1), which could create new reference locks. So we add an inhibitor.
+ success, cleanup, err := mgr.housekeepingManager.AddPackRefsInhibitor(ctx, mgr.repositoryPath)
+ if !success {
+ return nil, fmt.Errorf("add pack-refs inhibitor: %w", err)
+ }
+ defer cleanup()
+
+ // We ask housekeeping to cleanup stale reference locks. We don't add a grace period, because
+ // transaction manager is the only process which writes into the repository, so it is safe
+ // to delete these locks.
+ if err := mgr.housekeepingManager.CleanStaleData(ctx, mgr.repository, housekeeping.OnlyStaleReferenceLockCleanup(0)); err != nil {
+ return nil, fmt.Errorf("running reflock cleanup: %w", err)
}
- }
- if err := updater.Prepare(); err != nil {
- return nil, fmt.Errorf("prepare: %w", err)
+ // We try a second time, this should succeed. If not, there is something wrong and
+ // we return the error.
+ //
+ // Do note, that we've already added an inhibitor above, so git-pack-refs(1) won't run
+ // again until we return from this function so ideally this should work, but in case it
+ // doesn't we return the error.
+ return updateFunc()
}
- return updater, nil
+ return nil, err
}
// appendLogEntry appends the transaction to the write-ahead log. References that failed verification are skipped and thus not
diff --git a/internal/gitaly/transaction_manager_hook_test.go b/internal/gitaly/transaction_manager_hook_test.go
index 3ecee4e14..e529ea8b4 100644
--- a/internal/gitaly/transaction_manager_hook_test.go
+++ b/internal/gitaly/transaction_manager_hook_test.go
@@ -1,7 +1,6 @@
package gitaly
import (
- "context"
"regexp"
"runtime"
"strings"
@@ -9,7 +8,6 @@ import (
"github.com/dgraph-io/badger/v3"
"github.com/stretchr/testify/require"
- "gitlab.com/gitlab-org/gitaly/v16/internal/git"
)
// hookFunc is a function that is executed at a specific point. It gets a hookContext that allows it to
@@ -31,8 +29,8 @@ type hookContext struct {
type hooks struct {
// beforeReadLogEntry is invoked before a log entry is read from the database.
beforeReadLogEntry hookFunc
- // beforeResolveRevision is invoked before ResolveRevision is invoked.
- beforeResolveRevision hookFunc
+ // beforeStoreLogEntry is invoked before the log entry is stored to the database.
+ beforeStoreLogEntry hookFunc
// beforeDeferredStop is invoked before the deferred Stop is invoked in Run.
beforeDeferredStop hookFunc
// beforeDeleteLogEntry is invoked before a log entry is deleted from the database.
@@ -66,26 +64,6 @@ func installHooks(tb testing.TB, transactionManager *TransactionManager, databas
hooks: hooks,
hookContext: hookContext,
}
-
- transactionManager.repository = repositoryHook{
- repository: transactionManager.repository,
- hookContext: hookContext,
- hooks: hooks,
- }
-}
-
-type repositoryHook struct {
- repository
- hookContext
- hooks
-}
-
-func (hook repositoryHook) ResolveRevision(ctx context.Context, revision git.Revision) (git.ObjectID, error) {
- if hook.beforeResolveRevision != nil {
- hook.hooks.beforeResolveRevision(hook.hookContext)
- }
-
- return hook.repository.ResolveRevision(ctx, revision)
}
type databaseHook struct {
@@ -165,6 +143,11 @@ func (hook writeBatchHook) Set(key []byte, value []byte) error {
if regexLogIndex.Match(key) && hook.hooks.beforeStoreAppliedLogIndex != nil {
hook.hooks.beforeStoreAppliedLogIndex(hook.hookContext)
}
+
+ if regexLogEntry.Match(key) && hook.hooks.beforeStoreLogEntry != nil {
+ hook.hooks.beforeStoreLogEntry(hook.hookContext)
+ }
+
return hook.writeBatch.Set(key, value)
}
diff --git a/internal/gitaly/transaction_manager_test.go b/internal/gitaly/transaction_manager_test.go
index 7c277f3b2..f4817c3d2 100644
--- a/internal/gitaly/transaction_manager_test.go
+++ b/internal/gitaly/transaction_manager_test.go
@@ -21,10 +21,13 @@ import (
"gitlab.com/gitlab-org/gitaly/v16/internal/git"
"gitlab.com/gitlab-org/gitaly/v16/internal/git/catfile"
"gitlab.com/gitlab-org/gitaly/v16/internal/git/gittest"
+ "gitlab.com/gitlab-org/gitaly/v16/internal/git/housekeeping"
"gitlab.com/gitlab-org/gitaly/v16/internal/git/localrepo"
"gitlab.com/gitlab-org/gitaly/v16/internal/git/updateref"
"gitlab.com/gitlab-org/gitaly/v16/internal/gitaly/config"
"gitlab.com/gitlab-org/gitaly/v16/internal/gitaly/repoutil"
+ "gitlab.com/gitlab-org/gitaly/v16/internal/gitaly/transaction"
+ "gitlab.com/gitlab-org/gitaly/v16/internal/grpc/backchannel"
"gitlab.com/gitlab-org/gitaly/v16/internal/helper/perm"
"gitlab.com/gitlab-org/gitaly/v16/internal/testhelper"
"gitlab.com/gitlab-org/gitaly/v16/internal/testhelper/testcfg"
@@ -420,6 +423,30 @@ func TestTransactionManager(t *testing.T) {
},
},
{
+ desc: "create reference with existing reference lock",
+ steps: steps{
+ StartManager{
+ ModifyRepository: func(_ testing.TB, repoPath string) {
+ err := os.WriteFile(fmt.Sprintf("%s/refs/heads/main.lock", repoPath), []byte{}, 0o666)
+ require.NoError(t, err)
+ },
+ },
+ Begin{},
+ Commit{
+ ReferenceUpdates: ReferenceUpdates{
+ "refs/heads/main": {OldOID: setup.ObjectHash.ZeroOID, NewOID: setup.Commits.First.OID},
+ },
+ },
+ },
+ expectedState: StateAssertion{
+ DefaultBranch: "refs/heads/main",
+ References: []git.Reference{{Name: "refs/heads/main", Target: setup.Commits.First.OID.String()}},
+ Database: DatabaseState{
+ string(keyAppliedLogIndex(relativePath)): LogIndex(1).toProto(),
+ },
+ },
+ },
+ {
desc: "create a file-directory reference conflict different transaction",
steps: steps{
StartManager{},
@@ -2864,11 +2891,14 @@ func TestTransactionManager(t *testing.T) {
stagingDir := t.TempDir()
storagePath := setup.Config.Storages[0].Path
+ txManager := transaction.NewManager(setup.Config, backchannel.NewRegistry())
+ housekeepingManager := housekeeping.NewManager(setup.Config.Prometheus, txManager)
+
var (
// managerRunning tracks whether the manager is running or stopped.
managerRunning bool
// transactionManager is the current TransactionManager instance.
- transactionManager = NewTransactionManager(database, storagePath, relativePath, stagingDir, setup.RepositoryFactory, setup.CommandFactory, noopTransactionFinalizer)
+ transactionManager = NewTransactionManager(database, storagePath, relativePath, stagingDir, setup.CommandFactory, housekeepingManager, setup.RepositoryFactory, noopTransactionFinalizer)
// managerErr is used for synchronizing manager stopping and returning
// the error from Run.
managerErr chan error
@@ -2909,10 +2939,11 @@ func TestTransactionManager(t *testing.T) {
managerRunning = true
managerErr = make(chan error)
- transactionManager = NewTransactionManager(database, storagePath, relativePath, stagingDir, setup.RepositoryFactory, setup.CommandFactory, noopTransactionFinalizer)
+ transactionManager = NewTransactionManager(database, storagePath, relativePath, stagingDir, setup.CommandFactory, housekeepingManager, setup.RepositoryFactory, noopTransactionFinalizer)
+
installHooks(t, transactionManager, database, hooks{
- beforeReadLogEntry: step.Hooks.BeforeApplyLogEntry,
- beforeResolveRevision: step.Hooks.BeforeAppendLogEntry,
+ beforeReadLogEntry: step.Hooks.BeforeApplyLogEntry,
+ beforeStoreLogEntry: step.Hooks.BeforeAppendLogEntry,
beforeDeferredStop: func(hookContext) {
if step.Hooks.WaitForTransactionsWhenStopping {
inflightTransactions.Wait()
@@ -3215,6 +3246,9 @@ func BenchmarkTransactionManager(b *testing.B) {
require.NoError(b, err)
defer testhelper.MustClose(b, database)
+ txManager := transaction.NewManager(cfg, backchannel.NewRegistry())
+ housekeepingManager := housekeeping.NewManager(cfg.Prometheus, txManager)
+
var (
// managerWG records the running TransactionManager.Run goroutines.
managerWG sync.WaitGroup
@@ -3256,7 +3290,8 @@ func BenchmarkTransactionManager(b *testing.B) {
commit1 = gittest.WriteCommit(b, cfg, repoPath, gittest.WithParents())
commit2 = gittest.WriteCommit(b, cfg, repoPath, gittest.WithParents(commit1))
- manager := NewTransactionManager(database, cfg.Storages[0].Path, repo.RelativePath, b.TempDir(), repositoryFactory, cmdFactory, noopTransactionFinalizer)
+ manager := NewTransactionManager(database, cfg.Storages[0].Path, repo.RelativePath, b.TempDir(), cmdFactory, housekeepingManager, repositoryFactory, noopTransactionFinalizer)
+
managers = append(managers, manager)
managerWG.Add(1)
diff --git a/internal/testhelper/testserver/gitaly.go b/internal/testhelper/testserver/gitaly.go
index 43ac10b0c..d46061101 100644
--- a/internal/testhelper/testserver/gitaly.go
+++ b/internal/testhelper/testserver/gitaly.go
@@ -355,6 +355,7 @@ func (gsd *gitalyServerDeps) createDependencies(tb testing.TB, cfg config.Cfg) *
partitionManager, err = gitaly.NewPartitionManager(
cfg.Storages,
gsd.gitCmdFactory,
+ gsd.housekeepingManager,
localrepo.NewFactory(gsd.locator, gsd.gitCmdFactory, gsd.catfileCache),
gsd.logger,
)