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>2021-05-10 12:43:29 +0300
committerSami Hiltunen <shiltunen@gitlab.com>2021-05-10 12:43:29 +0300
commit0e4cbf83d05a1350600b6ccf82cee2cb81b87b40 (patch)
tree05f7af6de8a5d75b3fd9a7257b7949dcd1a1f4a7
parent1966c1d9d4f11d05e73519e13ed8d0f56e3f0155 (diff)
parent0319e9ce9350d14d5aef9733ddcddbe6b764f6ae (diff)
Merge branch 'pks-catfile-drop-global' into 'master'
catfile: Drop global variable See merge request gitlab-org/gitaly!3447
-rw-r--r--cmd/gitaly/main.go5
-rw-r--r--internal/git/catfile/batch.go117
-rw-r--r--internal/git/catfile/batch_cache.go209
-rw-r--r--internal/git/catfile/batch_cache_test.go62
-rw-r--r--internal/git/catfile/batch_check_process.go16
-rw-r--r--internal/git/catfile/batch_process.go12
-rw-r--r--internal/git/catfile/batch_test.go35
-rw-r--r--internal/git/localrepo/config_test.go5
-rw-r--r--internal/git/localrepo/objects.go3
-rw-r--r--internal/git/localrepo/objects_test.go4
-rw-r--r--internal/git/localrepo/refs_test.go7
-rw-r--r--internal/git/localrepo/remote_test.go24
-rw-r--r--internal/git/localrepo/repo.go8
-rw-r--r--internal/git/localrepo/repo_test.go13
-rw-r--r--internal/git/log/commit_test.go3
-rw-r--r--internal/git/log/log.go4
-rw-r--r--internal/git/objectpool/clone_test.go12
-rw-r--r--internal/git/objectpool/pool.go22
-rw-r--r--internal/git/objectpool/pool_test.go10
-rw-r--r--internal/git/objectpool/proto.go11
-rw-r--r--internal/git/remoterepo/repository_test.go17
-rw-r--r--internal/gitaly/maintenance/optimize_test.go4
-rw-r--r--internal/gitaly/server/auth_test.go3
-rw-r--r--internal/gitaly/service/blob/get_blob.go2
-rw-r--r--internal/gitaly/service/blob/get_blobs.go2
-rw-r--r--internal/gitaly/service/blob/server.go7
-rw-r--r--internal/gitaly/service/blob/testhelper_test.go7
-rw-r--r--internal/gitaly/service/cleanup/apply_bfg_object_map_stream.go2
-rw-r--r--internal/gitaly/service/cleanup/notifier/notifier.go4
-rw-r--r--internal/gitaly/service/cleanup/server.go10
-rw-r--r--internal/gitaly/service/cleanup/testhelper_test.go6
-rw-r--r--internal/gitaly/service/commit/between.go3
-rw-r--r--internal/gitaly/service/commit/commit_messages.go3
-rw-r--r--internal/gitaly/service/commit/commit_signatures.go2
-rw-r--r--internal/gitaly/service/commit/commits_by_message.go2
-rw-r--r--internal/gitaly/service/commit/commits_helper.go14
-rw-r--r--internal/gitaly/service/commit/filter_shas_with_signatures.go2
-rw-r--r--internal/gitaly/service/commit/find_all_commits.go2
-rw-r--r--internal/gitaly/service/commit/find_commit_test.go5
-rw-r--r--internal/gitaly/service/commit/find_commits.go2
-rw-r--r--internal/gitaly/service/commit/last_commit_for_path.go3
-rw-r--r--internal/gitaly/service/commit/list_commits_by_oid.go2
-rw-r--r--internal/gitaly/service/commit/list_commits_by_ref_name.go2
-rw-r--r--internal/gitaly/service/commit/list_last_commits_for_tree.go3
-rw-r--r--internal/gitaly/service/commit/server.go20
-rw-r--r--internal/gitaly/service/commit/testhelper_test.go12
-rw-r--r--internal/gitaly/service/commit/tree_entries.go2
-rw-r--r--internal/gitaly/service/commit/tree_entry.go2
-rw-r--r--internal/gitaly/service/conflicts/server.go7
-rw-r--r--internal/gitaly/service/conflicts/testhelper_test.go24
-rw-r--r--internal/gitaly/service/dependencies.go7
-rw-r--r--internal/gitaly/service/diff/server.go7
-rw-r--r--internal/gitaly/service/diff/testhelper_test.go10
-rw-r--r--internal/gitaly/service/objectpool/alternates_test.go2
-rw-r--r--internal/gitaly/service/objectpool/create.go2
-rw-r--r--internal/gitaly/service/objectpool/create_test.go6
-rw-r--r--internal/gitaly/service/objectpool/fetch_into_object_pool.go2
-rw-r--r--internal/gitaly/service/objectpool/fetch_into_object_pool_test.go11
-rw-r--r--internal/gitaly/service/objectpool/get.go2
-rw-r--r--internal/gitaly/service/objectpool/get_test.go2
-rw-r--r--internal/gitaly/service/objectpool/link_test.go14
-rw-r--r--internal/gitaly/service/objectpool/reduplicate_test.go2
-rw-r--r--internal/gitaly/service/objectpool/server.go16
-rw-r--r--internal/gitaly/service/objectpool/testhelper_test.go3
-rw-r--r--internal/gitaly/service/operations/branches_test.go20
-rw-r--r--internal/gitaly/service/operations/server.go15
-rw-r--r--internal/gitaly/service/operations/tags.go3
-rw-r--r--internal/gitaly/service/operations/tags_test.go10
-rw-r--r--internal/gitaly/service/operations/testhelper_test.go35
-rw-r--r--internal/gitaly/service/operations/update_with_hooks_test.go4
-rw-r--r--internal/gitaly/service/ref/delete_refs_test.go8
-rw-r--r--internal/gitaly/service/ref/list_new_blobs.go3
-rw-r--r--internal/gitaly/service/ref/list_new_commits.go3
-rw-r--r--internal/gitaly/service/ref/refs.go8
-rw-r--r--internal/gitaly/service/ref/refs_test.go6
-rw-r--r--internal/gitaly/service/ref/remote_branches.go3
-rw-r--r--internal/gitaly/service/ref/server.go13
-rw-r--r--internal/gitaly/service/ref/tag_messages.go3
-rw-r--r--internal/gitaly/service/ref/testhelper_test.go8
-rw-r--r--internal/gitaly/service/remote/fetch_internal_remote_test.go16
-rw-r--r--internal/gitaly/service/remote/server.go13
-rw-r--r--internal/gitaly/service/remote/testhelper_test.go8
-rw-r--r--internal/gitaly/service/remote/update_remote_mirror_test.go32
-rw-r--r--internal/gitaly/service/repository/apply_gitattributes.go2
-rw-r--r--internal/gitaly/service/repository/apply_gitattributes_test.go9
-rw-r--r--internal/gitaly/service/repository/archive.go3
-rw-r--r--internal/gitaly/service/repository/clone_from_pool.go4
-rw-r--r--internal/gitaly/service/repository/clone_from_pool_internal.go4
-rw-r--r--internal/gitaly/service/repository/clone_from_pool_internal_test.go12
-rw-r--r--internal/gitaly/service/repository/fetch_remote_test.go9
-rw-r--r--internal/gitaly/service/repository/fork_test.go10
-rw-r--r--internal/gitaly/service/repository/gc.go2
-rw-r--r--internal/gitaly/service/repository/raw_changes.go2
-rw-r--r--internal/gitaly/service/repository/replicate_test.go12
-rw-r--r--internal/gitaly/service/repository/search_files_test.go23
-rw-r--r--internal/gitaly/service/repository/server.go12
-rw-r--r--internal/gitaly/service/repository/snapshot_test.go5
-rw-r--r--internal/gitaly/service/repository/testhelper_test.go33
-rw-r--r--internal/gitaly/service/setup/register.go77
-rw-r--r--internal/gitaly/service/smarthttp/inforefs_test.go9
-rw-r--r--internal/gitaly/service/ssh/receive_pack_test.go9
-rw-r--r--internal/middleware/commandstatshandler/commandstatshandler_test.go11
-rw-r--r--internal/praefect/info_service_test.go9
-rw-r--r--internal/praefect/replicator_test.go2
-rw-r--r--internal/testhelper/testserver/gitaly.go15
105 files changed, 916 insertions, 413 deletions
diff --git a/cmd/gitaly/main.go b/cmd/gitaly/main.go
index f9d47757b..f5fb50ba4 100644
--- a/cmd/gitaly/main.go
+++ b/cmd/gitaly/main.go
@@ -15,6 +15,7 @@ import (
"gitlab.com/gitlab-org/gitaly/internal/bootstrap/starter"
"gitlab.com/gitlab-org/gitaly/internal/cgroups"
"gitlab.com/gitlab-org/gitaly/internal/git"
+ "gitlab.com/gitlab-org/gitaly/internal/git/catfile"
"gitlab.com/gitlab-org/gitaly/internal/gitaly/config"
"gitlab.com/gitlab-org/gitaly/internal/gitaly/config/sentry"
"gitlab.com/gitlab-org/gitaly/internal/gitaly/hook"
@@ -168,6 +169,9 @@ func run(cfg config.Cfg) error {
gitCmdFactory := git.NewExecCommandFactory(cfg)
prometheus.MustRegister(gitCmdFactory)
+ catfileCache := catfile.NewCache(gitCmdFactory, cfg)
+ prometheus.MustRegister(catfileCache)
+
gitalyServerFactory := server.NewGitalyServerFactory(cfg, registry)
defer gitalyServerFactory.Stop()
@@ -216,6 +220,7 @@ func run(cfg config.Cfg) error {
ClientPool: conns,
GitCmdFactory: gitCmdFactory,
Linguist: ling,
+ CatfileCache: catfileCache,
})
b.RegisterStarter(starter.New(c, srv))
}
diff --git a/internal/git/catfile/batch.go b/internal/git/catfile/batch.go
index a59f74973..f4114079e 100644
--- a/internal/git/catfile/batch.go
+++ b/internal/git/catfile/batch.go
@@ -8,37 +8,6 @@ import (
"github.com/prometheus/client_golang/prometheus"
"gitlab.com/gitlab-org/gitaly/internal/git"
"gitlab.com/gitlab-org/gitaly/internal/git/repository"
- "gitlab.com/gitlab-org/gitaly/internal/metadata"
-)
-
-var catfileCacheCounter = prometheus.NewCounterVec(
- prometheus.CounterOpts{
- Name: "gitaly_catfile_cache_total",
- Help: "Counter of catfile cache hit/miss",
- },
- []string{"type"},
-)
-
-var currentCatfileProcesses = prometheus.NewGauge(
- prometheus.GaugeOpts{
- Name: "gitaly_catfile_processes",
- Help: "Gauge of active catfile processes",
- },
-)
-
-var totalCatfileProcesses = prometheus.NewCounter(
- prometheus.CounterOpts{
- Name: "gitaly_catfile_processes_total",
- Help: "Counter of catfile processes",
- },
-)
-
-var catfileLookupCounter = prometheus.NewCounterVec(
- prometheus.CounterOpts{
- Name: "gitaly_catfile_lookups_total",
- Help: "Git catfile lookups by object type",
- },
- []string{"type"},
)
const (
@@ -46,13 +15,6 @@ const (
SessionIDField = "gitaly-session-id"
)
-func init() {
- prometheus.MustRegister(catfileCacheCounter)
- prometheus.MustRegister(currentCatfileProcesses)
- prometheus.MustRegister(totalCatfileProcesses)
- prometheus.MustRegister(catfileLookupCounter)
-}
-
// Batch abstracts 'git cat-file --batch' and 'git cat-file --batch-check'.
// It lets you retrieve object metadata and raw objects from a Git repo.
//
@@ -136,69 +98,11 @@ func (c *batch) isClosed() bool {
return c.closed
}
-// New returns a new Batch instance. It is important that ctx gets canceled
-// somewhere, because if it doesn't the cat-file processes spawned by
-// New() never terminate.
-func New(ctx context.Context, gitCmdFactory git.CommandFactory, repo repository.GitRepo) (Batch, error) {
- if ctx.Done() == nil {
- panic("empty ctx.Done() in catfile.Batch.New()")
- }
-
- sessionID := metadata.GetValue(ctx, SessionIDField)
- if sessionID == "" {
- c, err := newBatch(ctx, gitCmdFactory, repo)
- if err != nil {
- return nil, err
- }
- return newInstrumentedBatch(c), err
- }
-
- cacheKey := newCacheKey(sessionID, repo)
- requestDone := ctx.Done()
-
- if c, ok := cache.Checkout(cacheKey); ok {
- go returnToCacheWhenDone(requestDone, cache, cacheKey, c)
- return newInstrumentedBatch(c), nil
- }
-
- // if we are using caching, create a fresh context for the new batch
- // and initialize the new batch with a cache key and cancel function
- cacheCtx, cacheCancel := context.WithCancel(context.Background())
- c, err := newBatch(cacheCtx, gitCmdFactory, repo)
- if err != nil {
- cacheCancel()
- return nil, err
- }
-
- c.cancel = cacheCancel
- go returnToCacheWhenDone(requestDone, cache, cacheKey, c)
-
- return newInstrumentedBatch(c), nil
-}
-
-func returnToCacheWhenDone(done <-chan struct{}, bc *batchCache, cacheKey key, c *batch) {
- <-done
-
- if c == nil || c.isClosed() {
- return
- }
-
- if c.hasUnreadData() {
- catfileCacheCounter.WithLabelValues("dirty").Inc()
- c.Close()
- return
- }
-
- bc.Add(cacheKey, c)
-}
-
-var injectSpawnErrors = false
-
type simulatedBatchSpawnError struct{}
func (simulatedBatchSpawnError) Error() string { return "simulated spawn error" }
-func newBatch(ctx context.Context, gitCmdFactory git.CommandFactory, repo repository.GitRepo) (_ *batch, err error) {
+func (bc *BatchCache) newBatch(ctx context.Context, repo repository.GitRepo) (_ *batch, err error) {
ctx, cancel := context.WithCancel(ctx)
defer func() {
if err != nil {
@@ -206,12 +110,12 @@ func newBatch(ctx context.Context, gitCmdFactory git.CommandFactory, repo reposi
}
}()
- batchProcess, err := newBatchProcess(ctx, gitCmdFactory, repo)
+ batchProcess, err := bc.newBatchProcess(ctx, repo)
if err != nil {
return nil, err
}
- batchCheckProcess, err := newBatchCheckProcess(ctx, gitCmdFactory, repo)
+ batchCheckProcess, err := bc.newBatchCheckProcess(ctx, repo)
if err != nil {
return nil, err
}
@@ -219,19 +123,20 @@ func newBatch(ctx context.Context, gitCmdFactory git.CommandFactory, repo reposi
return &batch{batchProcess: batchProcess, batchCheckProcess: batchCheckProcess}, nil
}
-func newInstrumentedBatch(c Batch) Batch {
- return &instrumentedBatch{c}
+func newInstrumentedBatch(c Batch, catfileLookupCounter *prometheus.CounterVec) Batch {
+ return &instrumentedBatch{c, catfileLookupCounter}
}
type instrumentedBatch struct {
Batch
+ catfileLookupCounter *prometheus.CounterVec
}
func (ib *instrumentedBatch) Info(ctx context.Context, revision git.Revision) (*ObjectInfo, error) {
span, ctx := opentracing.StartSpanFromContext(ctx, "Batch.Info", opentracing.Tag{"revision", revision})
defer span.Finish()
- catfileLookupCounter.WithLabelValues("info").Inc()
+ ib.catfileLookupCounter.WithLabelValues("info").Inc()
return ib.Batch.Info(ctx, revision)
}
@@ -240,7 +145,7 @@ func (ib *instrumentedBatch) Tree(ctx context.Context, revision git.Revision) (*
span, ctx := opentracing.StartSpanFromContext(ctx, "Batch.Tree", opentracing.Tag{"revision", revision})
defer span.Finish()
- catfileLookupCounter.WithLabelValues("tree").Inc()
+ ib.catfileLookupCounter.WithLabelValues("tree").Inc()
return ib.Batch.Tree(ctx, revision)
}
@@ -249,7 +154,7 @@ func (ib *instrumentedBatch) Commit(ctx context.Context, revision git.Revision)
span, ctx := opentracing.StartSpanFromContext(ctx, "Batch.Commit", opentracing.Tag{"revision", revision})
defer span.Finish()
- catfileLookupCounter.WithLabelValues("commit").Inc()
+ ib.catfileLookupCounter.WithLabelValues("commit").Inc()
return ib.Batch.Commit(ctx, revision)
}
@@ -258,7 +163,7 @@ func (ib *instrumentedBatch) Blob(ctx context.Context, revision git.Revision) (*
span, ctx := opentracing.StartSpanFromContext(ctx, "Batch.Blob", opentracing.Tag{"revision", revision})
defer span.Finish()
- catfileLookupCounter.WithLabelValues("blob").Inc()
+ ib.catfileLookupCounter.WithLabelValues("blob").Inc()
return ib.Batch.Blob(ctx, revision)
}
@@ -267,7 +172,7 @@ func (ib *instrumentedBatch) Tag(ctx context.Context, revision git.Revision) (*O
span, ctx := opentracing.StartSpanFromContext(ctx, "Batch.Tag", opentracing.Tag{"revision", revision})
defer span.Finish()
- catfileLookupCounter.WithLabelValues("tag").Inc()
+ ib.catfileLookupCounter.WithLabelValues("tag").Inc()
return ib.Batch.Tag(ctx, revision)
}
diff --git a/internal/git/catfile/batch_cache.go b/internal/git/catfile/batch_cache.go
index 4e92954d5..998f3460a 100644
--- a/internal/git/catfile/batch_cache.go
+++ b/internal/git/catfile/batch_cache.go
@@ -1,19 +1,21 @@
package catfile
import (
+ "context"
"strings"
"sync"
"time"
"github.com/prometheus/client_golang/prometheus"
- "github.com/prometheus/client_golang/prometheus/promauto"
+ "gitlab.com/gitlab-org/gitaly/internal/git"
"gitlab.com/gitlab-org/gitaly/internal/git/repository"
"gitlab.com/gitlab-org/gitaly/internal/gitaly/config"
+ "gitlab.com/gitlab-org/gitaly/internal/metadata"
)
const (
- // DefaultBatchfileTTL is the default ttl for batch files to live in the cache
- DefaultBatchfileTTL = 10 * time.Second
+ // defaultBatchfileTTL is the default ttl for batch files to live in the cache
+ defaultBatchfileTTL = 10 * time.Second
defaultEvictionInterval = 1 * time.Second
@@ -21,20 +23,13 @@ const (
defaultMaxLen = 100
)
-var catfileCacheMembers = promauto.NewGauge(
- prometheus.GaugeOpts{
- Name: "gitaly_catfile_cache_members",
- Help: "Gauge of catfile cache members",
- },
-)
-
-var cache *batchCache
-
-func init() {
- config.RegisterHook(func(cfg *config.Cfg) error {
- cache = newCache(DefaultBatchfileTTL, cfg.Git.CatfileCacheSize)
- return nil
- })
+// Cache is a cache for git-cat-file(1) processes.
+type Cache interface {
+ // BatchProcess either creates a new git-cat-file(1) process or returns a cached one for
+ // the given repository.
+ BatchProcess(context.Context, repository.GitRepo) (Batch, error)
+ // Evict evicts all cached processes from the cache.
+ Evict()
}
func newCacheKey(sessionID string, repo repository.GitRepo) key {
@@ -61,11 +56,13 @@ type entry struct {
expiry time.Time
}
-// batchCache entries always get added to the back of the list. If the
+// BatchCache entries always get added to the back of the list. If the
// list gets too long, we evict entries from the front of the list. When
// an entry gets added it gets an expiry time based on a fixed TTL. A
// monitor goroutine periodically evicts expired entries.
-type batchCache struct {
+type BatchCache struct {
+ gitCmdFactory git.CommandFactory
+
entries []*entry
sync.Mutex
@@ -74,42 +71,158 @@ type batchCache struct {
// ttl is the fixed ttl for cache entries
ttl time.Duration
+
+ // injectSpawnErrors is used for testing purposes only. If set to true, then spawned batch
+ // processes will simulate spawn errors.
+ injectSpawnErrors bool
+
+ catfileCacheCounter *prometheus.CounterVec
+ currentCatfileProcesses prometheus.Gauge
+ totalCatfileProcesses prometheus.Counter
+ catfileLookupCounter *prometheus.CounterVec
+ catfileCacheMembers prometheus.Gauge
}
-func newCache(ttl time.Duration, maxLen int) *batchCache {
- return newCacheWithRefresh(ttl, maxLen, defaultEvictionInterval)
+// NewCache creates a new catfile process cache.
+func NewCache(gitCmdFactory git.CommandFactory, cfg config.Cfg) *BatchCache {
+ return newCache(gitCmdFactory,
+ defaultBatchfileTTL,
+ cfg.Git.CatfileCacheSize,
+ defaultEvictionInterval,
+ )
}
-func newCacheWithRefresh(ttl time.Duration, maxLen int, refreshInterval time.Duration) *batchCache {
+func newCache(gitCmdFactory git.CommandFactory, ttl time.Duration, maxLen int, refreshInterval time.Duration) *BatchCache {
if maxLen <= 0 {
maxLen = defaultMaxLen
}
- bc := &batchCache{
- maxLen: maxLen,
- ttl: ttl,
+ bc := &BatchCache{
+ gitCmdFactory: gitCmdFactory,
+ maxLen: maxLen,
+ ttl: ttl,
+ catfileCacheCounter: prometheus.NewCounterVec(
+ prometheus.CounterOpts{
+ Name: "gitaly_catfile_cache_total",
+ Help: "Counter of catfile cache hit/miss",
+ },
+ []string{"type"},
+ ),
+ currentCatfileProcesses: prometheus.NewGauge(
+ prometheus.GaugeOpts{
+ Name: "gitaly_catfile_processes",
+ Help: "Gauge of active catfile processes",
+ },
+ ),
+ totalCatfileProcesses: prometheus.NewCounter(
+ prometheus.CounterOpts{
+ Name: "gitaly_catfile_processes_total",
+ Help: "Counter of catfile processes",
+ },
+ ),
+ catfileLookupCounter: prometheus.NewCounterVec(
+ prometheus.CounterOpts{
+ Name: "gitaly_catfile_lookups_total",
+ Help: "Git catfile lookups by object type",
+ },
+ []string{"type"},
+ ),
+ catfileCacheMembers: prometheus.NewGauge(
+ prometheus.GaugeOpts{
+ Name: "gitaly_catfile_cache_members",
+ Help: "Gauge of catfile cache members",
+ },
+ ),
}
go bc.monitor(refreshInterval)
return bc
}
-func (bc *batchCache) monitor(refreshInterval time.Duration) {
+// Describe describes all metrics exposed by BatchCache.
+func (bc *BatchCache) Describe(descs chan<- *prometheus.Desc) {
+ prometheus.DescribeByCollect(bc, descs)
+}
+
+// Collect collects all metrics exposed by BatchCache.
+func (bc *BatchCache) Collect(metrics chan<- prometheus.Metric) {
+ bc.catfileCacheCounter.Collect(metrics)
+ bc.currentCatfileProcesses.Collect(metrics)
+ bc.totalCatfileProcesses.Collect(metrics)
+ bc.catfileLookupCounter.Collect(metrics)
+ bc.catfileCacheMembers.Collect(metrics)
+}
+
+func (bc *BatchCache) monitor(refreshInterval time.Duration) {
ticker := time.NewTicker(refreshInterval)
for range ticker.C {
- bc.EnforceTTL(time.Now())
+ bc.enforceTTL(time.Now())
+ }
+}
+
+// BatchProcess creates a new Batch process for the given repository.
+func (bc *BatchCache) BatchProcess(ctx context.Context, repo repository.GitRepo) (Batch, error) {
+ if ctx.Done() == nil {
+ panic("empty ctx.Done() in catfile.Batch.New()")
+ }
+
+ sessionID := metadata.GetValue(ctx, SessionIDField)
+ if sessionID == "" {
+ c, err := bc.newBatch(ctx, repo)
+ if err != nil {
+ return nil, err
+ }
+ return newInstrumentedBatch(c, bc.catfileLookupCounter), err
}
+
+ cacheKey := newCacheKey(sessionID, repo)
+ requestDone := ctx.Done()
+
+ if c, ok := bc.checkout(cacheKey); ok {
+ go bc.returnWhenDone(requestDone, cacheKey, c)
+ return newInstrumentedBatch(c, bc.catfileLookupCounter), nil
+ }
+
+ // if we are using caching, create a fresh context for the new batch
+ // and initialize the new batch with a bc key and cancel function
+ cacheCtx, cacheCancel := context.WithCancel(context.Background())
+ c, err := bc.newBatch(cacheCtx, repo)
+ if err != nil {
+ cacheCancel()
+ return nil, err
+ }
+
+ c.cancel = cacheCancel
+ go bc.returnWhenDone(requestDone, cacheKey, c)
+
+ return newInstrumentedBatch(c, bc.catfileLookupCounter), nil
+}
+
+func (bc *BatchCache) returnWhenDone(done <-chan struct{}, cacheKey key, c *batch) {
+ <-done
+
+ if c == nil || c.isClosed() {
+ return
+ }
+
+ if c.hasUnreadData() {
+ bc.catfileCacheCounter.WithLabelValues("dirty").Inc()
+ c.Close()
+ return
+ }
+
+ bc.add(cacheKey, c)
}
-// Add adds a key, value pair to bc. If there are too many keys in bc
-// already Add will evict old keys until the length is OK again.
-func (bc *batchCache) Add(k key, b *batch) {
+// add adds a key, value pair to bc. If there are too many keys in bc
+// already add will evict old keys until the length is OK again.
+func (bc *BatchCache) add(k key, b *batch) {
bc.Lock()
defer bc.Unlock()
if i, ok := bc.lookup(k); ok {
- catfileCacheCounter.WithLabelValues("duplicate").Inc()
+ bc.catfileCacheCounter.WithLabelValues("duplicate").Inc()
bc.delete(i, true)
}
@@ -120,34 +233,34 @@ func (bc *batchCache) Add(k key, b *batch) {
bc.evictHead()
}
- catfileCacheMembers.Set(float64(bc.len()))
+ bc.catfileCacheMembers.Set(float64(bc.len()))
}
-func (bc *batchCache) head() *entry { return bc.entries[0] }
-func (bc *batchCache) evictHead() { bc.delete(0, true) }
-func (bc *batchCache) len() int { return len(bc.entries) }
+func (bc *BatchCache) head() *entry { return bc.entries[0] }
+func (bc *BatchCache) evictHead() { bc.delete(0, true) }
+func (bc *BatchCache) len() int { return len(bc.entries) }
-// Checkout removes a value from bc. After use the caller can re-add the value with bc.Add.
-func (bc *batchCache) Checkout(k key) (*batch, bool) {
+// checkout removes a value from bc. After use the caller can re-add the value with bc.Add.
+func (bc *BatchCache) checkout(k key) (*batch, bool) {
bc.Lock()
defer bc.Unlock()
i, ok := bc.lookup(k)
if !ok {
- catfileCacheCounter.WithLabelValues("miss").Inc()
+ bc.catfileCacheCounter.WithLabelValues("miss").Inc()
return nil, false
}
- catfileCacheCounter.WithLabelValues("hit").Inc()
+ bc.catfileCacheCounter.WithLabelValues("hit").Inc()
ent := bc.entries[i]
bc.delete(i, false)
return ent.value, true
}
-// EnforceTTL evicts all entries older than now, assuming the entry
+// enforceTTL evicts all entries older than now, assuming the entry
// expiry times are increasing.
-func (bc *batchCache) EnforceTTL(now time.Time) {
+func (bc *BatchCache) enforceTTL(now time.Time) {
bc.Lock()
defer bc.Unlock()
@@ -156,7 +269,8 @@ func (bc *batchCache) EnforceTTL(now time.Time) {
}
}
-func (bc *batchCache) EvictAll() {
+// Evict evicts all cached processes from the cache.
+func (bc *BatchCache) Evict() {
bc.Lock()
defer bc.Unlock()
@@ -165,12 +279,7 @@ func (bc *batchCache) EvictAll() {
}
}
-// ExpireAll is used to expire all of the batches in the cache
-func ExpireAll() {
- cache.EvictAll()
-}
-
-func (bc *batchCache) lookup(k key) (int, bool) {
+func (bc *BatchCache) lookup(k key) (int, bool) {
for i, ent := range bc.entries {
if ent.key == k {
return i, true
@@ -180,7 +289,7 @@ func (bc *batchCache) lookup(k key) (int, bool) {
return -1, false
}
-func (bc *batchCache) delete(i int, wantClose bool) {
+func (bc *BatchCache) delete(i int, wantClose bool) {
ent := bc.entries[i]
if wantClose {
@@ -188,5 +297,5 @@ func (bc *batchCache) delete(i int, wantClose bool) {
}
bc.entries = append(bc.entries[:i], bc.entries[i+1:]...)
- catfileCacheMembers.Set(float64(bc.len()))
+ bc.catfileCacheMembers.Set(float64(bc.len()))
}
diff --git a/internal/git/catfile/batch_cache_test.go b/internal/git/catfile/batch_cache_test.go
index 4d22a898c..f13e18d13 100644
--- a/internal/git/catfile/batch_cache_test.go
+++ b/internal/git/catfile/batch_cache_test.go
@@ -6,29 +6,33 @@ import (
"time"
"github.com/stretchr/testify/require"
+ "gitlab.com/gitlab-org/gitaly/internal/git"
+ "gitlab.com/gitlab-org/gitaly/internal/testhelper/testcfg"
)
func TestCacheAdd(t *testing.T) {
+ cfg := testcfg.Build(t)
+
const maxLen = 3
- bc := newCache(time.Hour, maxLen)
+ bc := newCache(git.NewExecCommandFactory(cfg), time.Hour, maxLen, defaultEvictionInterval)
key0 := testKey(0)
value0 := testValue()
- bc.Add(key0, value0)
+ bc.add(key0, value0)
requireCacheValid(t, bc)
key1 := testKey(1)
- bc.Add(key1, testValue())
+ bc.add(key1, testValue())
requireCacheValid(t, bc)
key2 := testKey(2)
- bc.Add(key2, testValue())
+ bc.add(key2, testValue())
requireCacheValid(t, bc)
// Because maxLen is 3, and key0 is oldest, we expect that adding key3
// will kick out key0.
key3 := testKey(3)
- bc.Add(key3, testValue())
+ bc.add(key3, testValue())
requireCacheValid(t, bc)
require.Equal(t, maxLen, bc.len(), "length should be maxLen")
@@ -37,21 +41,23 @@ func TestCacheAdd(t *testing.T) {
}
func TestCacheAddTwice(t *testing.T) {
- bc := newCache(time.Hour, 10)
+ cfg := testcfg.Build(t)
+
+ bc := newCache(git.NewExecCommandFactory(cfg), time.Hour, 10, defaultEvictionInterval)
key0 := testKey(0)
value0 := testValue()
- bc.Add(key0, value0)
+ bc.add(key0, value0)
requireCacheValid(t, bc)
key1 := testKey(1)
- bc.Add(key1, testValue())
+ bc.add(key1, testValue())
requireCacheValid(t, bc)
require.Equal(t, key0, bc.head().key, "key0 should be oldest key")
value2 := testValue()
- bc.Add(key0, value2)
+ bc.add(key0, value2)
requireCacheValid(t, bc)
require.Equal(t, key1, bc.head().key, "key1 should be oldest key")
@@ -61,18 +67,20 @@ func TestCacheAddTwice(t *testing.T) {
}
func TestCacheCheckout(t *testing.T) {
- bc := newCache(time.Hour, 10)
+ cfg := testcfg.Build(t)
+
+ bc := newCache(git.NewExecCommandFactory(cfg), time.Hour, 10, defaultEvictionInterval)
key0 := testKey(0)
value0 := testValue()
- bc.Add(key0, value0)
+ bc.add(key0, value0)
- v, ok := bc.Checkout(key{sessionID: "foo"})
+ v, ok := bc.checkout(key{sessionID: "foo"})
requireCacheValid(t, bc)
require.Nil(t, v, "expect nil value when key not found")
require.False(t, ok, "ok flag")
- v, ok = bc.Checkout(key0)
+ v, ok = bc.checkout(key0)
requireCacheValid(t, bc)
require.Equal(t, value0, v)
@@ -80,42 +88,44 @@ func TestCacheCheckout(t *testing.T) {
require.False(t, v.isClosed(), "value should not be closed after checkout")
- v, ok = bc.Checkout(key0)
+ v, ok = bc.checkout(key0)
require.False(t, ok, "ok flag after second checkout")
require.Nil(t, v, "value from second checkout")
}
func TestCacheEnforceTTL(t *testing.T) {
+ cfg := testcfg.Build(t)
+
ttl := time.Hour
- bc := newCache(ttl, 10)
+ bc := newCache(git.NewExecCommandFactory(cfg), ttl, 10, defaultEvictionInterval)
sleep := func() { time.Sleep(2 * time.Millisecond) }
key0 := testKey(0)
value0 := testValue()
- bc.Add(key0, value0)
+ bc.add(key0, value0)
sleep()
key1 := testKey(1)
value1 := testValue()
- bc.Add(key1, value1)
+ bc.add(key1, value1)
sleep()
cutoff := time.Now().Add(ttl)
sleep()
key2 := testKey(2)
- bc.Add(key2, testValue())
+ bc.add(key2, testValue())
sleep()
key3 := testKey(3)
- bc.Add(key3, testValue())
+ bc.add(key3, testValue())
sleep()
requireCacheValid(t, bc)
// We expect this cutoff to cause eviction of key0 and key1 but no other keys.
- bc.EnforceTTL(cutoff)
+ bc.enforceTTL(cutoff)
requireCacheValid(t, bc)
@@ -125,20 +135,22 @@ func TestCacheEnforceTTL(t *testing.T) {
require.Equal(t, []key{key2, key3}, keys(bc), "remaining keys after EnforceTTL")
- bc.EnforceTTL(cutoff)
+ bc.enforceTTL(cutoff)
requireCacheValid(t, bc)
require.Equal(t, []key{key2, key3}, keys(bc), "remaining keys after second EnforceTTL")
}
func TestAutoExpiry(t *testing.T) {
+ cfg := testcfg.Build(t)
+
ttl := 5 * time.Millisecond
refresh := 1 * time.Millisecond
- bc := newCacheWithRefresh(ttl, 10, refresh)
+ bc := newCache(git.NewExecCommandFactory(cfg), ttl, 10, refresh)
key0 := testKey(0)
value0 := testValue()
- bc.Add(key0, value0)
+ bc.add(key0, value0)
requireCacheValid(t, bc)
require.Contains(t, keys(bc), key0, "key should still be in map")
@@ -157,7 +169,7 @@ func TestAutoExpiry(t *testing.T) {
require.True(t, value0.isClosed(), "value should be closed after eviction")
}
-func requireCacheValid(t *testing.T, bc *batchCache) {
+func requireCacheValid(t *testing.T, bc *BatchCache) {
bc.Lock()
defer bc.Unlock()
@@ -171,7 +183,7 @@ func testValue() *batch { return &batch{} }
func testKey(i int) key { return key{sessionID: fmt.Sprintf("key-%d", i)} }
-func keys(bc *batchCache) []key {
+func keys(bc *BatchCache) []key {
bc.Lock()
defer bc.Unlock()
diff --git a/internal/git/catfile/batch_check_process.go b/internal/git/catfile/batch_check_process.go
index 2d1d7d637..16b46e006 100644
--- a/internal/git/catfile/batch_check_process.go
+++ b/internal/git/catfile/batch_check_process.go
@@ -20,18 +20,18 @@ type batchCheckProcess struct {
sync.Mutex
}
-func newBatchCheckProcess(ctx context.Context, gitCmdFactory git.CommandFactory, repo repository.GitRepo) (*batchCheckProcess, error) {
- bc := &batchCheckProcess{}
+func (bc *BatchCache) newBatchCheckProcess(ctx context.Context, repo repository.GitRepo) (*batchCheckProcess, error) {
+ process := &batchCheckProcess{}
var stdinReader io.Reader
- stdinReader, bc.w = io.Pipe()
+ stdinReader, process.w = io.Pipe()
// batch processes are long-lived and reused across RPCs,
// so we de-correlate the process from the RPC
ctx = correlation.ContextWithCorrelation(ctx, "")
ctx = opentracing.ContextWithSpan(ctx, nil)
- batchCmd, err := gitCmdFactory.New(ctx, repo,
+ batchCmd, err := bc.gitCmdFactory.New(ctx, repo,
git.SubCmd{
Name: "cat-file",
Flags: []git.Option{
@@ -44,19 +44,19 @@ func newBatchCheckProcess(ctx context.Context, gitCmdFactory git.CommandFactory,
return nil, err
}
- bc.r = bufio.NewReader(batchCmd)
+ process.r = bufio.NewReader(batchCmd)
go func() {
<-ctx.Done()
// This is crucial to prevent leaking file descriptors.
- bc.w.Close()
+ process.w.Close()
}()
- if injectSpawnErrors {
+ if bc.injectSpawnErrors {
// Testing only: intentionally leak process
return nil, &simulatedBatchSpawnError{}
}
- return bc, nil
+ return process, nil
}
func (bc *batchCheckProcess) info(revision git.Revision) (*ObjectInfo, error) {
diff --git a/internal/git/catfile/batch_process.go b/internal/git/catfile/batch_process.go
index f34292cb1..79b8ef59f 100644
--- a/internal/git/catfile/batch_process.go
+++ b/internal/git/catfile/batch_process.go
@@ -33,8 +33,8 @@ type batchProcess struct {
sync.Mutex
}
-func newBatchProcess(ctx context.Context, gitCmdFactory git.CommandFactory, repo repository.GitRepo) (*batchProcess, error) {
- totalCatfileProcesses.Inc()
+func (bc *BatchCache) newBatchProcess(ctx context.Context, repo repository.GitRepo) (*batchProcess, error) {
+ bc.totalCatfileProcesses.Inc()
b := &batchProcess{}
var stdinReader io.Reader
@@ -45,7 +45,7 @@ func newBatchProcess(ctx context.Context, gitCmdFactory git.CommandFactory, repo
ctx = correlation.ContextWithCorrelation(ctx, "")
ctx = opentracing.ContextWithSpan(ctx, nil)
- batchCmd, err := gitCmdFactory.New(ctx, repo,
+ batchCmd, err := bc.gitCmdFactory.New(ctx, repo,
git.SubCmd{
Name: "cat-file",
Flags: []git.Option{
@@ -60,15 +60,15 @@ func newBatchProcess(ctx context.Context, gitCmdFactory git.CommandFactory, repo
b.r = bufio.NewReader(batchCmd)
- currentCatfileProcesses.Inc()
+ bc.currentCatfileProcesses.Inc()
go func() {
<-ctx.Done()
// This Close() is crucial to prevent leaking file descriptors.
b.w.Close()
- currentCatfileProcesses.Dec()
+ bc.currentCatfileProcesses.Dec()
}()
- if injectSpawnErrors {
+ if bc.injectSpawnErrors {
// Testing only: intentionally leak process
return nil, &simulatedBatchSpawnError{}
}
diff --git a/internal/git/catfile/batch_test.go b/internal/git/catfile/batch_test.go
index 570e878d5..cc3c1ff0e 100644
--- a/internal/git/catfile/batch_test.go
+++ b/internal/git/catfile/batch_test.go
@@ -14,10 +14,10 @@ import (
"github.com/stretchr/testify/require"
"gitlab.com/gitlab-org/gitaly/internal/command"
"gitlab.com/gitlab-org/gitaly/internal/git"
+ "gitlab.com/gitlab-org/gitaly/internal/git/repository"
"gitlab.com/gitlab-org/gitaly/internal/helper/text"
"gitlab.com/gitlab-org/gitaly/internal/testhelper"
"gitlab.com/gitlab-org/gitaly/internal/testhelper/testcfg"
- "gitlab.com/gitlab-org/gitaly/proto/go/gitalypb"
"google.golang.org/grpc/metadata"
)
@@ -26,10 +26,11 @@ func setupBatch(t *testing.T, ctx context.Context) Batch {
cfg, repo, _ := testcfg.BuildWithRepo(t)
- c, err := New(ctx, git.NewExecCommandFactory(cfg), repo)
+ cache := newCache(git.NewExecCommandFactory(cfg), 1*time.Hour, 1000, defaultEvictionInterval)
+ batch, err := cache.BatchProcess(ctx, repo)
require.NoError(t, err)
- return c
+ return batch
}
func TestInfo(t *testing.T) {
@@ -330,14 +331,9 @@ func TestRepeatedCalls(t *testing.T) {
}
func TestSpawnFailure(t *testing.T) {
- defer func() { injectSpawnErrors = false }()
-
- // reset global cache
- defer func(old *batchCache) { cache = old }(cache)
+ cfg, testRepo, _ := testcfg.BuildWithRepo(t)
- // Use very high values to effectively disable auto-expiry
- cache = newCache(1*time.Hour, 1000)
- defer cache.EvictAll()
+ cache := newCache(git.NewExecCommandFactory(cfg), 1*time.Hour, 1000, defaultEvictionInterval)
require.True(
t,
@@ -349,12 +345,7 @@ func TestSpawnFailure(t *testing.T) {
ctx1, cancel1 := testhelper.Context()
defer cancel1()
- cfg, testRepo, _ := testcfg.BuildWithRepo(t)
-
- gitCmdFactory := git.NewExecCommandFactory(cfg)
-
- injectSpawnErrors = false
- _, err := catfileWithFreshSessionID(ctx1, gitCmdFactory, testRepo)
+ _, err := catfileWithFreshSessionID(ctx1, cache, testRepo)
require.NoError(t, err, "catfile spawn should succeed in normal circumstances")
require.Equal(t, 2, numGitChildren(t), "there should be 2 git child processes")
@@ -369,7 +360,7 @@ func TestSpawnFailure(t *testing.T) {
require.Equal(t, 2, numGitChildren(t), "there should still be 2 git child processes")
- cache.EvictAll()
+ cache.Evict()
require.Equal(t, 0, cacheSize(cache), "the cache should be empty now")
require.True(
@@ -381,8 +372,8 @@ func TestSpawnFailure(t *testing.T) {
ctx2, cancel2 := testhelper.Context()
defer cancel2()
- injectSpawnErrors = true
- _, err = catfileWithFreshSessionID(ctx2, gitCmdFactory, testRepo)
+ cache.injectSpawnErrors = true
+ _, err = catfileWithFreshSessionID(ctx2, cache, testRepo)
require.Error(t, err, "expect simulated error")
require.IsType(t, &simulatedBatchSpawnError{}, err)
@@ -393,7 +384,7 @@ func TestSpawnFailure(t *testing.T) {
)
}
-func catfileWithFreshSessionID(ctx context.Context, gitCmdFactory git.CommandFactory, repo *gitalypb.Repository) (Batch, error) {
+func catfileWithFreshSessionID(ctx context.Context, cache Cache, repo repository.GitRepo) (Batch, error) {
id, err := text.RandomHex(4)
if err != nil {
return nil, err
@@ -403,7 +394,7 @@ func catfileWithFreshSessionID(ctx context.Context, gitCmdFactory git.CommandFac
SessionIDField: id,
})
- return New(metadata.NewIncomingContext(ctx, md), gitCmdFactory, repo)
+ return cache.BatchProcess(metadata.NewIncomingContext(ctx, md), repo)
}
func waitTrue(callback func() bool) bool {
@@ -431,7 +422,7 @@ func numGitChildren(t *testing.T) int {
return bytes.Count(out, []byte("\n"))
}
-func cacheSize(bc *batchCache) int {
+func cacheSize(bc *BatchCache) int {
bc.Lock()
defer bc.Unlock()
return bc.len()
diff --git a/internal/git/localrepo/config_test.go b/internal/git/localrepo/config_test.go
index f44bf682e..513e2dbdf 100644
--- a/internal/git/localrepo/config_test.go
+++ b/internal/git/localrepo/config_test.go
@@ -9,6 +9,7 @@ import (
"github.com/stretchr/testify/require"
"gitlab.com/gitlab-org/gitaly/internal/git"
+ "gitlab.com/gitlab-org/gitaly/internal/git/catfile"
"gitlab.com/gitlab-org/gitaly/internal/git/gittest"
"gitlab.com/gitlab-org/gitaly/internal/helper/text"
"gitlab.com/gitlab-org/gitaly/internal/testhelper"
@@ -23,7 +24,9 @@ func setupRepoConfig(t *testing.T) (Config, string) {
repoProto, repoPath, cleanup := gittest.InitBareRepoAt(t, cfg, cfg.Storages[0])
t.Cleanup(cleanup)
- repo := New(git.NewExecCommandFactory(cfg), repoProto, cfg)
+ gitCmdFactory := git.NewExecCommandFactory(cfg)
+ repo := New(gitCmdFactory, catfile.NewCache(gitCmdFactory, cfg), repoProto, cfg)
+
return repo.Config(), repoPath
}
diff --git a/internal/git/localrepo/objects.go b/internal/git/localrepo/objects.go
index 312b3f593..f53ddc507 100644
--- a/internal/git/localrepo/objects.go
+++ b/internal/git/localrepo/objects.go
@@ -11,7 +11,6 @@ import (
"gitlab.com/gitlab-org/gitaly/internal/command"
"gitlab.com/gitlab-org/gitaly/internal/git"
- "gitlab.com/gitlab-org/gitaly/internal/git/catfile"
"gitlab.com/gitlab-org/gitaly/internal/git/log"
"gitlab.com/gitlab-org/gitaly/internal/helper/text"
"gitlab.com/gitlab-org/gitaly/proto/go/gitalypb"
@@ -227,7 +226,7 @@ func (repo *Repo) ReadCommit(ctx context.Context, revision git.Revision, opts ..
opt(&cfg)
}
- c, err := catfile.New(ctx, repo.gitCmdFactory, repo)
+ c, err := repo.catfileCache.BatchProcess(ctx, repo)
if err != nil {
return nil, err
}
diff --git a/internal/git/localrepo/objects_test.go b/internal/git/localrepo/objects_test.go
index 18d18ca84..494be4973 100644
--- a/internal/git/localrepo/objects_test.go
+++ b/internal/git/localrepo/objects_test.go
@@ -14,6 +14,7 @@ import (
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
"gitlab.com/gitlab-org/gitaly/internal/git"
+ "gitlab.com/gitlab-org/gitaly/internal/git/catfile"
"gitlab.com/gitlab-org/gitaly/internal/git/gittest"
"gitlab.com/gitlab-org/gitaly/internal/helper/text"
"gitlab.com/gitlab-org/gitaly/internal/testhelper"
@@ -36,7 +37,8 @@ func setupRepo(t *testing.T, bare bool) (*Repo, string) {
}
t.Cleanup(repoCleanUp)
- return New(git.NewExecCommandFactory(cfg), repoProto, cfg), repoPath
+ gitCmdFactory := git.NewExecCommandFactory(cfg)
+ return New(gitCmdFactory, catfile.NewCache(gitCmdFactory, cfg), repoProto, cfg), repoPath
}
type ReaderFunc func([]byte) (int, error)
diff --git a/internal/git/localrepo/refs_test.go b/internal/git/localrepo/refs_test.go
index 1620a0729..baacfa8be 100644
--- a/internal/git/localrepo/refs_test.go
+++ b/internal/git/localrepo/refs_test.go
@@ -6,6 +6,7 @@ import (
"github.com/stretchr/testify/require"
"gitlab.com/gitlab-org/gitaly/internal/git"
+ "gitlab.com/gitlab-org/gitaly/internal/git/catfile"
"gitlab.com/gitlab-org/gitaly/internal/git/gittest"
"gitlab.com/gitlab-org/gitaly/internal/helper/text"
"gitlab.com/gitlab-org/gitaly/internal/testhelper"
@@ -236,8 +237,10 @@ func TestRepo_GetRemoteReferences(t *testing.T) {
annotatedTagOID := text.ChompBytes(testhelper.MustRunCommand(t, nil, "git", "-C", repoPath, "rev-parse", "annotated-tag"))
+ gitCmdFactory := git.NewExecCommandFactory(cfg)
repo := New(
- git.NewExecCommandFactory(cfg),
+ gitCmdFactory,
+ catfile.NewCache(gitCmdFactory, cfg),
&gitalypb.Repository{StorageName: "default", RelativePath: filepath.Join(relativePath, ".git")},
cfg,
)
@@ -398,7 +401,7 @@ func TestRepo_UpdateRef(t *testing.T) {
t.Run(tc.desc, func(t *testing.T) {
// Re-create repo for each testcase.
repoProto, _, _ := gittest.CloneRepoAtStorage(t, repo.cfg.Storages[0], t.Name())
- repo := New(repo.gitCmdFactory, repoProto, repo.cfg)
+ repo := New(repo.gitCmdFactory, repo.catfileCache, repoProto, repo.cfg)
err := repo.UpdateRef(ctx, git.ReferenceName(tc.ref), tc.newValue, tc.oldValue)
tc.verify(t, repo, err)
})
diff --git a/internal/git/localrepo/remote_test.go b/internal/git/localrepo/remote_test.go
index b53f0075a..a924a0263 100644
--- a/internal/git/localrepo/remote_test.go
+++ b/internal/git/localrepo/remote_test.go
@@ -13,6 +13,7 @@ import (
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
"gitlab.com/gitlab-org/gitaly/internal/git"
+ "gitlab.com/gitlab-org/gitaly/internal/git/catfile"
"gitlab.com/gitlab-org/gitaly/internal/git/gittest"
"gitlab.com/gitlab-org/gitaly/internal/gitaly/config"
"gitlab.com/gitlab-org/gitaly/internal/helper/text"
@@ -38,13 +39,14 @@ func setupRepoRemote(t *testing.T, bare bool) (Remote, string) {
}
t.Cleanup(repoCleanUp)
- return New(git.NewExecCommandFactory(cfg), repoProto, cfg).Remote(), repoPath
+ gitCmdFactory := git.NewExecCommandFactory(cfg)
+ return New(gitCmdFactory, catfile.NewCache(gitCmdFactory, cfg), repoProto, cfg).Remote(), repoPath
}
func TestRepo_Remote(t *testing.T) {
repository := &gitalypb.Repository{StorageName: "stub", RelativePath: "/stub"}
- repo := New(nil, repository, config.Cfg{})
+ repo := New(nil, nil, repository, config.Cfg{})
require.Equal(t, Remote{repo: repo}, repo.Remote())
}
@@ -317,11 +319,11 @@ func TestRepo_FetchRemote(t *testing.T) {
require.NoError(t, err)
}
- return New(remoteCmd.repo.gitCmdFactory, testRepo, cfg), testRepoPath, cleanup
+ return New(remoteCmd.repo.gitCmdFactory, remoteCmd.repo.catfileCache, testRepo, cfg), testRepoPath, cleanup
}
t.Run("invalid name", func(t *testing.T) {
- repo := New(remoteCmd.repo.gitCmdFactory, nil, cfg)
+ repo := New(remoteCmd.repo.gitCmdFactory, remoteCmd.repo.catfileCache, nil, cfg)
err := repo.FetchRemote(ctx, " ", FetchOpts{})
require.True(t, errors.Is(err, git.ErrInvalidArg))
@@ -329,7 +331,7 @@ func TestRepo_FetchRemote(t *testing.T) {
})
t.Run("unknown remote", func(t *testing.T) {
- repo := New(remoteCmd.repo.gitCmdFactory, remoteCmd.repo, cfg)
+ repo := New(remoteCmd.repo.gitCmdFactory, remoteCmd.repo.catfileCache, remoteCmd.repo, cfg)
var stderr bytes.Buffer
err := repo.FetchRemote(ctx, "stub", FetchOpts{Stderr: &stderr})
require.Error(t, err)
@@ -361,7 +363,7 @@ func TestRepo_FetchRemote(t *testing.T) {
_, sourceRepoPath, _ := gittest.CloneRepoAtStorage(t, cfg.Storages[0], t.Name()+"-1")
testRepo, testRepoPath, _ := gittest.CloneRepoAtStorage(t, cfg.Storages[0], t.Name()+"-2")
- repo := New(remoteCmd.repo.gitCmdFactory, testRepo, cfg)
+ repo := New(remoteCmd.repo.gitCmdFactory, remoteCmd.repo.catfileCache, testRepo, cfg)
testhelper.MustRunCommand(t, nil, "git", "-C", testRepoPath, "remote", "add", "source", sourceRepoPath)
var stderr bytes.Buffer
@@ -373,7 +375,7 @@ func TestRepo_FetchRemote(t *testing.T) {
_, sourceRepoPath, _ := gittest.CloneRepoAtStorage(t, cfg.Storages[0], t.Name()+"-1")
testRepo, testRepoPath, _ := gittest.CloneRepoAtStorage(t, cfg.Storages[0], t.Name()+"-2")
- repo := New(remoteCmd.repo.gitCmdFactory, testRepo, cfg)
+ repo := New(remoteCmd.repo.gitCmdFactory, remoteCmd.repo.catfileCache, testRepo, cfg)
testhelper.MustRunCommand(t, nil, "git", "-C", testRepoPath, "remote", "add", "source", sourceRepoPath)
require.NoError(t, repo.FetchRemote(ctx, "source", FetchOpts{}))
@@ -400,7 +402,7 @@ func TestRepo_FetchRemote(t *testing.T) {
_, sourceRepoPath, _ := gittest.CloneRepoAtStorage(t, cfg.Storages[0], t.Name()+"-1")
testRepo, testRepoPath, _ := gittest.CloneRepoAtStorage(t, cfg.Storages[0], t.Name()+"-2")
- repo := New(remoteCmd.repo.gitCmdFactory, testRepo, cfg)
+ repo := New(remoteCmd.repo.gitCmdFactory, remoteCmd.repo.catfileCache, testRepo, cfg)
testhelper.MustRunCommand(t, nil, "git", "-C", testRepoPath, "remote", "add", "source", sourceRepoPath)
require.NoError(t, repo.FetchRemote(ctx, "source", FetchOpts{}))
@@ -460,7 +462,8 @@ if [ -z ${GIT_SSH_COMMAND+x} ];then rm -f %q ;else echo -n "$GIT_SSH_COMMAND" >
)
cfg.Git.BinPath = gitPath
- sourceRepo := New(git.NewExecCommandFactory(cfg), sourceRepoPb, cfg)
+ gitCmdFactory := git.NewExecCommandFactory(cfg)
+ sourceRepo := New(gitCmdFactory, catfile.NewCache(gitCmdFactory, cfg), sourceRepoPb, cfg)
for _, tc := range []struct {
desc string
@@ -526,7 +529,8 @@ if [ -z ${GIT_SSH_COMMAND+x} ];then rm -f %q ;else echo -n "$GIT_SSH_COMMAND" >
} {
t.Run(tc.desc, func(t *testing.T) {
pushRepoPb, pushRepoPath, _ := gittest.InitBareRepoAt(t, cfg, cfg.Storages[0])
- pushRepo := New(git.NewExecCommandFactory(cfg), pushRepoPb, cfg)
+ gitCmdFactory := git.NewExecCommandFactory(cfg)
+ pushRepo := New(gitCmdFactory, catfile.NewCache(gitCmdFactory, cfg), pushRepoPb, cfg)
if tc.setupPushRepo != nil {
tc.setupPushRepo(t, pushRepo)
diff --git a/internal/git/localrepo/repo.go b/internal/git/localrepo/repo.go
index 28a8fae5c..1610ac198 100644
--- a/internal/git/localrepo/repo.go
+++ b/internal/git/localrepo/repo.go
@@ -7,6 +7,7 @@ import (
"gitlab.com/gitlab-org/gitaly/internal/command"
"gitlab.com/gitlab-org/gitaly/internal/git"
+ "gitlab.com/gitlab-org/gitaly/internal/git/catfile"
"gitlab.com/gitlab-org/gitaly/internal/git/repository"
"gitlab.com/gitlab-org/gitaly/internal/gitaly/config"
"gitlab.com/gitlab-org/gitaly/internal/storage"
@@ -18,14 +19,16 @@ type Repo struct {
gitCmdFactory git.CommandFactory
cfg config.Cfg
locator storage.Locator
+ catfileCache catfile.Cache
}
// New creates a new Repo from its protobuf representation.
-func New(gitCmdFactory git.CommandFactory, repo repository.GitRepo, cfg config.Cfg) *Repo {
+func New(gitCmdFactory git.CommandFactory, catfileCache catfile.Cache, repo repository.GitRepo, cfg config.Cfg) *Repo {
return &Repo{
GitRepo: repo,
cfg: cfg,
gitCmdFactory: gitCmdFactory,
+ catfileCache: catfileCache,
locator: config.NewLocator(cfg),
}
}
@@ -33,7 +36,8 @@ func New(gitCmdFactory git.CommandFactory, repo repository.GitRepo, cfg config.C
// NewTestRepo constructs a Repo. It is intended as a helper function for tests which assembles
// dependencies ad-hoc from the given config.
func NewTestRepo(t testing.TB, cfg config.Cfg, repo repository.GitRepo) *Repo {
- return New(git.NewExecCommandFactory(cfg), repo, cfg)
+ gitCmdFactory := git.NewExecCommandFactory(cfg)
+ return New(gitCmdFactory, catfile.NewCache(gitCmdFactory, cfg), repo, cfg)
}
// Path returns the on-disk path of the repository.
diff --git a/internal/git/localrepo/repo_test.go b/internal/git/localrepo/repo_test.go
index 67da2ac34..aa564d2ed 100644
--- a/internal/git/localrepo/repo_test.go
+++ b/internal/git/localrepo/repo_test.go
@@ -6,6 +6,7 @@ import (
"github.com/stretchr/testify/require"
"gitlab.com/gitlab-org/gitaly/internal/git"
+ "gitlab.com/gitlab-org/gitaly/internal/git/catfile"
"gitlab.com/gitlab-org/gitaly/internal/git/gittest"
"gitlab.com/gitlab-org/gitaly/internal/helper"
"gitlab.com/gitlab-org/gitaly/internal/testhelper/testcfg"
@@ -18,14 +19,16 @@ func TestRepo(t *testing.T) {
gittest.TestRepository(t, cfg, func(t testing.TB, pbRepo *gitalypb.Repository) git.Repository {
t.Helper()
- return New(git.NewExecCommandFactory(cfg), pbRepo, cfg)
+ gitCmdFactory := git.NewExecCommandFactory(cfg)
+ return New(gitCmdFactory, catfile.NewCache(gitCmdFactory, cfg), pbRepo, cfg)
})
}
func TestRepo_Path(t *testing.T) {
t.Run("valid repository", func(t *testing.T) {
cfg, repoProto, repoPath := testcfg.BuildWithRepo(t)
- repo := New(git.NewExecCommandFactory(cfg), repoProto, cfg)
+ gitCmdFactory := git.NewExecCommandFactory(cfg)
+ repo := New(gitCmdFactory, catfile.NewCache(gitCmdFactory, cfg), repoProto, cfg)
path, err := repo.Path()
require.NoError(t, err)
@@ -34,7 +37,8 @@ func TestRepo_Path(t *testing.T) {
t.Run("deleted repository", func(t *testing.T) {
cfg, repoProto, repoPath := testcfg.BuildWithRepo(t)
- repo := New(git.NewExecCommandFactory(cfg), repoProto, cfg)
+ gitCmdFactory := git.NewExecCommandFactory(cfg)
+ repo := New(gitCmdFactory, catfile.NewCache(gitCmdFactory, cfg), repoProto, cfg)
require.NoError(t, os.RemoveAll(repoPath))
@@ -44,7 +48,8 @@ func TestRepo_Path(t *testing.T) {
t.Run("non-git repository", func(t *testing.T) {
cfg, repoProto, repoPath := testcfg.BuildWithRepo(t)
- repo := New(git.NewExecCommandFactory(cfg), repoProto, cfg)
+ gitCmdFactory := git.NewExecCommandFactory(cfg)
+ repo := New(gitCmdFactory, catfile.NewCache(gitCmdFactory, cfg), repoProto, cfg)
// Recreate the repository as a simple empty directory to simulate
// that the repository is in a partially-created state.
diff --git a/internal/git/log/commit_test.go b/internal/git/log/commit_test.go
index 918308dde..b6fda3378 100644
--- a/internal/git/log/commit_test.go
+++ b/internal/git/log/commit_test.go
@@ -33,7 +33,8 @@ func setupBatch(t *testing.T, ctx context.Context) (config.Cfg, catfile.Batch, *
cfg, repo, _ := testcfg.BuildWithRepo(t)
- c, err := catfile.New(ctx, git.NewExecCommandFactory(cfg), repo)
+ catfileCache := catfile.NewCache(git.NewExecCommandFactory(cfg), cfg)
+ c, err := catfileCache.BatchProcess(ctx, repo)
require.NoError(t, err)
return cfg, c, repo
diff --git a/internal/git/log/log.go b/internal/git/log/log.go
index 3987071a9..34ff4f5d1 100644
--- a/internal/git/log/log.go
+++ b/internal/git/log/log.go
@@ -24,8 +24,8 @@ type Parser struct {
}
// NewLogParser returns a new Parser
-func NewLogParser(ctx context.Context, gitCmdFactory git.CommandFactory, repo repository.GitRepo, src io.Reader) (*Parser, error) {
- c, err := catfile.New(ctx, gitCmdFactory, repo)
+func NewLogParser(ctx context.Context, catfileCache catfile.Cache, repo repository.GitRepo, src io.Reader) (*Parser, error) {
+ c, err := catfileCache.BatchProcess(ctx, repo)
if err != nil {
return nil, err
}
diff --git a/internal/git/objectpool/clone_test.go b/internal/git/objectpool/clone_test.go
index 78d52dcd0..037e0de18 100644
--- a/internal/git/objectpool/clone_test.go
+++ b/internal/git/objectpool/clone_test.go
@@ -7,6 +7,7 @@ import (
"github.com/stretchr/testify/require"
"gitlab.com/gitlab-org/gitaly/internal/git"
+ "gitlab.com/gitlab-org/gitaly/internal/git/catfile"
"gitlab.com/gitlab-org/gitaly/internal/git/gittest"
"gitlab.com/gitlab-org/gitaly/internal/gitaly/config"
"gitlab.com/gitlab-org/gitaly/internal/testhelper"
@@ -18,9 +19,18 @@ func setupObjectPool(t *testing.T) (*ObjectPool, *gitalypb.Repository) {
t.Helper()
cfg, repo, _ := testcfg.BuildWithRepo(t)
+ gitCommandFactory := git.NewExecCommandFactory(cfg)
- pool, err := NewObjectPool(cfg, config.NewLocator(cfg), git.NewExecCommandFactory(cfg), repo.GetStorageName(), gittest.NewObjectPoolName(t))
+ pool, err := NewObjectPool(
+ cfg,
+ config.NewLocator(cfg),
+ gitCommandFactory,
+ catfile.NewCache(gitCommandFactory, cfg),
+ repo.GetStorageName(),
+ gittest.NewObjectPoolName(t),
+ )
require.NoError(t, err)
+
t.Cleanup(func() {
if err := pool.Remove(context.TODO()); err != nil {
panic(err)
diff --git a/internal/git/objectpool/pool.go b/internal/git/objectpool/pool.go
index 68d15ed52..f23e5a854 100644
--- a/internal/git/objectpool/pool.go
+++ b/internal/git/objectpool/pool.go
@@ -13,6 +13,7 @@ import (
"strings"
"gitlab.com/gitlab-org/gitaly/internal/git"
+ "gitlab.com/gitlab-org/gitaly/internal/git/catfile"
"gitlab.com/gitlab-org/gitaly/internal/git/localrepo"
"gitlab.com/gitlab-org/gitaly/internal/gitaly/config"
"gitlab.com/gitlab-org/gitaly/internal/storage"
@@ -46,7 +47,14 @@ type ObjectPool struct {
// NewObjectPool will initialize the object with the required data on the storage
// shard. Relative path is validated to match the expected naming and directory
// structure. If the shard cannot be found, this function returns an error.
-func NewObjectPool(cfg config.Cfg, locator storage.Locator, gitCmdFactory git.CommandFactory, storageName, relativePath string) (*ObjectPool, error) {
+func NewObjectPool(
+ cfg config.Cfg,
+ locator storage.Locator,
+ gitCmdFactory git.CommandFactory,
+ catfileCache catfile.Cache,
+ storageName,
+ relativePath string,
+) (*ObjectPool, error) {
storagePath, err := locator.GetStorageByName(storageName)
if err != nil {
return nil, err
@@ -65,7 +73,7 @@ func NewObjectPool(cfg config.Cfg, locator storage.Locator, gitCmdFactory git.Co
storagePath: storagePath,
relativePath: relativePath,
}
- pool.poolRepo = localrepo.New(gitCmdFactory, pool, cfg)
+ pool.poolRepo = localrepo.New(gitCmdFactory, catfileCache, pool, cfg)
return pool, nil
}
@@ -150,7 +158,13 @@ func (o *ObjectPool) Init(ctx context.Context) (err error) {
}
// FromRepo returns an instance of ObjectPool that the repository points to
-func FromRepo(cfg config.Cfg, locator storage.Locator, gitCmdFactory git.CommandFactory, repo *gitalypb.Repository) (*ObjectPool, error) {
+func FromRepo(
+ cfg config.Cfg,
+ locator storage.Locator,
+ gitCmdFactory git.CommandFactory,
+ catfileCache catfile.Cache,
+ repo *gitalypb.Repository,
+) (*ObjectPool, error) {
dir, err := getAlternateObjectDir(locator, repo)
if err != nil {
return nil, err
@@ -170,7 +184,7 @@ func FromRepo(cfg config.Cfg, locator storage.Locator, gitCmdFactory git.Command
return nil, err
}
- return NewObjectPool(cfg, locator, gitCmdFactory, repo.GetStorageName(), filepath.Dir(altPathRelativeToStorage))
+ return NewObjectPool(cfg, locator, gitCmdFactory, catfileCache, repo.GetStorageName(), filepath.Dir(altPathRelativeToStorage))
}
var (
diff --git a/internal/git/objectpool/pool_test.go b/internal/git/objectpool/pool_test.go
index f1105805f..85c93e824 100644
--- a/internal/git/objectpool/pool_test.go
+++ b/internal/git/objectpool/pool_test.go
@@ -20,10 +20,10 @@ func TestNewObjectPool(t *testing.T) {
locator := config.NewLocator(cfg)
- _, err := NewObjectPool(cfg, locator, nil, cfg.Storages[0].Name, gittest.NewObjectPoolName(t))
+ _, err := NewObjectPool(cfg, locator, nil, nil, cfg.Storages[0].Name, gittest.NewObjectPoolName(t))
require.NoError(t, err)
- _, err = NewObjectPool(cfg, locator, nil, "mepmep", gittest.NewObjectPoolName(t))
+ _, err = NewObjectPool(cfg, locator, nil, nil, "mepmep", gittest.NewObjectPoolName(t))
require.Error(t, err, "creating pool in storage that does not exist should fail")
}
@@ -36,7 +36,7 @@ func TestNewFromRepoSuccess(t *testing.T) {
require.NoError(t, pool.Create(ctx, testRepo))
require.NoError(t, pool.Link(ctx, testRepo))
- poolFromRepo, err := FromRepo(pool.cfg, pool.locator, pool.gitCmdFactory, testRepo)
+ poolFromRepo, err := FromRepo(pool.cfg, pool.locator, pool.gitCmdFactory, nil, testRepo)
require.NoError(t, err)
require.Equal(t, pool.relativePath, poolFromRepo.relativePath)
require.Equal(t, pool.storageName, poolFromRepo.storageName)
@@ -48,7 +48,7 @@ func TestNewFromRepoNoObjectPool(t *testing.T) {
testRepoPath := filepath.Join(pool.cfg.Storages[0].Path, testRepo.RelativePath)
// no alternates file
- poolFromRepo, err := FromRepo(pool.cfg, pool.locator, pool.gitCmdFactory, testRepo)
+ poolFromRepo, err := FromRepo(pool.cfg, pool.locator, pool.gitCmdFactory, nil, testRepo)
require.Equal(t, ErrAlternateObjectDirNotExist, err)
require.Nil(t, poolFromRepo)
@@ -81,7 +81,7 @@ func TestNewFromRepoNoObjectPool(t *testing.T) {
t.Run(tc.desc, func(t *testing.T) {
alternateFilePath := filepath.Join(testRepoPath, "objects", "info", "alternates")
require.NoError(t, ioutil.WriteFile(alternateFilePath, tc.fileContent, 0644))
- poolFromRepo, err := FromRepo(pool.cfg, pool.locator, pool.gitCmdFactory, testRepo)
+ poolFromRepo, err := FromRepo(pool.cfg, pool.locator, pool.gitCmdFactory, nil, testRepo)
require.Equal(t, tc.expectedErr, err)
require.Nil(t, poolFromRepo)
diff --git a/internal/git/objectpool/proto.go b/internal/git/objectpool/proto.go
index e4065f708..6feebdfb5 100644
--- a/internal/git/objectpool/proto.go
+++ b/internal/git/objectpool/proto.go
@@ -2,14 +2,21 @@ package objectpool
import (
"gitlab.com/gitlab-org/gitaly/internal/git"
+ "gitlab.com/gitlab-org/gitaly/internal/git/catfile"
"gitlab.com/gitlab-org/gitaly/internal/gitaly/config"
"gitlab.com/gitlab-org/gitaly/internal/storage"
"gitlab.com/gitlab-org/gitaly/proto/go/gitalypb"
)
// FromProto returns an object pool object from a git repository object
-func FromProto(cfg config.Cfg, locator storage.Locator, gitCmdFactory git.CommandFactory, o *gitalypb.ObjectPool) (*ObjectPool, error) {
- return NewObjectPool(cfg, locator, gitCmdFactory, o.GetRepository().GetStorageName(), o.GetRepository().GetRelativePath())
+func FromProto(
+ cfg config.Cfg,
+ locator storage.Locator,
+ gitCmdFactory git.CommandFactory,
+ catfileCache catfile.Cache,
+ o *gitalypb.ObjectPool,
+) (*ObjectPool, error) {
+ return NewObjectPool(cfg, locator, gitCmdFactory, catfileCache, o.GetRepository().GetStorageName(), o.GetRepository().GetRelativePath())
}
// ToProto returns a new struct that is the protobuf definition of the ObjectPool
diff --git a/internal/git/remoterepo/repository_test.go b/internal/git/remoterepo/repository_test.go
index 4bf533544..2124f24cc 100644
--- a/internal/git/remoterepo/repository_test.go
+++ b/internal/git/remoterepo/repository_test.go
@@ -23,8 +23,21 @@ func TestRepository(t *testing.T) {
cfg := testcfg.Build(t)
serverSocketPath := testserver.RunGitalyServer(t, cfg, nil, func(srv *grpc.Server, deps *service.Dependencies) {
- gitalypb.RegisterRepositoryServiceServer(srv, repository.NewServer(deps.GetCfg(), deps.GetRubyServer(), deps.GetLocator(), deps.GetTxManager(), deps.GetGitCmdFactory()))
- gitalypb.RegisterCommitServiceServer(srv, commit.NewServer(deps.GetCfg(), deps.GetLocator(), deps.GetGitCmdFactory(), deps.GetLinguist()))
+ gitalypb.RegisterRepositoryServiceServer(srv, repository.NewServer(
+ deps.GetCfg(),
+ deps.GetRubyServer(),
+ deps.GetLocator(),
+ deps.GetTxManager(),
+ deps.GetGitCmdFactory(),
+ deps.GetCatfileCache(),
+ ))
+ gitalypb.RegisterCommitServiceServer(srv, commit.NewServer(
+ deps.GetCfg(),
+ deps.GetLocator(),
+ deps.GetGitCmdFactory(),
+ deps.GetLinguist(),
+ deps.GetCatfileCache(),
+ ))
})
ctx, cancel := testhelper.Context()
diff --git a/internal/gitaly/maintenance/optimize_test.go b/internal/gitaly/maintenance/optimize_test.go
index c205bbd2e..245592a7c 100644
--- a/internal/gitaly/maintenance/optimize_test.go
+++ b/internal/gitaly/maintenance/optimize_test.go
@@ -10,6 +10,7 @@ import (
"github.com/stretchr/testify/require"
"gitlab.com/gitlab-org/gitaly/internal/backchannel"
"gitlab.com/gitlab-org/gitaly/internal/git"
+ "gitlab.com/gitlab-org/gitaly/internal/git/catfile"
"gitlab.com/gitlab-org/gitaly/internal/gitaly/config"
"gitlab.com/gitlab-org/gitaly/internal/gitaly/service/repository"
"gitlab.com/gitlab-org/gitaly/internal/gitaly/transaction"
@@ -30,7 +31,8 @@ func (mo *mockOptimizer) OptimizeRepository(ctx context.Context, req *gitalypb.O
mo.actual = append(mo.actual, req.Repository)
l := config.NewLocator(mo.cfg)
gitCmdFactory := git.NewExecCommandFactory(mo.cfg)
- resp, err := repository.NewServer(mo.cfg, nil, l, transaction.NewManager(mo.cfg, backchannel.NewRegistry()), gitCmdFactory).OptimizeRepository(ctx, req)
+ catfileCache := catfile.NewCache(gitCmdFactory, mo.cfg)
+ resp, err := repository.NewServer(mo.cfg, nil, l, transaction.NewManager(mo.cfg, backchannel.NewRegistry()), gitCmdFactory, catfileCache).OptimizeRepository(ctx, req)
assert.NoError(mo.t, err)
return resp, err
}
diff --git a/internal/gitaly/server/auth_test.go b/internal/gitaly/server/auth_test.go
index 9f34903dc..3c42cb197 100644
--- a/internal/gitaly/server/auth_test.go
+++ b/internal/gitaly/server/auth_test.go
@@ -17,6 +17,7 @@ import (
"gitlab.com/gitlab-org/gitaly/client"
"gitlab.com/gitlab-org/gitaly/internal/backchannel"
"gitlab.com/gitlab-org/gitaly/internal/git"
+ "gitlab.com/gitlab-org/gitaly/internal/git/catfile"
"gitlab.com/gitlab-org/gitaly/internal/git/gittest"
"gitlab.com/gitlab-org/gitaly/internal/gitaly/config"
"gitlab.com/gitlab-org/gitaly/internal/gitaly/config/auth"
@@ -199,6 +200,7 @@ func runServer(t *testing.T, cfg config.Cfg) string {
txManager := transaction.NewManager(cfg, registry)
hookManager := hook.NewManager(locator, txManager, gitlab.NewMockClient(), cfg)
gitCmdFactory := git.NewExecCommandFactory(cfg)
+ catfileCache := catfile.NewCache(gitCmdFactory, cfg)
srv, err := New(false, cfg, testhelper.DiscardTestEntry(t), registry)
require.NoError(t, err)
@@ -210,6 +212,7 @@ func runServer(t *testing.T, cfg config.Cfg) string {
StorageLocator: locator,
ClientPool: conns,
GitCmdFactory: gitCmdFactory,
+ CatfileCache: catfileCache,
})
serverSocketPath := testhelper.GetTemporaryGitalySocketFileName(t)
diff --git a/internal/gitaly/service/blob/get_blob.go b/internal/gitaly/service/blob/get_blob.go
index 213552416..6116a9c99 100644
--- a/internal/gitaly/service/blob/get_blob.go
+++ b/internal/gitaly/service/blob/get_blob.go
@@ -20,7 +20,7 @@ func (s *server) GetBlob(in *gitalypb.GetBlobRequest, stream gitalypb.BlobServic
return status.Errorf(codes.InvalidArgument, "GetBlob: %v", err)
}
- c, err := catfile.New(stream.Context(), s.gitCmdFactory, in.Repository)
+ c, err := s.catfileCache.BatchProcess(stream.Context(), in.Repository)
if err != nil {
return status.Errorf(codes.Internal, "GetBlob: %v", err)
}
diff --git a/internal/gitaly/service/blob/get_blobs.go b/internal/gitaly/service/blob/get_blobs.go
index 2dcf4122f..1de3c5b65 100644
--- a/internal/gitaly/service/blob/get_blobs.go
+++ b/internal/gitaly/service/blob/get_blobs.go
@@ -144,7 +144,7 @@ func (s *server) GetBlobs(req *gitalypb.GetBlobsRequest, stream gitalypb.BlobSer
return err
}
- c, err := catfile.New(stream.Context(), s.gitCmdFactory, req.Repository)
+ c, err := s.catfileCache.BatchProcess(stream.Context(), req.Repository)
if err != nil {
return err
}
diff --git a/internal/gitaly/service/blob/server.go b/internal/gitaly/service/blob/server.go
index dee0badc5..4b89b5766 100644
--- a/internal/gitaly/service/blob/server.go
+++ b/internal/gitaly/service/blob/server.go
@@ -2,6 +2,7 @@ package blob
import (
"gitlab.com/gitlab-org/gitaly/internal/git"
+ "gitlab.com/gitlab-org/gitaly/internal/git/catfile"
"gitlab.com/gitlab-org/gitaly/internal/git/localrepo"
"gitlab.com/gitlab-org/gitaly/internal/git/repository"
"gitlab.com/gitlab-org/gitaly/internal/gitaly/config"
@@ -13,17 +14,19 @@ type server struct {
cfg config.Cfg
locator storage.Locator
gitCmdFactory git.CommandFactory
+ catfileCache catfile.Cache
}
// NewServer creates a new instance of a grpc BlobServer
-func NewServer(cfg config.Cfg, locator storage.Locator, gitCmdFactory git.CommandFactory) gitalypb.BlobServiceServer {
+func NewServer(cfg config.Cfg, locator storage.Locator, gitCmdFactory git.CommandFactory, catfileCache catfile.Cache) gitalypb.BlobServiceServer {
return &server{
cfg: cfg,
locator: locator,
gitCmdFactory: gitCmdFactory,
+ catfileCache: catfileCache,
}
}
func (s *server) localrepo(repo repository.GitRepo) *localrepo.Repo {
- return localrepo.New(s.gitCmdFactory, repo, s.cfg)
+ return localrepo.New(s.gitCmdFactory, s.catfileCache, repo, s.cfg)
}
diff --git a/internal/gitaly/service/blob/testhelper_test.go b/internal/gitaly/service/blob/testhelper_test.go
index 07df48c2d..8b99b8752 100644
--- a/internal/gitaly/service/blob/testhelper_test.go
+++ b/internal/gitaly/service/blob/testhelper_test.go
@@ -35,7 +35,12 @@ func setup(t *testing.T) (config.Cfg, *gitalypb.Repository, string, gitalypb.Blo
t.Cleanup(cleanup)
addr := testserver.RunGitalyServer(t, cfg, nil, func(srv *grpc.Server, deps *service.Dependencies) {
- gitalypb.RegisterBlobServiceServer(srv, NewServer(deps.GetCfg(), deps.GetLocator(), deps.GetGitCmdFactory()))
+ gitalypb.RegisterBlobServiceServer(srv, NewServer(
+ deps.GetCfg(),
+ deps.GetLocator(),
+ deps.GetGitCmdFactory(),
+ deps.GetCatfileCache(),
+ ))
})
conn, err := grpc.Dial(addr, grpc.WithInsecure())
diff --git a/internal/gitaly/service/cleanup/apply_bfg_object_map_stream.go b/internal/gitaly/service/cleanup/apply_bfg_object_map_stream.go
index 7d023ecca..3f70ea27f 100644
--- a/internal/gitaly/service/cleanup/apply_bfg_object_map_stream.go
+++ b/internal/gitaly/service/cleanup/apply_bfg_object_map_stream.go
@@ -40,7 +40,7 @@ func (s *server) ApplyBfgObjectMapStream(server gitalypb.CleanupService_ApplyBfg
reader := &bfgStreamReader{firstRequest: firstRequest, server: server}
chunker := chunk.New(&bfgStreamWriter{server: server})
- notifier, err := notifier.New(ctx, s.gitCmdFactory, repo, chunker)
+ notifier, err := notifier.New(ctx, s.catfileCache, repo, chunker)
if err != nil {
return helper.ErrInternal(err)
}
diff --git a/internal/gitaly/service/cleanup/notifier/notifier.go b/internal/gitaly/service/cleanup/notifier/notifier.go
index 865a16e91..578870fd8 100644
--- a/internal/gitaly/service/cleanup/notifier/notifier.go
+++ b/internal/gitaly/service/cleanup/notifier/notifier.go
@@ -17,8 +17,8 @@ type Notifier struct {
}
// New instantiates a new Notifier
-func New(ctx context.Context, gitCmdFactory git.CommandFactory, repo *gitalypb.Repository, chunker *chunk.Chunker) (*Notifier, error) {
- catfile, err := catfile.New(ctx, gitCmdFactory, repo)
+func New(ctx context.Context, catfileCache catfile.Cache, repo *gitalypb.Repository, chunker *chunk.Chunker) (*Notifier, error) {
+ catfile, err := catfileCache.BatchProcess(ctx, repo)
if err != nil {
return nil, err
}
diff --git a/internal/gitaly/service/cleanup/server.go b/internal/gitaly/service/cleanup/server.go
index 7d58ad4b1..0172b6af6 100644
--- a/internal/gitaly/service/cleanup/server.go
+++ b/internal/gitaly/service/cleanup/server.go
@@ -2,6 +2,7 @@ package cleanup
import (
"gitlab.com/gitlab-org/gitaly/internal/git"
+ "gitlab.com/gitlab-org/gitaly/internal/git/catfile"
"gitlab.com/gitlab-org/gitaly/internal/gitaly/config"
"gitlab.com/gitlab-org/gitaly/proto/go/gitalypb"
)
@@ -9,9 +10,14 @@ import (
type server struct {
cfg config.Cfg
gitCmdFactory git.CommandFactory
+ catfileCache catfile.Cache
}
// NewServer creates a new instance of a grpc CleanupServer
-func NewServer(cfg config.Cfg, gitCmdFactory git.CommandFactory) gitalypb.CleanupServiceServer {
- return &server{cfg: cfg, gitCmdFactory: gitCmdFactory}
+func NewServer(cfg config.Cfg, gitCmdFactory git.CommandFactory, catfileCache catfile.Cache) gitalypb.CleanupServiceServer {
+ return &server{
+ cfg: cfg,
+ gitCmdFactory: gitCmdFactory,
+ catfileCache: catfileCache,
+ }
}
diff --git a/internal/gitaly/service/cleanup/testhelper_test.go b/internal/gitaly/service/cleanup/testhelper_test.go
index 1989e3873..795fe7950 100644
--- a/internal/gitaly/service/cleanup/testhelper_test.go
+++ b/internal/gitaly/service/cleanup/testhelper_test.go
@@ -38,7 +38,11 @@ func setupCleanupService(t *testing.T) (config.Cfg, *gitalypb.Repository, string
func runCleanupServiceServer(t *testing.T, cfg config.Cfg) string {
return testserver.RunGitalyServer(t, cfg, nil, func(srv *grpc.Server, deps *service.Dependencies) {
- gitalypb.RegisterCleanupServiceServer(srv, NewServer(deps.GetCfg(), deps.GetGitCmdFactory()))
+ gitalypb.RegisterCleanupServiceServer(srv, NewServer(
+ deps.GetCfg(),
+ deps.GetGitCmdFactory(),
+ deps.GetCatfileCache(),
+ ))
gitalypb.RegisterHookServiceServer(srv, hookservice.NewServer(deps.GetCfg(), deps.GetHookManager(), deps.GetGitCmdFactory()))
})
}
diff --git a/internal/gitaly/service/commit/between.go b/internal/gitaly/service/commit/between.go
index e0d82c6f7..5d2d74c20 100644
--- a/internal/gitaly/service/commit/between.go
+++ b/internal/gitaly/service/commit/between.go
@@ -34,10 +34,9 @@ func (s *server) CommitsBetween(in *gitalypb.CommitsBetweenRequest, stream gital
from, to, limit := normalizedCommitsBetweenParams(in)
revisionRange := fmt.Sprintf("%s..%s", from, to)
- if err := sendCommits(
+ if err := s.sendCommits(
stream.Context(),
sender,
- s.gitCmdFactory,
in.GetRepository(),
[]string{revisionRange},
nil,
diff --git a/internal/gitaly/service/commit/commit_messages.go b/internal/gitaly/service/commit/commit_messages.go
index d6229c1c8..f19851aa9 100644
--- a/internal/gitaly/service/commit/commit_messages.go
+++ b/internal/gitaly/service/commit/commit_messages.go
@@ -6,7 +6,6 @@ import (
"io"
"gitlab.com/gitlab-org/gitaly/internal/git"
- "gitlab.com/gitlab-org/gitaly/internal/git/catfile"
"gitlab.com/gitlab-org/gitaly/internal/git/log"
"gitlab.com/gitlab-org/gitaly/internal/helper"
"gitlab.com/gitlab-org/gitaly/proto/go/gitalypb"
@@ -26,7 +25,7 @@ func (s *server) GetCommitMessages(request *gitalypb.GetCommitMessagesRequest, s
func (s *server) getAndStreamCommitMessages(request *gitalypb.GetCommitMessagesRequest, stream gitalypb.CommitService_GetCommitMessagesServer) error {
ctx := stream.Context()
- c, err := catfile.New(ctx, s.gitCmdFactory, request.GetRepository())
+ c, err := s.catfileCache.BatchProcess(ctx, request.GetRepository())
if err != nil {
return err
}
diff --git a/internal/gitaly/service/commit/commit_signatures.go b/internal/gitaly/service/commit/commit_signatures.go
index 15ec9ab80..daee191c3 100644
--- a/internal/gitaly/service/commit/commit_signatures.go
+++ b/internal/gitaly/service/commit/commit_signatures.go
@@ -29,7 +29,7 @@ func (s *server) GetCommitSignatures(request *gitalypb.GetCommitSignaturesReques
func (s *server) getCommitSignatures(request *gitalypb.GetCommitSignaturesRequest, stream gitalypb.CommitService_GetCommitSignaturesServer) error {
ctx := stream.Context()
- c, err := catfile.New(ctx, s.gitCmdFactory, request.GetRepository())
+ c, err := s.catfileCache.BatchProcess(ctx, request.GetRepository())
if err != nil {
return helper.ErrInternal(err)
}
diff --git a/internal/gitaly/service/commit/commits_by_message.go b/internal/gitaly/service/commit/commits_by_message.go
index 47735eeef..d6e679b70 100644
--- a/internal/gitaly/service/commit/commits_by_message.go
+++ b/internal/gitaly/service/commit/commits_by_message.go
@@ -65,7 +65,7 @@ func (s *server) commitsByMessage(in *gitalypb.CommitsByMessageRequest, stream g
paths = append(paths, string(path))
}
- return sendCommits(stream.Context(), sender, s.gitCmdFactory, in.GetRepository(), []string{string(revision)}, paths, in.GetGlobalOptions(), gitLogExtraOptions...)
+ return s.sendCommits(stream.Context(), sender, in.GetRepository(), []string{string(revision)}, paths, in.GetGlobalOptions(), gitLogExtraOptions...)
}
func validateCommitsByMessageRequest(in *gitalypb.CommitsByMessageRequest) error {
diff --git a/internal/gitaly/service/commit/commits_helper.go b/internal/gitaly/service/commit/commits_helper.go
index 17aa143b7..775634653 100644
--- a/internal/gitaly/service/commit/commits_helper.go
+++ b/internal/gitaly/service/commit/commits_helper.go
@@ -10,18 +10,26 @@ import (
"gitlab.com/gitlab-org/gitaly/proto/go/gitalypb"
)
-func sendCommits(ctx context.Context, sender chunk.Sender, gitCmdFactory git.CommandFactory, repo *gitalypb.Repository, revisionRange []string, paths []string, options *gitalypb.GlobalOptions, extraArgs ...git.Option) error {
+func (s *server) sendCommits(
+ ctx context.Context,
+ sender chunk.Sender,
+ repo *gitalypb.Repository,
+ revisionRange []string,
+ paths []string,
+ options *gitalypb.GlobalOptions,
+ extraArgs ...git.Option,
+) error {
revisions := make([]git.Revision, len(revisionRange))
for i, revision := range revisionRange {
revisions[i] = git.Revision(revision)
}
- cmd, err := log.GitLogCommand(ctx, gitCmdFactory, repo, revisions, paths, options, extraArgs...)
+ cmd, err := log.GitLogCommand(ctx, s.gitCmdFactory, repo, revisions, paths, options, extraArgs...)
if err != nil {
return err
}
- logParser, err := log.NewLogParser(ctx, gitCmdFactory, repo, cmd)
+ logParser, err := log.NewLogParser(ctx, s.catfileCache, repo, cmd)
if err != nil {
return err
}
diff --git a/internal/gitaly/service/commit/filter_shas_with_signatures.go b/internal/gitaly/service/commit/filter_shas_with_signatures.go
index 4c3e64f4a..3f041c8c5 100644
--- a/internal/gitaly/service/commit/filter_shas_with_signatures.go
+++ b/internal/gitaly/service/commit/filter_shas_with_signatures.go
@@ -37,7 +37,7 @@ func validateFirstFilterShasWithSignaturesRequest(in *gitalypb.FilterShasWithSig
func (s *server) filterShasWithSignatures(bidi gitalypb.CommitService_FilterShasWithSignaturesServer, firstRequest *gitalypb.FilterShasWithSignaturesRequest) error {
ctx := bidi.Context()
- c, err := catfile.New(ctx, s.gitCmdFactory, firstRequest.GetRepository())
+ c, err := s.catfileCache.BatchProcess(ctx, firstRequest.GetRepository())
if err != nil {
return err
}
diff --git a/internal/gitaly/service/commit/find_all_commits.go b/internal/gitaly/service/commit/find_all_commits.go
index 3d4e3324d..2e6715130 100644
--- a/internal/gitaly/service/commit/find_all_commits.go
+++ b/internal/gitaly/service/commit/find_all_commits.go
@@ -80,5 +80,5 @@ func (s *server) findAllCommits(in *gitalypb.FindAllCommitsRequest, stream gital
gitLogExtraOptions = append(gitLogExtraOptions, git.Flag{Name: "--topo-order"})
}
- return sendCommits(stream.Context(), sender, s.gitCmdFactory, in.GetRepository(), revisions, nil, nil, gitLogExtraOptions...)
+ return s.sendCommits(stream.Context(), sender, in.GetRepository(), revisions, nil, nil, gitLogExtraOptions...)
}
diff --git a/internal/gitaly/service/commit/find_commit_test.go b/internal/gitaly/service/commit/find_commit_test.go
index 2e05ceda0..34e697aeb 100644
--- a/internal/gitaly/service/commit/find_commit_test.go
+++ b/internal/gitaly/service/commit/find_commit_test.go
@@ -9,7 +9,6 @@ import (
"github.com/golang/protobuf/ptypes/timestamp"
"github.com/stretchr/testify/require"
"gitlab.com/gitlab-org/gitaly/internal/git"
- "gitlab.com/gitlab-org/gitaly/internal/git/catfile"
"gitlab.com/gitlab-org/gitaly/internal/git/gittest"
"gitlab.com/gitlab-org/gitaly/internal/git/localrepo"
"gitlab.com/gitlab-org/gitaly/internal/helper"
@@ -316,8 +315,6 @@ func benchmarkFindCommit(withCache bool, b *testing.B) {
require.NoError(b, logCmd.Wait())
- defer catfile.ExpireAll()
-
for i := 0; i < b.N; i++ {
revision := revisions[b.N%len(revisions)]
if withCache {
@@ -357,8 +354,6 @@ func TestFindCommitWithCache(t *testing.T) {
require.NoError(t, logCmd.Wait())
- defer catfile.ExpireAll()
-
for i := 0; i < 10; i++ {
revision := revisions[i%len(revisions)]
md := metadata.New(map[string]string{
diff --git a/internal/gitaly/service/commit/find_commits.go b/internal/gitaly/service/commit/find_commits.go
index 8662ab593..0e1c6a600 100644
--- a/internal/gitaly/service/commit/find_commits.go
+++ b/internal/gitaly/service/commit/find_commits.go
@@ -56,7 +56,7 @@ func (s *server) findCommits(ctx context.Context, req *gitalypb.FindCommitsReque
return fmt.Errorf("error when creating git log command: %v", err)
}
- batch, err := catfile.New(ctx, s.gitCmdFactory, req.GetRepository())
+ batch, err := s.catfileCache.BatchProcess(ctx, req.GetRepository())
if err != nil {
return fmt.Errorf("creating catfile: %v", err)
}
diff --git a/internal/gitaly/service/commit/last_commit_for_path.go b/internal/gitaly/service/commit/last_commit_for_path.go
index 5c6a1cbce..55ff1fce5 100644
--- a/internal/gitaly/service/commit/last_commit_for_path.go
+++ b/internal/gitaly/service/commit/last_commit_for_path.go
@@ -4,7 +4,6 @@ import (
"context"
"gitlab.com/gitlab-org/gitaly/internal/git"
- "gitlab.com/gitlab-org/gitaly/internal/git/catfile"
"gitlab.com/gitlab-org/gitaly/internal/git/log"
"gitlab.com/gitlab-org/gitaly/internal/helper"
"gitlab.com/gitlab-org/gitaly/proto/go/gitalypb"
@@ -30,7 +29,7 @@ func (s *server) lastCommitForPath(ctx context.Context, in *gitalypb.LastCommitF
}
repo := in.GetRepository()
- c, err := catfile.New(ctx, s.gitCmdFactory, repo)
+ c, err := s.catfileCache.BatchProcess(ctx, repo)
if err != nil {
return nil, err
}
diff --git a/internal/gitaly/service/commit/list_commits_by_oid.go b/internal/gitaly/service/commit/list_commits_by_oid.go
index b0a6463d3..04d36bb58 100644
--- a/internal/gitaly/service/commit/list_commits_by_oid.go
+++ b/internal/gitaly/service/commit/list_commits_by_oid.go
@@ -27,7 +27,7 @@ var (
func (s *server) ListCommitsByOid(in *gitalypb.ListCommitsByOidRequest, stream gitalypb.CommitService_ListCommitsByOidServer) error {
ctx := stream.Context()
- c, err := catfile.New(ctx, s.gitCmdFactory, in.Repository)
+ c, err := s.catfileCache.BatchProcess(ctx, in.Repository)
if err != nil {
return err
}
diff --git a/internal/gitaly/service/commit/list_commits_by_ref_name.go b/internal/gitaly/service/commit/list_commits_by_ref_name.go
index 6db277149..02a7a1a9f 100644
--- a/internal/gitaly/service/commit/list_commits_by_ref_name.go
+++ b/internal/gitaly/service/commit/list_commits_by_ref_name.go
@@ -13,7 +13,7 @@ import (
func (s *server) ListCommitsByRefName(in *gitalypb.ListCommitsByRefNameRequest, stream gitalypb.CommitService_ListCommitsByRefNameServer) error {
ctx := stream.Context()
- c, err := catfile.New(ctx, s.gitCmdFactory, in.Repository)
+ c, err := s.catfileCache.BatchProcess(ctx, in.Repository)
if err != nil {
return helper.ErrInternal(err)
}
diff --git a/internal/gitaly/service/commit/list_last_commits_for_tree.go b/internal/gitaly/service/commit/list_last_commits_for_tree.go
index 7c4adc059..5cea8a390 100644
--- a/internal/gitaly/service/commit/list_last_commits_for_tree.go
+++ b/internal/gitaly/service/commit/list_last_commits_for_tree.go
@@ -7,7 +7,6 @@ import (
"gitlab.com/gitlab-org/gitaly/internal/command"
"gitlab.com/gitlab-org/gitaly/internal/git"
- "gitlab.com/gitlab-org/gitaly/internal/git/catfile"
"gitlab.com/gitlab-org/gitaly/internal/git/log"
"gitlab.com/gitlab-org/gitaly/internal/git/lstree"
"gitlab.com/gitlab-org/gitaly/internal/helper"
@@ -45,7 +44,7 @@ func (s *server) listLastCommitsForTree(in *gitalypb.ListLastCommitsForTreeReque
ctx := stream.Context()
repo := in.GetRepository()
- c, err := catfile.New(ctx, s.gitCmdFactory, repo)
+ c, err := s.catfileCache.BatchProcess(ctx, repo)
if err != nil {
return err
}
diff --git a/internal/gitaly/service/commit/server.go b/internal/gitaly/service/commit/server.go
index c6230e5a1..1e6ea2586 100644
--- a/internal/gitaly/service/commit/server.go
+++ b/internal/gitaly/service/commit/server.go
@@ -2,6 +2,7 @@ package commit
import (
"gitlab.com/gitlab-org/gitaly/internal/git"
+ "gitlab.com/gitlab-org/gitaly/internal/git/catfile"
"gitlab.com/gitlab-org/gitaly/internal/git/localrepo"
"gitlab.com/gitlab-org/gitaly/internal/git/repository"
"gitlab.com/gitlab-org/gitaly/internal/gitaly/config"
@@ -16,6 +17,7 @@ type server struct {
locator storage.Locator
gitCmdFactory git.CommandFactory
linguist *linguist.Instance
+ catfileCache catfile.Cache
}
var (
@@ -23,10 +25,22 @@ var (
)
// NewServer creates a new instance of a grpc CommitServiceServer
-func NewServer(cfg config.Cfg, locator storage.Locator, gitCmdFactory git.CommandFactory, ling *linguist.Instance) gitalypb.CommitServiceServer {
- return &server{cfg: cfg, locator: locator, gitCmdFactory: gitCmdFactory, linguist: ling}
+func NewServer(
+ cfg config.Cfg,
+ locator storage.Locator,
+ gitCmdFactory git.CommandFactory,
+ ling *linguist.Instance,
+ catfileCache catfile.Cache,
+) gitalypb.CommitServiceServer {
+ return &server{
+ cfg: cfg,
+ locator: locator,
+ gitCmdFactory: gitCmdFactory,
+ linguist: ling,
+ catfileCache: catfileCache,
+ }
}
func (s *server) localrepo(repo repository.GitRepo) *localrepo.Repo {
- return localrepo.New(s.gitCmdFactory, repo, s.cfg)
+ return localrepo.New(s.gitCmdFactory, s.catfileCache, repo, s.cfg)
}
diff --git a/internal/gitaly/service/commit/testhelper_test.go b/internal/gitaly/service/commit/testhelper_test.go
index 4a05fc6eb..ec80ea9c8 100644
--- a/internal/gitaly/service/commit/testhelper_test.go
+++ b/internal/gitaly/service/commit/testhelper_test.go
@@ -9,6 +9,7 @@ import (
"github.com/golang/protobuf/ptypes/timestamp"
"github.com/stretchr/testify/require"
"gitlab.com/gitlab-org/gitaly/internal/git"
+ "gitlab.com/gitlab-org/gitaly/internal/git/catfile"
"gitlab.com/gitlab-org/gitaly/internal/git/gittest"
"gitlab.com/gitlab-org/gitaly/internal/gitaly/config"
"gitlab.com/gitlab-org/gitaly/internal/gitaly/linguist"
@@ -79,7 +80,16 @@ func startTestServices(t testing.TB, cfg config.Cfg) string {
ling, err := linguist.New(cfg)
require.NoError(t, err)
- gitalypb.RegisterCommitServiceServer(server, NewServer(cfg, config.NewLocator(cfg), git.NewExecCommandFactory(cfg), ling))
+ gitCommandFactory := git.NewExecCommandFactory(cfg)
+ catfileCache := catfile.NewCache(gitCommandFactory, cfg)
+
+ gitalypb.RegisterCommitServiceServer(server, NewServer(
+ cfg,
+ config.NewLocator(cfg),
+ gitCommandFactory,
+ ling,
+ catfileCache,
+ ))
go server.Serve(listener)
return "unix://" + serverSocketPath
diff --git a/internal/gitaly/service/commit/tree_entries.go b/internal/gitaly/service/commit/tree_entries.go
index 3bd5a414d..eaad9138a 100644
--- a/internal/gitaly/service/commit/tree_entries.go
+++ b/internal/gitaly/service/commit/tree_entries.go
@@ -99,7 +99,7 @@ func (s *server) GetTreeEntries(in *gitalypb.GetTreeEntriesRequest, stream gital
return status.Errorf(codes.InvalidArgument, "TreeEntry: %v", err)
}
- c, err := catfile.New(stream.Context(), s.gitCmdFactory, in.Repository)
+ c, err := s.catfileCache.BatchProcess(stream.Context(), in.Repository)
if err != nil {
return err
}
diff --git a/internal/gitaly/service/commit/tree_entry.go b/internal/gitaly/service/commit/tree_entry.go
index 1c28d282a..ddf3e6fb5 100644
--- a/internal/gitaly/service/commit/tree_entry.go
+++ b/internal/gitaly/service/commit/tree_entry.go
@@ -129,7 +129,7 @@ func (s *server) TreeEntry(in *gitalypb.TreeEntryRequest, stream gitalypb.Commit
requestPath = strings.TrimRight(requestPath, "/")
}
- c, err := catfile.New(stream.Context(), s.gitCmdFactory, in.Repository)
+ c, err := s.catfileCache.BatchProcess(stream.Context(), in.Repository)
if err != nil {
return err
diff --git a/internal/gitaly/service/conflicts/server.go b/internal/gitaly/service/conflicts/server.go
index 52392ead4..0e425f990 100644
--- a/internal/gitaly/service/conflicts/server.go
+++ b/internal/gitaly/service/conflicts/server.go
@@ -3,6 +3,7 @@ package conflicts
import (
"gitlab.com/gitlab-org/gitaly/client"
"gitlab.com/gitlab-org/gitaly/internal/git"
+ "gitlab.com/gitlab-org/gitaly/internal/git/catfile"
"gitlab.com/gitlab-org/gitaly/internal/git/localrepo"
"gitlab.com/gitlab-org/gitaly/internal/git/repository"
"gitlab.com/gitlab-org/gitaly/internal/gitaly/config"
@@ -14,15 +15,17 @@ type server struct {
cfg config.Cfg
locator storage.Locator
gitCmdFactory git.CommandFactory
+ catfileCache catfile.Cache
pool *client.Pool
}
// NewServer creates a new instance of a grpc ConflictsServer
-func NewServer(cfg config.Cfg, locator storage.Locator, gitCmdFactory git.CommandFactory) gitalypb.ConflictsServiceServer {
+func NewServer(cfg config.Cfg, locator storage.Locator, gitCmdFactory git.CommandFactory, catfileCache catfile.Cache) gitalypb.ConflictsServiceServer {
return &server{
cfg: cfg,
locator: locator,
gitCmdFactory: gitCmdFactory,
+ catfileCache: catfileCache,
pool: client.NewPoolWithOptions(
client.WithDialer(client.HealthCheckDialer(client.DialContext)),
client.WithDialOptions(client.FailOnNonTempDialError()...),
@@ -31,5 +34,5 @@ func NewServer(cfg config.Cfg, locator storage.Locator, gitCmdFactory git.Comman
}
func (s *server) localrepo(repo repository.GitRepo) *localrepo.Repo {
- return localrepo.New(s.gitCmdFactory, repo, s.cfg)
+ return localrepo.New(s.gitCmdFactory, s.catfileCache, repo, s.cfg)
}
diff --git a/internal/gitaly/service/conflicts/testhelper_test.go b/internal/gitaly/service/conflicts/testhelper_test.go
index 2bf640fc6..35cd6803b 100644
--- a/internal/gitaly/service/conflicts/testhelper_test.go
+++ b/internal/gitaly/service/conflicts/testhelper_test.go
@@ -72,8 +72,20 @@ func SetupConflictsService(t testing.TB, bare bool) (config.Cfg, *gitalypb.Repos
func runConflictsServer(t testing.TB, cfg config.Cfg) string {
return testserver.RunGitalyServer(t, cfg, nil, func(srv *grpc.Server, deps *service.Dependencies) {
- gitalypb.RegisterConflictsServiceServer(srv, NewServer(deps.GetCfg(), deps.GetLocator(), deps.GetGitCmdFactory()))
- gitalypb.RegisterRepositoryServiceServer(srv, repository.NewServer(deps.GetCfg(), deps.GetRubyServer(), deps.GetLocator(), deps.GetTxManager(), deps.GetGitCmdFactory()))
+ gitalypb.RegisterConflictsServiceServer(srv, NewServer(
+ deps.GetCfg(),
+ deps.GetLocator(),
+ deps.GetGitCmdFactory(),
+ deps.GetCatfileCache(),
+ ))
+ gitalypb.RegisterRepositoryServiceServer(srv, repository.NewServer(
+ deps.GetCfg(),
+ deps.GetRubyServer(),
+ deps.GetLocator(),
+ deps.GetTxManager(),
+ deps.GetGitCmdFactory(),
+ deps.GetCatfileCache(),
+ ))
gitalypb.RegisterSSHServiceServer(srv, ssh.NewServer(
deps.GetCfg(),
deps.GetLocator(),
@@ -81,7 +93,13 @@ func runConflictsServer(t testing.TB, cfg config.Cfg) string {
deps.GetTxManager(),
))
gitalypb.RegisterHookServiceServer(srv, hook_service.NewServer(deps.GetCfg(), deps.GetHookManager(), deps.GetGitCmdFactory()))
- gitalypb.RegisterCommitServiceServer(srv, commit.NewServer(deps.GetCfg(), deps.GetLocator(), deps.GetGitCmdFactory(), deps.GetLinguist()))
+ gitalypb.RegisterCommitServiceServer(srv, commit.NewServer(
+ deps.GetCfg(),
+ deps.GetLocator(),
+ deps.GetGitCmdFactory(),
+ deps.GetLinguist(),
+ deps.GetCatfileCache(),
+ ))
})
}
diff --git a/internal/gitaly/service/dependencies.go b/internal/gitaly/service/dependencies.go
index ee67bfd29..8f312e91e 100644
--- a/internal/gitaly/service/dependencies.go
+++ b/internal/gitaly/service/dependencies.go
@@ -4,6 +4,7 @@ import (
"gitlab.com/gitlab-org/gitaly/client"
"gitlab.com/gitlab-org/gitaly/internal/backchannel"
"gitlab.com/gitlab-org/gitaly/internal/git"
+ "gitlab.com/gitlab-org/gitaly/internal/git/catfile"
"gitlab.com/gitlab-org/gitaly/internal/gitaly/config"
gitalyhook "gitlab.com/gitlab-org/gitaly/internal/gitaly/hook"
"gitlab.com/gitlab-org/gitaly/internal/gitaly/linguist"
@@ -25,6 +26,7 @@ type Dependencies struct {
Linguist *linguist.Instance
BackchannelRegistry *backchannel.Registry
GitlabClient gitlab.Client
+ CatfileCache catfile.Cache
}
// GetCfg returns service configuration.
@@ -76,3 +78,8 @@ func (dc *Dependencies) GetBackchannelRegistry() *backchannel.Registry {
func (dc *Dependencies) GetGitlabClient() gitlab.Client {
return dc.GitlabClient
}
+
+// GetCatfileCache returns catfile cache.
+func (dc *Dependencies) GetCatfileCache() catfile.Cache {
+ return dc.CatfileCache
+}
diff --git a/internal/gitaly/service/diff/server.go b/internal/gitaly/service/diff/server.go
index edef73b66..72310b068 100644
--- a/internal/gitaly/service/diff/server.go
+++ b/internal/gitaly/service/diff/server.go
@@ -2,6 +2,7 @@ package diff
import (
"gitlab.com/gitlab-org/gitaly/internal/git"
+ "gitlab.com/gitlab-org/gitaly/internal/git/catfile"
"gitlab.com/gitlab-org/gitaly/internal/git/localrepo"
"gitlab.com/gitlab-org/gitaly/internal/git/repository"
"gitlab.com/gitlab-org/gitaly/internal/gitaly/config"
@@ -16,18 +17,20 @@ type server struct {
cfg config.Cfg
locator storage.Locator
gitCmdFactory git.CommandFactory
+ catfileCache catfile.Cache
}
// NewServer creates a new instance of a gRPC DiffServer
-func NewServer(cfg config.Cfg, locator storage.Locator, gitCmdFactory git.CommandFactory) gitalypb.DiffServiceServer {
+func NewServer(cfg config.Cfg, locator storage.Locator, gitCmdFactory git.CommandFactory, catfileCache catfile.Cache) gitalypb.DiffServiceServer {
return &server{
MsgSizeThreshold: msgSizeThreshold,
cfg: cfg,
locator: locator,
gitCmdFactory: gitCmdFactory,
+ catfileCache: catfileCache,
}
}
func (s *server) localrepo(repo repository.GitRepo) *localrepo.Repo {
- return localrepo.New(s.gitCmdFactory, repo, s.cfg)
+ return localrepo.New(s.gitCmdFactory, s.catfileCache, repo, s.cfg)
}
diff --git a/internal/gitaly/service/diff/testhelper_test.go b/internal/gitaly/service/diff/testhelper_test.go
index 00be98713..de31f8756 100644
--- a/internal/gitaly/service/diff/testhelper_test.go
+++ b/internal/gitaly/service/diff/testhelper_test.go
@@ -7,6 +7,7 @@ import (
"github.com/stretchr/testify/require"
"gitlab.com/gitlab-org/gitaly/internal/git"
+ "gitlab.com/gitlab-org/gitaly/internal/git/catfile"
"gitlab.com/gitlab-org/gitaly/internal/gitaly/config"
"gitlab.com/gitlab-org/gitaly/internal/testhelper"
"gitlab.com/gitlab-org/gitaly/internal/testhelper/testcfg"
@@ -44,7 +45,14 @@ func runDiffServer(t testing.TB, cfg config.Cfg) string {
listener, err := net.Listen("unix", serverSocketPath)
require.NoError(t, err)
- gitalypb.RegisterDiffServiceServer(server, NewServer(cfg, config.NewLocator(cfg), git.NewExecCommandFactory(cfg)))
+ gitCmdFactory := git.NewExecCommandFactory(cfg)
+
+ gitalypb.RegisterDiffServiceServer(server, NewServer(
+ cfg,
+ config.NewLocator(cfg),
+ gitCmdFactory,
+ catfile.NewCache(gitCmdFactory, cfg),
+ ))
go server.Serve(listener)
diff --git a/internal/gitaly/service/objectpool/alternates_test.go b/internal/gitaly/service/objectpool/alternates_test.go
index f4ebf689b..b32042513 100644
--- a/internal/gitaly/service/objectpool/alternates_test.go
+++ b/internal/gitaly/service/objectpool/alternates_test.go
@@ -21,7 +21,7 @@ func TestDisconnectGitAlternates(t *testing.T) {
defer cancel()
gitCmdFactory := git.NewExecCommandFactory(cfg)
- pool, err := objectpool.NewObjectPool(cfg, locator, gitCmdFactory, repo.GetStorageName(), gittest.NewObjectPoolName(t))
+ pool, err := objectpool.NewObjectPool(cfg, locator, gitCmdFactory, nil, repo.GetStorageName(), gittest.NewObjectPoolName(t))
require.NoError(t, err)
defer func() {
require.NoError(t, pool.Remove(ctx))
diff --git a/internal/gitaly/service/objectpool/create.go b/internal/gitaly/service/objectpool/create.go
index 7b614c8ff..1acf5a616 100644
--- a/internal/gitaly/service/objectpool/create.go
+++ b/internal/gitaly/service/objectpool/create.go
@@ -67,7 +67,7 @@ func (s *server) poolForRequest(req poolRequest) (*objectpool.ObjectPool, error)
return nil, errMissingPool
}
- pool, err := objectpool.NewObjectPool(s.cfg, s.locator, s.gitCmdFactory, poolRepo.GetStorageName(), poolRepo.GetRelativePath())
+ pool, err := objectpool.NewObjectPool(s.cfg, s.locator, s.gitCmdFactory, s.catfileCache, poolRepo.GetStorageName(), poolRepo.GetRelativePath())
if err != nil {
if err == objectpool.ErrInvalidPoolDir {
return nil, errInvalidPoolDir
diff --git a/internal/gitaly/service/objectpool/create_test.go b/internal/gitaly/service/objectpool/create_test.go
index 8171546a8..3e48e2c0a 100644
--- a/internal/gitaly/service/objectpool/create_test.go
+++ b/internal/gitaly/service/objectpool/create_test.go
@@ -22,7 +22,7 @@ func TestCreate(t *testing.T) {
ctx, cancel := testhelper.Context()
defer cancel()
- pool, err := objectpool.NewObjectPool(cfg, locator, git.NewExecCommandFactory(cfg), repo.GetStorageName(), gittest.NewObjectPoolName(t))
+ pool, err := objectpool.NewObjectPool(cfg, locator, git.NewExecCommandFactory(cfg), nil, repo.GetStorageName(), gittest.NewObjectPoolName(t))
require.NoError(t, err)
poolReq := &gitalypb.CreateObjectPoolRequest{
@@ -61,7 +61,7 @@ func TestUnsuccessfulCreate(t *testing.T) {
validPoolPath := gittest.NewObjectPoolName(t)
storageName := repo.GetStorageName()
- pool, err := objectpool.NewObjectPool(cfg, locator, git.NewExecCommandFactory(cfg), storageName, validPoolPath)
+ pool, err := objectpool.NewObjectPool(cfg, locator, git.NewExecCommandFactory(cfg), nil, storageName, validPoolPath)
require.NoError(t, err)
defer func() {
require.NoError(t, pool.Remove(ctx))
@@ -155,7 +155,7 @@ func TestDelete(t *testing.T) {
defer cancel()
validPoolPath := gittest.NewObjectPoolName(t)
- pool, err := objectpool.NewObjectPool(cfg, locator, git.NewExecCommandFactory(cfg), repo.GetStorageName(), validPoolPath)
+ pool, err := objectpool.NewObjectPool(cfg, locator, git.NewExecCommandFactory(cfg), nil, repo.GetStorageName(), validPoolPath)
require.NoError(t, err)
require.NoError(t, pool.Create(ctx, repo))
diff --git a/internal/gitaly/service/objectpool/fetch_into_object_pool.go b/internal/gitaly/service/objectpool/fetch_into_object_pool.go
index 1680afb1e..301884cde 100644
--- a/internal/gitaly/service/objectpool/fetch_into_object_pool.go
+++ b/internal/gitaly/service/objectpool/fetch_into_object_pool.go
@@ -16,7 +16,7 @@ func (s *server) FetchIntoObjectPool(ctx context.Context, req *gitalypb.FetchInt
return nil, helper.ErrInvalidArgument(err)
}
- objectPool, err := objectpool.FromProto(s.cfg, s.locator, s.gitCmdFactory, req.GetObjectPool())
+ objectPool, err := objectpool.FromProto(s.cfg, s.locator, s.gitCmdFactory, s.catfileCache, req.GetObjectPool())
if err != nil {
return nil, helper.ErrInvalidArgument(fmt.Errorf("object pool invalid: %v", err))
}
diff --git a/internal/gitaly/service/objectpool/fetch_into_object_pool_test.go b/internal/gitaly/service/objectpool/fetch_into_object_pool_test.go
index 61d5e172b..97478ff59 100644
--- a/internal/gitaly/service/objectpool/fetch_into_object_pool_test.go
+++ b/internal/gitaly/service/objectpool/fetch_into_object_pool_test.go
@@ -15,6 +15,7 @@ import (
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
"gitlab.com/gitlab-org/gitaly/internal/git"
+ "gitlab.com/gitlab-org/gitaly/internal/git/catfile"
"gitlab.com/gitlab-org/gitaly/internal/git/gittest"
"gitlab.com/gitlab-org/gitaly/internal/git/hooks"
"gitlab.com/gitlab-org/gitaly/internal/git/objectpool"
@@ -35,7 +36,7 @@ func TestFetchIntoObjectPool_Success(t *testing.T) {
repoCommit := gittest.WriteCommit(t, cfg, repoPath, gittest.WithBranch(t.Name()))
- pool, err := objectpool.NewObjectPool(cfg, locator, git.NewExecCommandFactory(cfg), repo.GetStorageName(), gittest.NewObjectPoolName(t))
+ pool, err := objectpool.NewObjectPool(cfg, locator, git.NewExecCommandFactory(cfg), nil, repo.GetStorageName(), gittest.NewObjectPoolName(t))
require.NoError(t, err)
defer func() {
require.NoError(t, pool.Remove(ctx))
@@ -90,7 +91,7 @@ func TestFetchIntoObjectPool_hooks(t *testing.T) {
ctx, cancel := testhelper.Context()
defer cancel()
- pool, err := objectpool.NewObjectPool(cfg, locator, gitCmdFactory, repo.GetStorageName(), gittest.NewObjectPoolName(t))
+ pool, err := objectpool.NewObjectPool(cfg, locator, gitCmdFactory, nil, repo.GetStorageName(), gittest.NewObjectPoolName(t))
require.NoError(t, err)
defer func() {
require.NoError(t, pool.Remove(ctx))
@@ -131,7 +132,7 @@ func TestFetchIntoObjectPool_CollectLogStatistics(t *testing.T) {
defer cancel()
ctx = ctxlogrus.ToContext(ctx, log.WithField("test", "logging"))
- pool, err := objectpool.NewObjectPool(cfg, locator, git.NewExecCommandFactory(cfg), repo.GetStorageName(), gittest.NewObjectPoolName(t))
+ pool, err := objectpool.NewObjectPool(cfg, locator, git.NewExecCommandFactory(cfg), nil, repo.GetStorageName(), gittest.NewObjectPoolName(t))
require.NoError(t, err)
defer func() {
require.NoError(t, pool.Remove(ctx))
@@ -167,12 +168,12 @@ func TestFetchIntoObjectPool_Failure(t *testing.T) {
locator := config.NewLocator(cfg)
gitCmdFactory := git.NewExecCommandFactory(cfg)
- server := NewServer(cfg, locator, gitCmdFactory)
+ server := NewServer(cfg, locator, gitCmdFactory, catfile.NewCache(gitCmdFactory, cfg))
ctx, cancel := testhelper.Context()
defer cancel()
- pool, err := objectpool.NewObjectPool(cfg, locator, gitCmdFactory, repos[0].StorageName, gittest.NewObjectPoolName(t))
+ pool, err := objectpool.NewObjectPool(cfg, locator, gitCmdFactory, nil, repos[0].StorageName, gittest.NewObjectPoolName(t))
require.NoError(t, err)
defer func() {
require.NoError(t, pool.Remove(ctx))
diff --git a/internal/gitaly/service/objectpool/get.go b/internal/gitaly/service/objectpool/get.go
index 22ab81898..1f12de05d 100644
--- a/internal/gitaly/service/objectpool/get.go
+++ b/internal/gitaly/service/objectpool/get.go
@@ -15,7 +15,7 @@ func (s *server) GetObjectPool(ctx context.Context, in *gitalypb.GetObjectPoolRe
return nil, helper.ErrInternal(errors.New("repository is empty"))
}
- objectPool, err := objectpool.FromRepo(s.cfg, s.locator, s.gitCmdFactory, in.GetRepository())
+ objectPool, err := objectpool.FromRepo(s.cfg, s.locator, s.gitCmdFactory, s.catfileCache, in.GetRepository())
if err != nil {
ctxlogrus.Extract(ctx).
diff --git a/internal/gitaly/service/objectpool/get_test.go b/internal/gitaly/service/objectpool/get_test.go
index 5d58fc8ff..c5f13a622 100644
--- a/internal/gitaly/service/objectpool/get_test.go
+++ b/internal/gitaly/service/objectpool/get_test.go
@@ -19,7 +19,7 @@ func TestGetObjectPoolSuccess(t *testing.T) {
relativePoolPath := gittest.NewObjectPoolName(t)
- pool, err := objectpool.NewObjectPool(cfg, locator, git.NewExecCommandFactory(cfg), repo.GetStorageName(), relativePoolPath)
+ pool, err := objectpool.NewObjectPool(cfg, locator, git.NewExecCommandFactory(cfg), nil, repo.GetStorageName(), relativePoolPath)
require.NoError(t, err)
poolCtx, cancel := testhelper.Context()
diff --git a/internal/gitaly/service/objectpool/link_test.go b/internal/gitaly/service/objectpool/link_test.go
index ba403e09e..0eeb08e75 100644
--- a/internal/gitaly/service/objectpool/link_test.go
+++ b/internal/gitaly/service/objectpool/link_test.go
@@ -25,7 +25,7 @@ func TestLink(t *testing.T) {
localRepo := localrepo.NewTestRepo(t, cfg, repo)
- pool, err := objectpool.NewObjectPool(cfg, locator, git.NewExecCommandFactory(cfg), repo.GetStorageName(), gittest.NewObjectPoolName(t))
+ pool, err := objectpool.NewObjectPool(cfg, locator, git.NewExecCommandFactory(cfg), nil, repo.GetStorageName(), gittest.NewObjectPoolName(t))
require.NoError(t, err)
require.NoError(t, pool.Remove(ctx), "make sure pool does not exist at start of test")
@@ -92,7 +92,7 @@ func TestLinkIdempotent(t *testing.T) {
ctx, cancel := testhelper.Context()
defer cancel()
- pool, err := objectpool.NewObjectPool(cfg, locator, git.NewExecCommandFactory(cfg), repo.GetStorageName(), gittest.NewObjectPoolName(t))
+ pool, err := objectpool.NewObjectPool(cfg, locator, git.NewExecCommandFactory(cfg), nil, repo.GetStorageName(), gittest.NewObjectPoolName(t))
require.NoError(t, err)
defer func() {
require.NoError(t, pool.Remove(ctx))
@@ -117,7 +117,7 @@ func TestLinkNoClobber(t *testing.T) {
ctx, cancel := testhelper.Context()
defer cancel()
- pool, err := objectpool.NewObjectPool(cfg, locator, git.NewExecCommandFactory(cfg), repo.GetStorageName(), gittest.NewObjectPoolName(t))
+ pool, err := objectpool.NewObjectPool(cfg, locator, git.NewExecCommandFactory(cfg), nil, repo.GetStorageName(), gittest.NewObjectPoolName(t))
require.NoError(t, err)
defer func() {
require.NoError(t, pool.Remove(ctx))
@@ -151,7 +151,7 @@ func TestLinkNoPool(t *testing.T) {
ctx, cancel := testhelper.Context()
defer cancel()
- pool, err := objectpool.NewObjectPool(cfg, locator, git.NewExecCommandFactory(cfg), repo.GetStorageName(), gittest.NewObjectPoolName(t))
+ pool, err := objectpool.NewObjectPool(cfg, locator, git.NewExecCommandFactory(cfg), nil, repo.GetStorageName(), gittest.NewObjectPoolName(t))
require.NoError(t, err)
// intentionally do not call pool.Create
defer func() {
@@ -182,7 +182,7 @@ func TestUnlink(t *testing.T) {
defer removeDeletedRepo()
gitCmdFactory := git.NewExecCommandFactory(cfg)
- pool, err := objectpool.NewObjectPool(cfg, locator, gitCmdFactory, repo.GetStorageName(), gittest.NewObjectPoolName(t))
+ pool, err := objectpool.NewObjectPool(cfg, locator, gitCmdFactory, nil, repo.GetStorageName(), gittest.NewObjectPoolName(t))
require.NoError(t, err)
defer func() {
require.NoError(t, pool.Remove(ctx))
@@ -195,7 +195,7 @@ func TestUnlink(t *testing.T) {
removeDeletedRepo()
testhelper.AssertPathNotExists(t, deletedRepoPath)
- pool2, err := objectpool.NewObjectPool(cfg, locator, gitCmdFactory, repo.GetStorageName(), gittest.NewObjectPoolName(t))
+ pool2, err := objectpool.NewObjectPool(cfg, locator, gitCmdFactory, nil, repo.GetStorageName(), gittest.NewObjectPoolName(t))
require.NoError(t, err)
require.NoError(t, pool2.Create(ctx, repo), "create pool 2")
defer func() {
@@ -288,7 +288,7 @@ func TestUnlinkIdempotent(t *testing.T) {
ctx, cancel := testhelper.Context()
defer cancel()
- pool, err := objectpool.NewObjectPool(cfg, locator, git.NewExecCommandFactory(cfg), repo.GetStorageName(), gittest.NewObjectPoolName(t))
+ pool, err := objectpool.NewObjectPool(cfg, locator, git.NewExecCommandFactory(cfg), nil, repo.GetStorageName(), gittest.NewObjectPoolName(t))
require.NoError(t, err)
defer func() {
require.NoError(t, pool.Remove(ctx))
diff --git a/internal/gitaly/service/objectpool/reduplicate_test.go b/internal/gitaly/service/objectpool/reduplicate_test.go
index a10517920..42abc62ea 100644
--- a/internal/gitaly/service/objectpool/reduplicate_test.go
+++ b/internal/gitaly/service/objectpool/reduplicate_test.go
@@ -19,7 +19,7 @@ func TestReduplicate(t *testing.T) {
defer cancel()
gitCmdFactory := git.NewExecCommandFactory(cfg)
- pool, err := objectpool.NewObjectPool(cfg, locator, gitCmdFactory, repo.GetStorageName(), gittest.NewObjectPoolName(t))
+ pool, err := objectpool.NewObjectPool(cfg, locator, gitCmdFactory, nil, repo.GetStorageName(), gittest.NewObjectPoolName(t))
require.NoError(t, err)
defer func() {
require.NoError(t, pool.Remove(ctx))
diff --git a/internal/gitaly/service/objectpool/server.go b/internal/gitaly/service/objectpool/server.go
index e281b3335..ab183d6b1 100644
--- a/internal/gitaly/service/objectpool/server.go
+++ b/internal/gitaly/service/objectpool/server.go
@@ -2,6 +2,7 @@ package objectpool
import (
"gitlab.com/gitlab-org/gitaly/internal/git"
+ "gitlab.com/gitlab-org/gitaly/internal/git/catfile"
"gitlab.com/gitlab-org/gitaly/internal/gitaly/config"
"gitlab.com/gitlab-org/gitaly/internal/storage"
"gitlab.com/gitlab-org/gitaly/proto/go/gitalypb"
@@ -11,9 +12,20 @@ type server struct {
cfg config.Cfg
locator storage.Locator
gitCmdFactory git.CommandFactory
+ catfileCache catfile.Cache
}
// NewServer creates a new instance of a gRPC repo server
-func NewServer(cfg config.Cfg, locator storage.Locator, gitCmdFactory git.CommandFactory) gitalypb.ObjectPoolServiceServer {
- return &server{cfg: cfg, locator: locator, gitCmdFactory: gitCmdFactory}
+func NewServer(
+ cfg config.Cfg,
+ locator storage.Locator,
+ gitCmdFactory git.CommandFactory,
+ catfileCache catfile.Cache,
+) gitalypb.ObjectPoolServiceServer {
+ return &server{
+ cfg: cfg,
+ locator: locator,
+ gitCmdFactory: gitCmdFactory,
+ catfileCache: catfileCache,
+ }
}
diff --git a/internal/gitaly/service/objectpool/testhelper_test.go b/internal/gitaly/service/objectpool/testhelper_test.go
index f4037a62b..f5b95d35a 100644
--- a/internal/gitaly/service/objectpool/testhelper_test.go
+++ b/internal/gitaly/service/objectpool/testhelper_test.go
@@ -9,6 +9,7 @@ import (
"github.com/stretchr/testify/require"
"gitlab.com/gitlab-org/gitaly/internal/backchannel"
"gitlab.com/gitlab-org/gitaly/internal/git"
+ "gitlab.com/gitlab-org/gitaly/internal/git/catfile"
"gitlab.com/gitlab-org/gitaly/internal/gitaly/config"
"gitlab.com/gitlab-org/gitaly/internal/gitaly/hook"
hookservice "gitlab.com/gitlab-org/gitaly/internal/gitaly/service/hook"
@@ -63,7 +64,7 @@ func runObjectPoolServer(t *testing.T, cfg config.Cfg, locator storage.Locator)
hookManager := hook.NewManager(locator, txManager, gitlab.NewMockClient(), cfg)
gitCmdFactory := git.NewExecCommandFactory(cfg)
- gitalypb.RegisterObjectPoolServiceServer(server, NewServer(cfg, locator, gitCmdFactory))
+ gitalypb.RegisterObjectPoolServiceServer(server, NewServer(cfg, locator, gitCmdFactory, catfile.NewCache(gitCmdFactory, cfg)))
gitalypb.RegisterHookServiceServer(server, hookservice.NewServer(cfg, hookManager, gitCmdFactory))
go server.Serve(listener)
diff --git a/internal/gitaly/service/operations/branches_test.go b/internal/gitaly/service/operations/branches_test.go
index 9c535eac2..9bcc0fba3 100644
--- a/internal/gitaly/service/operations/branches_test.go
+++ b/internal/gitaly/service/operations/branches_test.go
@@ -122,7 +122,15 @@ func TestUserCreateBranchWithTransaction(t *testing.T) {
cfg.ListenAddr = "127.0.0.1:0" // runs gitaly on the TCP address
addr := testserver.RunGitalyServer(t, cfg, nil, func(srv *grpc.Server, deps *service.Dependencies) {
- gitalypb.RegisterOperationServiceServer(srv, NewServer(deps.GetCfg(), nil, deps.GetHookManager(), deps.GetLocator(), deps.GetConnsPool(), deps.GetGitCmdFactory()))
+ gitalypb.RegisterOperationServiceServer(srv, NewServer(
+ deps.GetCfg(),
+ nil,
+ deps.GetHookManager(),
+ deps.GetLocator(),
+ deps.GetConnsPool(),
+ deps.GetGitCmdFactory(),
+ deps.GetCatfileCache(),
+ ))
gitalypb.RegisterHookServiceServer(srv, hook.NewServer(deps.GetCfg(), deps.GetHookManager(), deps.GetGitCmdFactory()))
// Praefect proxy execution disabled as praefect runs only on the UNIX socket, but
// the test requires a TCP listening address.
@@ -497,7 +505,15 @@ func TestUserDeleteBranch_transaction(t *testing.T) {
transactionServer := &testTransactionServer{}
testserver.RunGitalyServer(t, cfg, nil, func(srv *grpc.Server, deps *service.Dependencies) {
- gitalypb.RegisterOperationServiceServer(srv, NewServer(deps.GetCfg(), nil, deps.GetHookManager(), deps.GetLocator(), deps.GetConnsPool(), deps.GetGitCmdFactory()))
+ gitalypb.RegisterOperationServiceServer(srv, NewServer(
+ deps.GetCfg(),
+ nil,
+ deps.GetHookManager(),
+ deps.GetLocator(),
+ deps.GetConnsPool(),
+ deps.GetGitCmdFactory(),
+ deps.GetCatfileCache(),
+ ))
})
praefect := txinfo.PraefectServer{
diff --git a/internal/gitaly/service/operations/server.go b/internal/gitaly/service/operations/server.go
index 27dcea134..1dfb11630 100644
--- a/internal/gitaly/service/operations/server.go
+++ b/internal/gitaly/service/operations/server.go
@@ -5,6 +5,7 @@ import (
"gitlab.com/gitlab-org/gitaly/client"
"gitlab.com/gitlab-org/gitaly/internal/git"
+ "gitlab.com/gitlab-org/gitaly/internal/git/catfile"
"gitlab.com/gitlab-org/gitaly/internal/git/localrepo"
"gitlab.com/gitlab-org/gitaly/internal/git/repository"
"gitlab.com/gitlab-org/gitaly/internal/git2go"
@@ -22,10 +23,19 @@ type Server struct {
conns *client.Pool
git2go git2go.Executor
gitCmdFactory git.CommandFactory
+ catfileCache catfile.Cache
}
// NewServer creates a new instance of a grpc OperationServiceServer
-func NewServer(cfg config.Cfg, rs *rubyserver.Server, hookManager hook.Manager, locator storage.Locator, conns *client.Pool, gitCmdFactory git.CommandFactory) *Server {
+func NewServer(
+ cfg config.Cfg,
+ rs *rubyserver.Server,
+ hookManager hook.Manager,
+ locator storage.Locator,
+ conns *client.Pool,
+ gitCmdFactory git.CommandFactory,
+ catfileCache catfile.Cache,
+) *Server {
return &Server{
ruby: rs,
cfg: cfg,
@@ -34,9 +44,10 @@ func NewServer(cfg config.Cfg, rs *rubyserver.Server, hookManager hook.Manager,
conns: conns,
git2go: git2go.New(filepath.Join(cfg.BinDir, "gitaly-git2go"), cfg.Git.BinPath),
gitCmdFactory: gitCmdFactory,
+ catfileCache: catfileCache,
}
}
func (s *Server) localrepo(repo repository.GitRepo) *localrepo.Repo {
- return localrepo.New(s.gitCmdFactory, repo, s.cfg)
+ return localrepo.New(s.gitCmdFactory, s.catfileCache, repo, s.cfg)
}
diff --git a/internal/gitaly/service/operations/tags.go b/internal/gitaly/service/operations/tags.go
index d31a75e31..e4f1d7bcd 100644
--- a/internal/gitaly/service/operations/tags.go
+++ b/internal/gitaly/service/operations/tags.go
@@ -10,7 +10,6 @@ import (
"github.com/golang/protobuf/ptypes"
"gitlab.com/gitlab-org/gitaly/internal/git"
- "gitlab.com/gitlab-org/gitaly/internal/git/catfile"
"gitlab.com/gitlab-org/gitaly/internal/git/localrepo"
"gitlab.com/gitlab-org/gitaly/internal/git/log"
"gitlab.com/gitlab-org/gitaly/internal/helper"
@@ -93,7 +92,7 @@ func (s *Server) UserCreateTag(ctx context.Context, req *gitalypb.UserCreateTagR
// Setup
repo := req.GetRepository()
- catFile, err := catfile.New(ctx, s.gitCmdFactory, repo)
+ catFile, err := s.catfileCache.BatchProcess(ctx, repo)
if err != nil {
return nil, status.Error(codes.Internal, err.Error())
}
diff --git a/internal/gitaly/service/operations/tags_test.go b/internal/gitaly/service/operations/tags_test.go
index aebde8963..676d7faf5 100644
--- a/internal/gitaly/service/operations/tags_test.go
+++ b/internal/gitaly/service/operations/tags_test.go
@@ -277,7 +277,15 @@ func TestUserCreateTagWithTransaction(t *testing.T) {
// running Praefect tests, which would break our test setup.
transactionServer := &testTransactionServer{}
testserver.RunGitalyServer(t, cfg, nil, func(srv *grpc.Server, deps *service.Dependencies) {
- gitalypb.RegisterOperationServiceServer(srv, NewServer(deps.GetCfg(), nil, deps.GetHookManager(), deps.GetLocator(), deps.GetConnsPool(), deps.GetGitCmdFactory()))
+ gitalypb.RegisterOperationServiceServer(srv, NewServer(
+ deps.GetCfg(),
+ nil,
+ deps.GetHookManager(),
+ deps.GetLocator(),
+ deps.GetConnsPool(),
+ deps.GetGitCmdFactory(),
+ deps.GetCatfileCache(),
+ ))
gitalypb.RegisterHookServiceServer(srv, hook.NewServer(deps.GetCfg(), deps.GetHookManager(), deps.GetGitCmdFactory()))
})
diff --git a/internal/gitaly/service/operations/testhelper_test.go b/internal/gitaly/service/operations/testhelper_test.go
index d06fc6748..c2c3560ad 100644
--- a/internal/gitaly/service/operations/testhelper_test.go
+++ b/internal/gitaly/service/operations/testhelper_test.go
@@ -135,11 +135,38 @@ func runOperationServiceServer(t testing.TB, cfg config.Cfg, rubySrv *rubyserver
t.Helper()
return testserver.RunGitalyServer(t, cfg, rubySrv, func(srv *grpc.Server, deps *service.Dependencies) {
- gitalypb.RegisterOperationServiceServer(srv, NewServer(deps.GetCfg(), deps.GetRubyServer(), deps.GetHookManager(), deps.GetLocator(), deps.GetConnsPool(), deps.GetGitCmdFactory()))
+ gitalypb.RegisterOperationServiceServer(srv, NewServer(
+ deps.GetCfg(),
+ deps.GetRubyServer(),
+ deps.GetHookManager(),
+ deps.GetLocator(),
+ deps.GetConnsPool(),
+ deps.GetGitCmdFactory(),
+ deps.GetCatfileCache(),
+ ))
gitalypb.RegisterHookServiceServer(srv, hook.NewServer(cfg, deps.GetHookManager(), deps.GetGitCmdFactory()))
- gitalypb.RegisterRepositoryServiceServer(srv, repository.NewServer(cfg, rubySrv, deps.GetLocator(), deps.GetTxManager(), deps.GetGitCmdFactory()))
- gitalypb.RegisterRefServiceServer(srv, ref.NewServer(cfg, deps.GetLocator(), deps.GetGitCmdFactory(), deps.GetTxManager()))
- gitalypb.RegisterCommitServiceServer(srv, commit.NewServer(cfg, deps.GetLocator(), deps.GetGitCmdFactory(), nil))
+ gitalypb.RegisterRepositoryServiceServer(srv, repository.NewServer(
+ cfg,
+ rubySrv,
+ deps.GetLocator(),
+ deps.GetTxManager(),
+ deps.GetGitCmdFactory(),
+ deps.GetCatfileCache(),
+ ))
+ gitalypb.RegisterRefServiceServer(srv, ref.NewServer(
+ cfg,
+ deps.GetLocator(),
+ deps.GetGitCmdFactory(),
+ deps.GetTxManager(),
+ deps.GetCatfileCache(),
+ ))
+ gitalypb.RegisterCommitServiceServer(srv, commit.NewServer(
+ cfg,
+ deps.GetLocator(),
+ deps.GetGitCmdFactory(),
+ nil,
+ deps.GetCatfileCache(),
+ ))
gitalypb.RegisterSSHServiceServer(srv, ssh.NewServer(
cfg,
deps.GetLocator(),
diff --git a/internal/gitaly/service/operations/update_with_hooks_test.go b/internal/gitaly/service/operations/update_with_hooks_test.go
index 8e1c4e44e..f4e6cc4db 100644
--- a/internal/gitaly/service/operations/update_with_hooks_test.go
+++ b/internal/gitaly/service/operations/update_with_hooks_test.go
@@ -62,7 +62,7 @@ func TestUpdateReferenceWithHooks_invalidParameters(t *testing.T) {
revA, revB := git.ObjectID(strings.Repeat("a", 40)), git.ObjectID(strings.Repeat("b", 40))
- server := NewServer(cfg, nil, &mockHookManager{}, nil, nil, nil)
+ server := NewServer(cfg, nil, &mockHookManager{}, nil, nil, nil, nil)
testCases := []struct {
desc string
@@ -264,7 +264,7 @@ func TestUpdateReferenceWithHooks(t *testing.T) {
}
gitCmdFactory := git.NewExecCommandFactory(cfg)
- hookServer := NewServer(cfg, nil, hookManager, nil, nil, gitCmdFactory)
+ hookServer := NewServer(cfg, nil, hookManager, nil, nil, gitCmdFactory, nil)
err := hookServer.updateReferenceWithHooks(ctx, repo, user, git.ReferenceName("refs/heads/master"), git.ZeroOID, git.ObjectID(oldRev))
if tc.expectedErr == "" {
diff --git a/internal/gitaly/service/ref/delete_refs_test.go b/internal/gitaly/service/ref/delete_refs_test.go
index 8c243040d..966a9faa1 100644
--- a/internal/gitaly/service/ref/delete_refs_test.go
+++ b/internal/gitaly/service/ref/delete_refs_test.go
@@ -92,7 +92,13 @@ func TestDeleteRefs_transaction(t *testing.T) {
}
addr := testserver.RunGitalyServer(t, cfg, nil, func(srv *grpc.Server, deps *service.Dependencies) {
- gitalypb.RegisterRefServiceServer(srv, NewServer(deps.GetCfg(), deps.GetLocator(), deps.GetGitCmdFactory(), deps.GetTxManager()))
+ gitalypb.RegisterRefServiceServer(srv, NewServer(
+ deps.GetCfg(),
+ deps.GetLocator(),
+ deps.GetGitCmdFactory(),
+ deps.GetTxManager(),
+ deps.GetCatfileCache(),
+ ))
gitalypb.RegisterHookServiceServer(srv, hookservice.NewServer(deps.GetCfg(), deps.GetHookManager(), deps.GetGitCmdFactory()))
}, testserver.WithTransactionManager(txManager))
diff --git a/internal/gitaly/service/ref/list_new_blobs.go b/internal/gitaly/service/ref/list_new_blobs.go
index b55f498cb..0c194f4e9 100644
--- a/internal/gitaly/service/ref/list_new_blobs.go
+++ b/internal/gitaly/service/ref/list_new_blobs.go
@@ -6,7 +6,6 @@ import (
"strings"
"gitlab.com/gitlab-org/gitaly/internal/git"
- "gitlab.com/gitlab-org/gitaly/internal/git/catfile"
"gitlab.com/gitlab-org/gitaly/internal/helper"
"gitlab.com/gitlab-org/gitaly/proto/go/gitalypb"
)
@@ -38,7 +37,7 @@ func (s *server) listNewBlobs(in *gitalypb.ListNewBlobsRequest, stream gitalypb.
return err
}
- batch, err := catfile.New(ctx, s.gitCmdFactory, in.GetRepository())
+ batch, err := s.catfileCache.BatchProcess(ctx, in.GetRepository())
if err != nil {
return err
}
diff --git a/internal/gitaly/service/ref/list_new_commits.go b/internal/gitaly/service/ref/list_new_commits.go
index 7abeca35c..02938c61e 100644
--- a/internal/gitaly/service/ref/list_new_commits.go
+++ b/internal/gitaly/service/ref/list_new_commits.go
@@ -4,7 +4,6 @@ import (
"bufio"
"gitlab.com/gitlab-org/gitaly/internal/git"
- "gitlab.com/gitlab-org/gitaly/internal/git/catfile"
"gitlab.com/gitlab-org/gitaly/internal/git/log"
"gitlab.com/gitlab-org/gitaly/internal/helper"
"gitlab.com/gitlab-org/gitaly/proto/go/gitalypb"
@@ -35,7 +34,7 @@ func (s *server) listNewCommits(in *gitalypb.ListNewCommitsRequest, stream gital
return err
}
- batch, err := catfile.New(ctx, s.gitCmdFactory, in.GetRepository())
+ batch, err := s.catfileCache.BatchProcess(ctx, in.GetRepository())
if err != nil {
return err
}
diff --git a/internal/gitaly/service/ref/refs.go b/internal/gitaly/service/ref/refs.go
index f545da2a8..3b698121f 100644
--- a/internal/gitaly/service/ref/refs.go
+++ b/internal/gitaly/service/ref/refs.go
@@ -97,7 +97,7 @@ func (s *server) parseAndReturnTags(ctx context.Context, repo *gitalypb.Reposito
return fmt.Errorf("for-each-ref error: %v", err)
}
- c, err := catfile.New(ctx, s.gitCmdFactory, repo)
+ c, err := s.catfileCache.BatchProcess(ctx, repo)
if err != nil {
return fmt.Errorf("error creating catfile: %v", err)
}
@@ -301,7 +301,7 @@ func (s *server) FindLocalBranches(in *gitalypb.FindLocalBranchesRequest, stream
func (s *server) findLocalBranches(in *gitalypb.FindLocalBranchesRequest, stream gitalypb.RefService_FindLocalBranchesServer) error {
ctx := stream.Context()
- c, err := catfile.New(ctx, s.gitCmdFactory, in.Repository)
+ c, err := s.catfileCache.BatchProcess(ctx, in.Repository)
if err != nil {
return err
}
@@ -351,7 +351,7 @@ func (s *server) findAllBranches(in *gitalypb.FindAllBranchesRequest, stream git
}
ctx := stream.Context()
- c, err := catfile.New(ctx, s.gitCmdFactory, in.Repository)
+ c, err := s.catfileCache.BatchProcess(ctx, in.Repository)
if err != nil {
return err
}
@@ -428,7 +428,7 @@ func (s *server) findTag(ctx context.Context, repository *gitalypb.Repository, t
return nil, fmt.Errorf("for-each-ref error: %v", err)
}
- c, err := catfile.New(ctx, s.gitCmdFactory, repository)
+ c, err := s.catfileCache.BatchProcess(ctx, repository)
if err != nil {
return nil, err
}
diff --git a/internal/gitaly/service/ref/refs_test.go b/internal/gitaly/service/ref/refs_test.go
index 76dd16457..b8cef0429 100644
--- a/internal/gitaly/service/ref/refs_test.go
+++ b/internal/gitaly/service/ref/refs_test.go
@@ -645,7 +645,8 @@ func TestFindAllTagNestedTags(t *testing.T) {
tags := bytes.NewReader(testhelper.MustRunCommand(t, nil, "git", "-C", testRepoCopyPath, "tag"))
testhelper.MustRunCommand(t, tags, "xargs", cfg.Git.BinPath, "-C", testRepoCopyPath, "tag", "-d")
- batch, err := catfile.New(ctx, git.NewExecCommandFactory(cfg), testRepoCopy)
+ catfileCache := catfile.NewCache(git.NewExecCommandFactory(cfg), cfg)
+ batch, err := catfileCache.BatchProcess(ctx, testRepoCopy)
require.NoError(t, err)
info, err := batch.Info(ctx, git.Revision(tc.originalOid))
@@ -1489,7 +1490,8 @@ func TestFindTagNestedTag(t *testing.T) {
tags := bytes.NewReader(testhelper.MustRunCommand(t, nil, "git", "-C", repoPath, "tag"))
testhelper.MustRunCommand(t, tags, "xargs", cfg.Git.BinPath, "-C", repoPath, "tag", "-d")
- batch, err := catfile.New(ctx, git.NewExecCommandFactory(cfg), repo)
+ catfileCache := catfile.NewCache(git.NewExecCommandFactory(cfg), cfg)
+ batch, err := catfileCache.BatchProcess(ctx, repo)
require.NoError(t, err)
info, err := batch.Info(ctx, git.Revision(tc.originalOid))
diff --git a/internal/gitaly/service/ref/remote_branches.go b/internal/gitaly/service/ref/remote_branches.go
index f753c4c9a..01fc5ffc8 100644
--- a/internal/gitaly/service/ref/remote_branches.go
+++ b/internal/gitaly/service/ref/remote_branches.go
@@ -5,7 +5,6 @@ import (
"strings"
"gitlab.com/gitlab-org/gitaly/internal/git"
- "gitlab.com/gitlab-org/gitaly/internal/git/catfile"
"gitlab.com/gitlab-org/gitaly/internal/helper"
"gitlab.com/gitlab-org/gitaly/proto/go/gitalypb"
)
@@ -30,7 +29,7 @@ func (s *server) findAllRemoteBranches(req *gitalypb.FindAllRemoteBranchesReques
patterns := []string{"refs/remotes/" + req.GetRemoteName()}
ctx := stream.Context()
- c, err := catfile.New(ctx, s.gitCmdFactory, req.GetRepository())
+ c, err := s.catfileCache.BatchProcess(ctx, req.GetRepository())
if err != nil {
return err
}
diff --git a/internal/gitaly/service/ref/server.go b/internal/gitaly/service/ref/server.go
index 64b097528..0dfb457d4 100644
--- a/internal/gitaly/service/ref/server.go
+++ b/internal/gitaly/service/ref/server.go
@@ -2,6 +2,7 @@ package ref
import (
"gitlab.com/gitlab-org/gitaly/internal/git"
+ "gitlab.com/gitlab-org/gitaly/internal/git/catfile"
"gitlab.com/gitlab-org/gitaly/internal/git/localrepo"
"gitlab.com/gitlab-org/gitaly/internal/git/repository"
"gitlab.com/gitlab-org/gitaly/internal/gitaly/config"
@@ -15,18 +16,26 @@ type server struct {
txManager transaction.Manager
locator storage.Locator
gitCmdFactory git.CommandFactory
+ catfileCache catfile.Cache
}
// NewServer creates a new instance of a grpc RefServer
-func NewServer(cfg config.Cfg, locator storage.Locator, gitCmdFactory git.CommandFactory, txManager transaction.Manager) gitalypb.RefServiceServer {
+func NewServer(
+ cfg config.Cfg,
+ locator storage.Locator,
+ gitCmdFactory git.CommandFactory,
+ txManager transaction.Manager,
+ catfileCache catfile.Cache,
+) gitalypb.RefServiceServer {
return &server{
cfg: cfg,
txManager: txManager,
locator: locator,
gitCmdFactory: gitCmdFactory,
+ catfileCache: catfileCache,
}
}
func (s *server) localrepo(repo repository.GitRepo) *localrepo.Repo {
- return localrepo.New(s.gitCmdFactory, repo, s.cfg)
+ return localrepo.New(s.gitCmdFactory, s.catfileCache, repo, s.cfg)
}
diff --git a/internal/gitaly/service/ref/tag_messages.go b/internal/gitaly/service/ref/tag_messages.go
index b5c536926..4744a0af7 100644
--- a/internal/gitaly/service/ref/tag_messages.go
+++ b/internal/gitaly/service/ref/tag_messages.go
@@ -6,7 +6,6 @@ import (
"io"
"gitlab.com/gitlab-org/gitaly/internal/git"
- "gitlab.com/gitlab-org/gitaly/internal/git/catfile"
"gitlab.com/gitlab-org/gitaly/internal/git/log"
"gitlab.com/gitlab-org/gitaly/internal/helper"
"gitlab.com/gitlab-org/gitaly/proto/go/gitalypb"
@@ -37,7 +36,7 @@ func validateGetTagMessagesRequest(request *gitalypb.GetTagMessagesRequest) erro
func (s *server) getAndStreamTagMessages(request *gitalypb.GetTagMessagesRequest, stream gitalypb.RefService_GetTagMessagesServer) error {
ctx := stream.Context()
- c, err := catfile.New(ctx, s.gitCmdFactory, request.GetRepository())
+ c, err := s.catfileCache.BatchProcess(ctx, request.GetRepository())
if err != nil {
return err
}
diff --git a/internal/gitaly/service/ref/testhelper_test.go b/internal/gitaly/service/ref/testhelper_test.go
index 0d54f3b64..2854cd704 100644
--- a/internal/gitaly/service/ref/testhelper_test.go
+++ b/internal/gitaly/service/ref/testhelper_test.go
@@ -70,7 +70,13 @@ func setupRefServiceWithoutRepo(t testing.TB) (config.Cfg, gitalypb.RefServiceCl
func runRefServiceServer(t testing.TB, cfg config.Cfg) string {
return testserver.RunGitalyServer(t, cfg, nil, func(srv *grpc.Server, deps *service.Dependencies) {
- gitalypb.RegisterRefServiceServer(srv, NewServer(deps.GetCfg(), deps.GetLocator(), deps.GetGitCmdFactory(), deps.GetTxManager()))
+ gitalypb.RegisterRefServiceServer(srv, NewServer(
+ deps.GetCfg(),
+ deps.GetLocator(),
+ deps.GetGitCmdFactory(),
+ deps.GetTxManager(),
+ deps.GetCatfileCache(),
+ ))
gitalypb.RegisterHookServiceServer(srv, hookservice.NewServer(deps.GetCfg(), deps.GetHookManager(), deps.GetGitCmdFactory()))
})
}
diff --git a/internal/gitaly/service/remote/fetch_internal_remote_test.go b/internal/gitaly/service/remote/fetch_internal_remote_test.go
index c1060d809..f6848f36d 100644
--- a/internal/gitaly/service/remote/fetch_internal_remote_test.go
+++ b/internal/gitaly/service/remote/fetch_internal_remote_test.go
@@ -157,7 +157,13 @@ func TestSuccessfulFetchInternalRemote(t *testing.T) {
deps.GetGitCmdFactory(),
deps.GetTxManager(),
))
- gitalypb.RegisterRefServiceServer(srv, ref.NewServer(deps.GetCfg(), deps.GetLocator(), deps.GetGitCmdFactory(), deps.GetTxManager()))
+ gitalypb.RegisterRefServiceServer(srv, ref.NewServer(
+ deps.GetCfg(),
+ deps.GetLocator(),
+ deps.GetGitCmdFactory(),
+ deps.GetTxManager(),
+ deps.GetCatfileCache(),
+ ))
gitalypb.RegisterHookServiceServer(srv, hook.NewServer(deps.GetCfg(), deps.GetHookManager(), deps.GetGitCmdFactory()))
}, testserver.WithDisablePraefect())
@@ -175,7 +181,13 @@ func TestSuccessfulFetchInternalRemote(t *testing.T) {
hookManager := &mockHookManager{}
localAddr := testserver.RunGitalyServer(t, localCfg, nil, func(srv *grpc.Server, deps *service.Dependencies) {
- gitalypb.RegisterRemoteServiceServer(srv, NewServer(deps.GetCfg(), deps.GetRubyServer(), deps.GetLocator(), deps.GetGitCmdFactory()))
+ gitalypb.RegisterRemoteServiceServer(srv, NewServer(
+ deps.GetCfg(),
+ deps.GetRubyServer(),
+ deps.GetLocator(),
+ deps.GetGitCmdFactory(),
+ deps.GetCatfileCache(),
+ ))
gitalypb.RegisterHookServiceServer(srv, hook.NewServer(deps.GetCfg(), deps.GetHookManager(), deps.GetGitCmdFactory()))
}, testserver.WithHookManager(hookManager), testserver.WithDisablePraefect())
diff --git a/internal/gitaly/service/remote/server.go b/internal/gitaly/service/remote/server.go
index 91b83f9b6..b8b0ffae4 100644
--- a/internal/gitaly/service/remote/server.go
+++ b/internal/gitaly/service/remote/server.go
@@ -3,6 +3,7 @@ package remote
import (
"gitlab.com/gitlab-org/gitaly/client"
"gitlab.com/gitlab-org/gitaly/internal/git"
+ "gitlab.com/gitlab-org/gitaly/internal/git/catfile"
"gitlab.com/gitlab-org/gitaly/internal/git/localrepo"
"gitlab.com/gitlab-org/gitaly/internal/git/repository"
"gitlab.com/gitlab-org/gitaly/internal/gitaly/config"
@@ -16,17 +17,25 @@ type server struct {
ruby *rubyserver.Server
locator storage.Locator
gitCmdFactory git.CommandFactory
+ catfileCache catfile.Cache
conns *client.Pool
}
// NewServer creates a new instance of a grpc RemoteServiceServer
-func NewServer(cfg config.Cfg, rs *rubyserver.Server, locator storage.Locator, gitCmdFactory git.CommandFactory) gitalypb.RemoteServiceServer {
+func NewServer(
+ cfg config.Cfg,
+ rs *rubyserver.Server,
+ locator storage.Locator,
+ gitCmdFactory git.CommandFactory,
+ catfileCache catfile.Cache,
+) gitalypb.RemoteServiceServer {
return &server{
cfg: cfg,
ruby: rs,
locator: locator,
gitCmdFactory: gitCmdFactory,
+ catfileCache: catfileCache,
conns: client.NewPoolWithOptions(
client.WithDialer(client.HealthCheckDialer(client.DialContext)),
client.WithDialOptions(client.FailOnNonTempDialError()...),
@@ -35,5 +44,5 @@ func NewServer(cfg config.Cfg, rs *rubyserver.Server, locator storage.Locator, g
}
func (s *server) localrepo(repo repository.GitRepo) *localrepo.Repo {
- return localrepo.New(s.gitCmdFactory, repo, s.cfg)
+ return localrepo.New(s.gitCmdFactory, s.catfileCache, repo, s.cfg)
}
diff --git a/internal/gitaly/service/remote/testhelper_test.go b/internal/gitaly/service/remote/testhelper_test.go
index cf317c7e3..82492835b 100644
--- a/internal/gitaly/service/remote/testhelper_test.go
+++ b/internal/gitaly/service/remote/testhelper_test.go
@@ -61,7 +61,13 @@ func setupRemoteServiceWithRuby(t *testing.T, cfg config.Cfg, rubySrv *rubyserve
t.Cleanup(cleanup)
addr := testserver.RunGitalyServer(t, cfg, rubySrv, func(srv *grpc.Server, deps *service.Dependencies) {
- gitalypb.RegisterRemoteServiceServer(srv, NewServer(deps.GetCfg(), deps.GetRubyServer(), deps.GetLocator(), deps.GetGitCmdFactory()))
+ gitalypb.RegisterRemoteServiceServer(srv, NewServer(
+ deps.GetCfg(),
+ deps.GetRubyServer(),
+ deps.GetLocator(),
+ deps.GetGitCmdFactory(),
+ deps.GetCatfileCache(),
+ ))
})
cfg.SocketPath = addr
diff --git a/internal/gitaly/service/remote/update_remote_mirror_test.go b/internal/gitaly/service/remote/update_remote_mirror_test.go
index fb19ffbb6..c4b4c4481 100644
--- a/internal/gitaly/service/remote/update_remote_mirror_test.go
+++ b/internal/gitaly/service/remote/update_remote_mirror_test.go
@@ -450,7 +450,13 @@ func testSuccessfulUpdateRemoteMirrorRequest(t *testing.T, cfg config.Cfg, rubyS
func testSuccessfulUpdateRemoteMirrorRequestFeatured(t *testing.T, ctx context.Context, cfg config.Cfg, rubySrv *rubyserver.Server) {
serverSocketPath := testserver.RunGitalyServer(t, cfg, rubySrv, func(srv *grpc.Server, deps *service.Dependencies) {
- gitalypb.RegisterRemoteServiceServer(srv, NewServer(deps.GetCfg(), deps.GetRubyServer(), deps.GetLocator(), deps.GetGitCmdFactory()))
+ gitalypb.RegisterRemoteServiceServer(srv, NewServer(
+ deps.GetCfg(),
+ deps.GetRubyServer(),
+ deps.GetLocator(),
+ deps.GetGitCmdFactory(),
+ deps.GetCatfileCache(),
+ ))
})
client, conn := newRemoteClient(t, serverSocketPath)
@@ -550,7 +556,13 @@ func testSuccessfulUpdateRemoteMirrorRequestWithWildcards(t *testing.T, cfg conf
func testSuccessfulUpdateRemoteMirrorRequestWithWildcardsFeatured(t *testing.T, ctx context.Context, cfg config.Cfg, rubySrv *rubyserver.Server) {
serverSocketPath := testserver.RunGitalyServer(t, cfg, rubySrv, func(srv *grpc.Server, deps *service.Dependencies) {
- gitalypb.RegisterRemoteServiceServer(srv, NewServer(deps.GetCfg(), deps.GetRubyServer(), deps.GetLocator(), deps.GetGitCmdFactory()))
+ gitalypb.RegisterRemoteServiceServer(srv, NewServer(
+ deps.GetCfg(),
+ deps.GetRubyServer(),
+ deps.GetLocator(),
+ deps.GetGitCmdFactory(),
+ deps.GetCatfileCache(),
+ ))
})
client, conn := newRemoteClient(t, serverSocketPath)
@@ -634,7 +646,13 @@ func testSuccessfulUpdateRemoteMirrorRequestWithKeepDivergentRefs(t *testing.T,
func testSuccessfulUpdateRemoteMirrorRequestWithKeepDivergentRefsFeatured(t *testing.T, ctx context.Context, cfg config.Cfg, rubySrv *rubyserver.Server) {
serverSocketPath := testserver.RunGitalyServer(t, cfg, rubySrv, func(srv *grpc.Server, deps *service.Dependencies) {
- gitalypb.RegisterRemoteServiceServer(srv, NewServer(deps.GetCfg(), deps.GetRubyServer(), deps.GetLocator(), deps.GetGitCmdFactory()))
+ gitalypb.RegisterRemoteServiceServer(srv, NewServer(
+ deps.GetCfg(),
+ deps.GetRubyServer(),
+ deps.GetLocator(),
+ deps.GetGitCmdFactory(),
+ deps.GetCatfileCache(),
+ ))
})
client, conn := newRemoteClient(t, serverSocketPath)
@@ -720,7 +738,13 @@ func testFailedUpdateRemoteMirrorRequestDueToValidation(t *testing.T, cfg config
func testFailedUpdateRemoteMirrorRequestDueToValidationFeatured(t *testing.T, ctx context.Context, cfg config.Cfg, rubySrv *rubyserver.Server) {
serverSocketPath := testserver.RunGitalyServer(t, cfg, rubySrv, func(srv *grpc.Server, deps *service.Dependencies) {
- gitalypb.RegisterRemoteServiceServer(srv, NewServer(deps.GetCfg(), deps.GetRubyServer(), deps.GetLocator(), deps.GetGitCmdFactory()))
+ gitalypb.RegisterRemoteServiceServer(srv, NewServer(
+ deps.GetCfg(),
+ deps.GetRubyServer(),
+ deps.GetLocator(),
+ deps.GetGitCmdFactory(),
+ deps.GetCatfileCache(),
+ ))
})
client, conn := newRemoteClient(t, serverSocketPath)
diff --git a/internal/gitaly/service/repository/apply_gitattributes.go b/internal/gitaly/service/repository/apply_gitattributes.go
index 92c46056e..4f615ba9d 100644
--- a/internal/gitaly/service/repository/apply_gitattributes.go
+++ b/internal/gitaly/service/repository/apply_gitattributes.go
@@ -131,7 +131,7 @@ func (s *server) ApplyGitattributes(ctx context.Context, in *gitalypb.ApplyGitat
return nil, status.Errorf(codes.InvalidArgument, "ApplyGitAttributes: revision: %v", err)
}
- c, err := catfile.New(ctx, s.gitCmdFactory, repo)
+ c, err := s.catfileCache.BatchProcess(ctx, repo)
if err != nil {
return nil, err
}
diff --git a/internal/gitaly/service/repository/apply_gitattributes_test.go b/internal/gitaly/service/repository/apply_gitattributes_test.go
index 5f58f8572..07169f013 100644
--- a/internal/gitaly/service/repository/apply_gitattributes_test.go
+++ b/internal/gitaly/service/repository/apply_gitattributes_test.go
@@ -88,7 +88,14 @@ func TestApplyGitattributesWithTransaction(t *testing.T) {
transactionServer := &testTransactionServer{}
testserver.RunGitalyServer(t, cfg, nil, func(srv *grpc.Server, deps *service.Dependencies) {
- gitalypb.RegisterRepositoryServiceServer(srv, NewServer(deps.GetCfg(), deps.GetRubyServer(), deps.GetLocator(), deps.GetTxManager(), deps.GetGitCmdFactory()))
+ gitalypb.RegisterRepositoryServiceServer(srv, NewServer(
+ deps.GetCfg(),
+ deps.GetRubyServer(),
+ deps.GetLocator(),
+ deps.GetTxManager(),
+ deps.GetGitCmdFactory(),
+ deps.GetCatfileCache(),
+ ))
})
// We're using internal listener in order to route around
diff --git a/internal/gitaly/service/repository/archive.go b/internal/gitaly/service/repository/archive.go
index 027f43cbb..c105a7c4d 100644
--- a/internal/gitaly/service/repository/archive.go
+++ b/internal/gitaly/service/repository/archive.go
@@ -12,7 +12,6 @@ import (
"gitlab.com/gitlab-org/gitaly/internal/command"
"gitlab.com/gitlab-org/gitaly/internal/git"
- "gitlab.com/gitlab-org/gitaly/internal/git/catfile"
"gitlab.com/gitlab-org/gitaly/internal/gitaly/service/commit"
"gitlab.com/gitlab-org/gitaly/internal/helper"
"gitlab.com/gitlab-org/gitaly/internal/log"
@@ -135,7 +134,7 @@ func validateGetArchiveRequest(in *gitalypb.GetArchiveRequest, format string, pa
}
func (s *server) validateGetArchivePrecondition(ctx context.Context, in *gitalypb.GetArchiveRequest, path string, exclude []string) error {
- c, err := catfile.New(ctx, s.gitCmdFactory, in.GetRepository())
+ c, err := s.catfileCache.BatchProcess(ctx, in.GetRepository())
if err != nil {
return err
}
diff --git a/internal/gitaly/service/repository/clone_from_pool.go b/internal/gitaly/service/repository/clone_from_pool.go
index 8dacfc4a7..f2281a530 100644
--- a/internal/gitaly/service/repository/clone_from_pool.go
+++ b/internal/gitaly/service/repository/clone_from_pool.go
@@ -32,7 +32,7 @@ func (s *server) CloneFromPool(ctx context.Context, req *gitalypb.CloneFromPoolR
return nil, helper.ErrInternalf("fetch http remote: %v", err)
}
- objectPool, err := objectpool.FromProto(s.cfg, s.locator, s.gitCmdFactory, req.GetPool())
+ objectPool, err := objectpool.FromProto(s.cfg, s.locator, s.gitCmdFactory, s.catfileCache, req.GetPool())
if err != nil {
return nil, helper.ErrInternalf("get object pool from request: %v", err)
}
@@ -54,7 +54,7 @@ func (s *server) validateCloneFromPoolRequestRepositoryState(req *gitalypb.Clone
return errors.New("target reopsitory already exists")
}
- objectPool, err := objectpool.FromProto(s.cfg, s.locator, s.gitCmdFactory, req.GetPool())
+ objectPool, err := objectpool.FromProto(s.cfg, s.locator, s.gitCmdFactory, s.catfileCache, req.GetPool())
if err != nil {
return fmt.Errorf("getting object pool from repository: %v", err)
}
diff --git a/internal/gitaly/service/repository/clone_from_pool_internal.go b/internal/gitaly/service/repository/clone_from_pool_internal.go
index cf1ee1cd1..ccacc52e8 100644
--- a/internal/gitaly/service/repository/clone_from_pool_internal.go
+++ b/internal/gitaly/service/repository/clone_from_pool_internal.go
@@ -46,7 +46,7 @@ func (s *server) CloneFromPoolInternal(ctx context.Context, req *gitalypb.CloneF
return nil, helper.ErrInternalf("fetch internal remote failed")
}
- objectPool, err := objectpool.FromProto(s.cfg, s.locator, s.gitCmdFactory, req.GetPool())
+ objectPool, err := objectpool.FromProto(s.cfg, s.locator, s.gitCmdFactory, s.catfileCache, req.GetPool())
if err != nil {
return nil, helper.ErrInternalf("get object pool from request: %v", err)
}
@@ -68,7 +68,7 @@ func (s *server) validateCloneFromPoolInternalRequestRepositoryState(req *gitaly
return errors.New("target reopsitory already exists")
}
- objectPool, err := objectpool.FromProto(s.cfg, s.locator, s.gitCmdFactory, req.GetPool())
+ objectPool, err := objectpool.FromProto(s.cfg, s.locator, s.gitCmdFactory, s.catfileCache, req.GetPool())
if err != nil {
return fmt.Errorf("getting object pool from repository: %v", err)
}
diff --git a/internal/gitaly/service/repository/clone_from_pool_internal_test.go b/internal/gitaly/service/repository/clone_from_pool_internal_test.go
index 8f899bb76..6103a0dee 100644
--- a/internal/gitaly/service/repository/clone_from_pool_internal_test.go
+++ b/internal/gitaly/service/repository/clone_from_pool_internal_test.go
@@ -8,6 +8,7 @@ import (
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
"gitlab.com/gitlab-org/gitaly/internal/git"
+ "gitlab.com/gitlab-org/gitaly/internal/git/catfile"
"gitlab.com/gitlab-org/gitaly/internal/git/gittest"
"gitlab.com/gitlab-org/gitaly/internal/git/objectpool"
"gitlab.com/gitlab-org/gitaly/internal/gitaly/config"
@@ -22,7 +23,16 @@ func newTestObjectPool(t *testing.T, cfg config.Cfg) (*objectpool.ObjectPool, *g
relativePath := gittest.NewObjectPoolName(t)
repo := gittest.InitRepoDir(t, cfg.Storages[0].Path, relativePath)
- pool, err := objectpool.NewObjectPool(cfg, config.NewLocator(cfg), git.NewExecCommandFactory(cfg), repo.GetStorageName(), relativePath)
+ gitCmdFactory := git.NewExecCommandFactory(cfg)
+
+ pool, err := objectpool.NewObjectPool(
+ cfg,
+ config.NewLocator(cfg),
+ gitCmdFactory,
+ catfile.NewCache(gitCmdFactory, cfg),
+ repo.GetStorageName(),
+ relativePath,
+ )
require.NoError(t, err)
return pool, repo
diff --git a/internal/gitaly/service/repository/fetch_remote_test.go b/internal/gitaly/service/repository/fetch_remote_test.go
index b0e3aaace..6e652502a 100644
--- a/internal/gitaly/service/repository/fetch_remote_test.go
+++ b/internal/gitaly/service/repository/fetch_remote_test.go
@@ -219,7 +219,14 @@ func TestFetchRemote_transaction(t *testing.T) {
txManager := &mockTxManager{}
addr := testserver.RunGitalyServer(t, sourceCfg, nil, func(srv *grpc.Server, deps *service.Dependencies) {
- gitalypb.RegisterRepositoryServiceServer(srv, NewServer(deps.GetCfg(), deps.GetRubyServer(), deps.GetLocator(), deps.GetTxManager(), deps.GetGitCmdFactory()))
+ gitalypb.RegisterRepositoryServiceServer(srv, NewServer(
+ deps.GetCfg(),
+ deps.GetRubyServer(),
+ deps.GetLocator(),
+ deps.GetTxManager(),
+ deps.GetGitCmdFactory(),
+ deps.GetCatfileCache(),
+ ))
}, testserver.WithTransactionManager(txManager))
client := newRepositoryClient(t, sourceCfg, addr)
diff --git a/internal/gitaly/service/repository/fork_test.go b/internal/gitaly/service/repository/fork_test.go
index fb7243a8a..8d0911834 100644
--- a/internal/gitaly/service/repository/fork_test.go
+++ b/internal/gitaly/service/repository/fork_test.go
@@ -13,6 +13,7 @@ import (
gclient "gitlab.com/gitlab-org/gitaly/client"
"gitlab.com/gitlab-org/gitaly/internal/backchannel"
"gitlab.com/gitlab-org/gitaly/internal/git"
+ "gitlab.com/gitlab-org/gitaly/internal/git/catfile"
"gitlab.com/gitlab-org/gitaly/internal/gitaly/config"
"gitlab.com/gitlab-org/gitaly/internal/gitaly/hook"
"gitlab.com/gitlab-org/gitaly/internal/gitaly/rubyserver"
@@ -219,13 +220,14 @@ func runSecureServer(t *testing.T, cfg config.Cfg, rubySrv *rubyserver.Server) s
txManager := transaction.NewManager(cfg, registry)
hookManager := hook.NewManager(locator, txManager, gitlab.NewMockClient(), cfg)
gitCmdFactory := git.NewExecCommandFactory(cfg)
+ catfileCache := catfile.NewCache(gitCmdFactory, cfg)
- gitalypb.RegisterRepositoryServiceServer(server, NewServer(cfg, rubySrv, locator, txManager, gitCmdFactory))
+ gitalypb.RegisterRepositoryServiceServer(server, NewServer(cfg, rubySrv, locator, txManager, gitCmdFactory, catfileCache))
gitalypb.RegisterHookServiceServer(server, hookservice.NewServer(cfg, hookManager, gitCmdFactory))
- gitalypb.RegisterRemoteServiceServer(server, remote.NewServer(cfg, rubySrv, locator, gitCmdFactory))
+ gitalypb.RegisterRemoteServiceServer(server, remote.NewServer(cfg, rubySrv, locator, gitCmdFactory, catfileCache))
gitalypb.RegisterSSHServiceServer(server, ssh.NewServer(cfg, locator, gitCmdFactory, txManager))
- gitalypb.RegisterRefServiceServer(server, ref.NewServer(cfg, locator, gitCmdFactory, txManager))
- gitalypb.RegisterCommitServiceServer(server, commit.NewServer(cfg, locator, gitCmdFactory, nil))
+ gitalypb.RegisterRefServiceServer(server, ref.NewServer(cfg, locator, gitCmdFactory, txManager, catfileCache))
+ gitalypb.RegisterCommitServiceServer(server, commit.NewServer(cfg, locator, gitCmdFactory, nil, catfileCache))
errQ := make(chan error, 1)
// This creates a secondary GRPC server which isn't "secure". Reusing
diff --git a/internal/gitaly/service/repository/gc.go b/internal/gitaly/service/repository/gc.go
index 925ef121a..3f5b3cc8c 100644
--- a/internal/gitaly/service/repository/gc.go
+++ b/internal/gitaly/service/repository/gc.go
@@ -119,7 +119,7 @@ func (s *server) cleanupKeepArounds(ctx context.Context, repo *localrepo.Repo) e
return nil
}
- batch, err := catfile.New(ctx, s.gitCmdFactory, repo)
+ batch, err := s.catfileCache.BatchProcess(ctx, repo)
if err != nil {
return nil
}
diff --git a/internal/gitaly/service/repository/raw_changes.go b/internal/gitaly/service/repository/raw_changes.go
index f8744072a..afea0c48a 100644
--- a/internal/gitaly/service/repository/raw_changes.go
+++ b/internal/gitaly/service/repository/raw_changes.go
@@ -21,7 +21,7 @@ func (s *server) GetRawChanges(req *gitalypb.GetRawChangesRequest, stream gitaly
ctx := stream.Context()
repo := req.Repository
- batch, err := catfile.New(stream.Context(), s.gitCmdFactory, repo)
+ batch, err := s.catfileCache.BatchProcess(stream.Context(), repo)
if err != nil {
return helper.ErrInternal(err)
}
diff --git a/internal/gitaly/service/repository/replicate_test.go b/internal/gitaly/service/repository/replicate_test.go
index 15dbf3f77..847c33b08 100644
--- a/internal/gitaly/service/repository/replicate_test.go
+++ b/internal/gitaly/service/repository/replicate_test.go
@@ -12,6 +12,7 @@ import (
"github.com/stretchr/testify/require"
"gitlab.com/gitlab-org/gitaly/internal/backchannel"
"gitlab.com/gitlab-org/gitaly/internal/git"
+ "gitlab.com/gitlab-org/gitaly/internal/git/catfile"
"gitlab.com/gitlab-org/gitaly/internal/git/gittest"
"gitlab.com/gitlab-org/gitaly/internal/gitaly/config"
"gitlab.com/gitlab-org/gitaly/internal/gitaly/transaction"
@@ -304,7 +305,16 @@ func runServerWithBadFetchInternalRemote(t *testing.T, cfg config.Cfg) string {
internalListener, err := net.Listen("unix", cfg.GitalyInternalSocketPath())
require.NoError(t, err)
- gitalypb.RegisterRepositoryServiceServer(server, NewServer(cfg, nil, config.NewLocator(cfg), transaction.NewManager(cfg, backchannel.NewRegistry()), git.NewExecCommandFactory(cfg)))
+ gitCmdFactory := git.NewExecCommandFactory(cfg)
+
+ gitalypb.RegisterRepositoryServiceServer(server, NewServer(
+ cfg,
+ nil,
+ config.NewLocator(cfg),
+ transaction.NewManager(cfg, backchannel.NewRegistry()),
+ gitCmdFactory,
+ catfile.NewCache(gitCmdFactory, cfg),
+ ))
gitalypb.RegisterRemoteServiceServer(server, &mockRemoteServer{})
reflection.Register(server)
diff --git a/internal/gitaly/service/repository/search_files_test.go b/internal/gitaly/service/repository/search_files_test.go
index 8a4e03986..cfaf90dc5 100644
--- a/internal/gitaly/service/repository/search_files_test.go
+++ b/internal/gitaly/service/repository/search_files_test.go
@@ -12,6 +12,7 @@ import (
"github.com/stretchr/testify/require"
"gitlab.com/gitlab-org/gitaly/internal/backchannel"
"gitlab.com/gitlab-org/gitaly/internal/git"
+ "gitlab.com/gitlab-org/gitaly/internal/git/catfile"
"gitlab.com/gitlab-org/gitaly/internal/gitaly/config"
"gitlab.com/gitlab-org/gitaly/internal/gitaly/transaction"
"gitlab.com/gitlab-org/gitaly/internal/testhelper"
@@ -201,7 +202,16 @@ func TestSearchFilesByContentLargeFile(t *testing.T) {
func TestSearchFilesByContentFailure(t *testing.T) {
cfg, repo, _ := testcfg.BuildWithRepo(t)
- server := NewServer(cfg, nil, config.NewLocator(cfg), transaction.NewManager(cfg, backchannel.NewRegistry()), git.NewExecCommandFactory(cfg))
+ gitCommandFactory := git.NewExecCommandFactory(cfg)
+
+ server := NewServer(
+ cfg,
+ nil,
+ config.NewLocator(cfg),
+ transaction.NewManager(cfg, backchannel.NewRegistry()),
+ gitCommandFactory,
+ catfile.NewCache(gitCommandFactory, cfg),
+ )
testCases := []struct {
desc string
@@ -313,7 +323,16 @@ func TestSearchFilesByNameSuccessful(t *testing.T) {
func TestSearchFilesByNameFailure(t *testing.T) {
cfg := testcfg.Build(t)
- server := NewServer(cfg, nil, config.NewLocator(cfg), transaction.NewManager(cfg, backchannel.NewRegistry()), git.NewExecCommandFactory(cfg))
+ gitCommandFactory := git.NewExecCommandFactory(cfg)
+
+ server := NewServer(
+ cfg,
+ nil,
+ config.NewLocator(cfg),
+ transaction.NewManager(cfg, backchannel.NewRegistry()),
+ gitCommandFactory,
+ catfile.NewCache(gitCommandFactory, cfg),
+ )
testCases := []struct {
desc string
diff --git a/internal/gitaly/service/repository/server.go b/internal/gitaly/service/repository/server.go
index f4224bb35..5b93c6309 100644
--- a/internal/gitaly/service/repository/server.go
+++ b/internal/gitaly/service/repository/server.go
@@ -3,6 +3,7 @@ package repository
import (
"gitlab.com/gitlab-org/gitaly/client"
"gitlab.com/gitlab-org/gitaly/internal/git"
+ "gitlab.com/gitlab-org/gitaly/internal/git/catfile"
"gitlab.com/gitlab-org/gitaly/internal/git/localrepo"
"gitlab.com/gitlab-org/gitaly/internal/git/repository"
"gitlab.com/gitlab-org/gitaly/internal/gitaly/config"
@@ -21,6 +22,7 @@ type server struct {
cfg config.Cfg
binDir string
loggingCfg config.Logging
+ catfileCache catfile.Cache
}
// NewServer creates a new instance of a gRPC repo server
@@ -30,6 +32,7 @@ func NewServer(
locator storage.Locator,
txManager transaction.Manager,
gitCmdFactory git.CommandFactory,
+ catfileCache catfile.Cache,
) gitalypb.RepositoryServiceServer {
return &server{
ruby: rs,
@@ -40,12 +43,13 @@ func NewServer(
client.WithDialer(client.HealthCheckDialer(client.DialContext)),
client.WithDialOptions(client.FailOnNonTempDialError()...),
),
- cfg: cfg,
- binDir: cfg.BinDir,
- loggingCfg: cfg.Logging,
+ cfg: cfg,
+ binDir: cfg.BinDir,
+ loggingCfg: cfg.Logging,
+ catfileCache: catfileCache,
}
}
func (s *server) localrepo(repo repository.GitRepo) *localrepo.Repo {
- return localrepo.New(s.gitCmdFactory, repo, s.cfg)
+ return localrepo.New(s.gitCmdFactory, s.catfileCache, repo, s.cfg)
}
diff --git a/internal/gitaly/service/repository/snapshot_test.go b/internal/gitaly/service/repository/snapshot_test.go
index 869b3afb8..c7281c923 100644
--- a/internal/gitaly/service/repository/snapshot_test.go
+++ b/internal/gitaly/service/repository/snapshot_test.go
@@ -129,9 +129,10 @@ func TestGetSnapshotWithDedupe(t *testing.T) {
locator := config.NewLocator(cfg)
gitCmdFactory := git.NewExecCommandFactory(cfg)
+ catfileCache := catfile.NewCache(gitCmdFactory, cfg)
// ensure commit cannot be found in current repository
- c, err := catfile.New(ctx, gitCmdFactory, repo)
+ c, err := catfileCache.BatchProcess(ctx, repo)
require.NoError(t, err)
_, err = c.Info(ctx, git.Revision(originalAlternatesCommit))
require.True(t, catfile.IsNotFound(err))
@@ -148,7 +149,7 @@ func TestGetSnapshotWithDedupe(t *testing.T) {
"commit", "--allow-empty", "-m", "Another empty commit")
commitSha = gittest.CreateCommitInAlternateObjectDirectory(t, cfg.Git.BinPath, repoPath, alternateObjDir, cmd)
- c, err = catfile.New(ctx, gitCmdFactory, repo)
+ c, err = catfileCache.BatchProcess(ctx, repo)
require.NoError(t, err)
_, err = c.Info(ctx, git.Revision(commitSha))
require.NoError(t, err)
diff --git a/internal/gitaly/service/repository/testhelper_test.go b/internal/gitaly/service/repository/testhelper_test.go
index 98c54b50e..a1960c51e 100644
--- a/internal/gitaly/service/repository/testhelper_test.go
+++ b/internal/gitaly/service/repository/testhelper_test.go
@@ -119,17 +119,42 @@ func assertModTimeAfter(t *testing.T, afterTime time.Time, paths ...string) bool
func runRepositoryServerWithConfig(t testing.TB, cfg config.Cfg, rubySrv *rubyserver.Server, opts ...testserver.GitalyServerOpt) string {
return testserver.RunGitalyServer(t, cfg, rubySrv, func(srv *grpc.Server, deps *service.Dependencies) {
- gitalypb.RegisterRepositoryServiceServer(srv, NewServer(cfg, deps.GetRubyServer(), deps.GetLocator(), deps.GetTxManager(), deps.GetGitCmdFactory()))
+ gitalypb.RegisterRepositoryServiceServer(srv, NewServer(
+ cfg,
+ deps.GetRubyServer(),
+ deps.GetLocator(),
+ deps.GetTxManager(),
+ deps.GetGitCmdFactory(),
+ deps.GetCatfileCache(),
+ ))
gitalypb.RegisterHookServiceServer(srv, hookservice.NewServer(cfg, deps.GetHookManager(), deps.GetGitCmdFactory()))
- gitalypb.RegisterRemoteServiceServer(srv, remote.NewServer(cfg, rubySrv, deps.GetLocator(), deps.GetGitCmdFactory()))
+ gitalypb.RegisterRemoteServiceServer(srv, remote.NewServer(
+ cfg,
+ rubySrv,
+ deps.GetLocator(),
+ deps.GetGitCmdFactory(),
+ deps.GetCatfileCache(),
+ ))
gitalypb.RegisterSSHServiceServer(srv, ssh.NewServer(
cfg,
deps.GetLocator(),
deps.GetGitCmdFactory(),
deps.GetTxManager(),
))
- gitalypb.RegisterRefServiceServer(srv, ref.NewServer(cfg, deps.GetLocator(), deps.GetGitCmdFactory(), deps.GetTxManager()))
- gitalypb.RegisterCommitServiceServer(srv, commit.NewServer(cfg, deps.GetLocator(), deps.GetGitCmdFactory(), nil))
+ gitalypb.RegisterRefServiceServer(srv, ref.NewServer(
+ cfg,
+ deps.GetLocator(),
+ deps.GetGitCmdFactory(),
+ deps.GetTxManager(),
+ deps.GetCatfileCache(),
+ ))
+ gitalypb.RegisterCommitServiceServer(srv, commit.NewServer(
+ cfg,
+ deps.GetLocator(),
+ deps.GetGitCmdFactory(),
+ nil,
+ deps.GetCatfileCache(),
+ ))
}, opts...)
}
diff --git a/internal/gitaly/service/setup/register.go b/internal/gitaly/service/setup/register.go
index d913ed1bc..6d2f07e4d 100644
--- a/internal/gitaly/service/setup/register.go
+++ b/internal/gitaly/service/setup/register.go
@@ -53,14 +53,55 @@ var (
// RegisterAll will register all the known gRPC services on the provided gRPC service instance.
func RegisterAll(srv *grpc.Server, deps *service.Dependencies) {
- gitalypb.RegisterBlobServiceServer(srv, blob.NewServer(deps.GetCfg(), deps.GetLocator(), deps.GetGitCmdFactory()))
- gitalypb.RegisterCleanupServiceServer(srv, cleanup.NewServer(deps.GetCfg(), deps.GetGitCmdFactory()))
- gitalypb.RegisterCommitServiceServer(srv, commit.NewServer(deps.GetCfg(), deps.GetLocator(), deps.GetGitCmdFactory(), deps.GetLinguist()))
- gitalypb.RegisterDiffServiceServer(srv, diff.NewServer(deps.GetCfg(), deps.GetLocator(), deps.GetGitCmdFactory()))
+ gitalypb.RegisterBlobServiceServer(srv, blob.NewServer(
+ deps.GetCfg(),
+ deps.GetLocator(),
+ deps.GetGitCmdFactory(),
+ deps.GetCatfileCache(),
+ ))
+ gitalypb.RegisterCleanupServiceServer(srv, cleanup.NewServer(
+ deps.GetCfg(),
+ deps.GetGitCmdFactory(),
+ deps.GetCatfileCache(),
+ ))
+ gitalypb.RegisterCommitServiceServer(srv, commit.NewServer(
+ deps.GetCfg(),
+ deps.GetLocator(),
+ deps.GetGitCmdFactory(),
+ deps.GetLinguist(),
+ deps.GetCatfileCache(),
+ ))
+ gitalypb.RegisterDiffServiceServer(srv, diff.NewServer(
+ deps.GetCfg(),
+ deps.GetLocator(),
+ deps.GetGitCmdFactory(),
+ deps.GetCatfileCache(),
+ ))
gitalypb.RegisterNamespaceServiceServer(srv, namespace.NewServer(deps.GetLocator()))
- gitalypb.RegisterOperationServiceServer(srv, operations.NewServer(deps.GetCfg(), deps.GetRubyServer(), deps.GetHookManager(), deps.GetLocator(), deps.GetConnsPool(), deps.GetGitCmdFactory()))
- gitalypb.RegisterRefServiceServer(srv, ref.NewServer(deps.GetCfg(), deps.GetLocator(), deps.GetGitCmdFactory(), deps.GetTxManager()))
- gitalypb.RegisterRepositoryServiceServer(srv, repository.NewServer(deps.GetCfg(), deps.GetRubyServer(), deps.GetLocator(), deps.GetTxManager(), deps.GetGitCmdFactory()))
+ gitalypb.RegisterOperationServiceServer(srv, operations.NewServer(
+ deps.GetCfg(),
+ deps.GetRubyServer(),
+ deps.GetHookManager(),
+ deps.GetLocator(),
+ deps.GetConnsPool(),
+ deps.GetGitCmdFactory(),
+ deps.GetCatfileCache(),
+ ))
+ gitalypb.RegisterRefServiceServer(srv, ref.NewServer(
+ deps.GetCfg(),
+ deps.GetLocator(),
+ deps.GetGitCmdFactory(),
+ deps.GetTxManager(),
+ deps.GetCatfileCache(),
+ ))
+ gitalypb.RegisterRepositoryServiceServer(srv, repository.NewServer(
+ deps.GetCfg(),
+ deps.GetRubyServer(),
+ deps.GetLocator(),
+ deps.GetTxManager(),
+ deps.GetGitCmdFactory(),
+ deps.GetCatfileCache(),
+ ))
gitalypb.RegisterSSHServiceServer(srv, ssh.NewServer(
deps.GetCfg(),
deps.GetLocator(),
@@ -75,10 +116,26 @@ func RegisterAll(srv *grpc.Server, deps *service.Dependencies) {
smarthttp.WithPackfileNegotiationMetrics(smarthttpPackfileNegotiationMetrics),
))
gitalypb.RegisterWikiServiceServer(srv, wiki.NewServer(deps.GetRubyServer(), deps.GetLocator()))
- gitalypb.RegisterConflictsServiceServer(srv, conflicts.NewServer(deps.GetCfg(), deps.GetLocator(), deps.GetGitCmdFactory()))
- gitalypb.RegisterRemoteServiceServer(srv, remote.NewServer(deps.GetCfg(), deps.GetRubyServer(), deps.GetLocator(), deps.GetGitCmdFactory()))
+ gitalypb.RegisterConflictsServiceServer(srv, conflicts.NewServer(
+ deps.GetCfg(),
+ deps.GetLocator(),
+ deps.GetGitCmdFactory(),
+ deps.GetCatfileCache(),
+ ))
+ gitalypb.RegisterRemoteServiceServer(srv, remote.NewServer(
+ deps.GetCfg(),
+ deps.GetRubyServer(),
+ deps.GetLocator(),
+ deps.GetGitCmdFactory(),
+ deps.GetCatfileCache(),
+ ))
gitalypb.RegisterServerServiceServer(srv, server.NewServer(deps.GetGitCmdFactory(), deps.GetCfg().Storages))
- gitalypb.RegisterObjectPoolServiceServer(srv, objectpool.NewServer(deps.GetCfg(), deps.GetLocator(), deps.GetGitCmdFactory()))
+ gitalypb.RegisterObjectPoolServiceServer(srv, objectpool.NewServer(
+ deps.GetCfg(),
+ deps.GetLocator(),
+ deps.GetGitCmdFactory(),
+ deps.GetCatfileCache(),
+ ))
gitalypb.RegisterHookServiceServer(srv, hook.NewServer(deps.GetCfg(), deps.GetHookManager(), deps.GetGitCmdFactory()))
gitalypb.RegisterInternalGitalyServer(srv, internalgitaly.NewServer(deps.GetCfg().Storages))
diff --git a/internal/gitaly/service/smarthttp/inforefs_test.go b/internal/gitaly/service/smarthttp/inforefs_test.go
index 481c2fe3c..29a86aa1d 100644
--- a/internal/gitaly/service/smarthttp/inforefs_test.go
+++ b/internal/gitaly/service/smarthttp/inforefs_test.go
@@ -179,7 +179,14 @@ func TestObjectPoolRefAdvertisementHiding(t *testing.T) {
ctx, cancel := testhelper.Context()
defer cancel()
- pool, err := objectpool.NewObjectPool(cfg, config.NewLocator(cfg), git.NewExecCommandFactory(cfg), repo.GetStorageName(), gittest.NewObjectPoolName(t))
+ pool, err := objectpool.NewObjectPool(
+ cfg,
+ config.NewLocator(cfg),
+ git.NewExecCommandFactory(cfg),
+ nil,
+ repo.GetStorageName(),
+ gittest.NewObjectPoolName(t),
+ )
require.NoError(t, err)
require.NoError(t, pool.Create(ctx, repo))
diff --git a/internal/gitaly/service/ssh/receive_pack_test.go b/internal/gitaly/service/ssh/receive_pack_test.go
index 1996d88ba..550c9c923 100644
--- a/internal/gitaly/service/ssh/receive_pack_test.go
+++ b/internal/gitaly/service/ssh/receive_pack_test.go
@@ -221,7 +221,14 @@ func TestObjectPoolRefAdvertisementHidingSSH(t *testing.T) {
stream, err := client.SSHReceivePack(ctx)
require.NoError(t, err)
- pool, err := objectpool.NewObjectPool(cfg, config.NewLocator(cfg), git.NewExecCommandFactory(cfg), repo.GetStorageName(), gittest.NewObjectPoolName(t))
+ pool, err := objectpool.NewObjectPool(
+ cfg,
+ config.NewLocator(cfg),
+ git.NewExecCommandFactory(cfg),
+ nil,
+ repo.GetStorageName(),
+ gittest.NewObjectPoolName(t),
+ )
require.NoError(t, err)
require.NoError(t, pool.Create(ctx, repo))
diff --git a/internal/middleware/commandstatshandler/commandstatshandler_test.go b/internal/middleware/commandstatshandler/commandstatshandler_test.go
index 740929ba9..0bef7c360 100644
--- a/internal/middleware/commandstatshandler/commandstatshandler_test.go
+++ b/internal/middleware/commandstatshandler/commandstatshandler_test.go
@@ -13,6 +13,7 @@ import (
"github.com/stretchr/testify/require"
"gitlab.com/gitlab-org/gitaly/internal/backchannel"
"gitlab.com/gitlab-org/gitaly/internal/git"
+ "gitlab.com/gitlab-org/gitaly/internal/git/catfile"
"gitlab.com/gitlab-org/gitaly/internal/gitaly/config"
"gitlab.com/gitlab-org/gitaly/internal/gitaly/service/ref"
"gitlab.com/gitlab-org/gitaly/internal/gitaly/transaction"
@@ -46,7 +47,15 @@ func createNewServer(t *testing.T, cfg config.Cfg) *grpc.Server {
server := grpc.NewServer(opts...)
- gitalypb.RegisterRefServiceServer(server, ref.NewServer(cfg, config.NewLocator(cfg), git.NewExecCommandFactory(cfg), transaction.NewManager(cfg, backchannel.NewRegistry())))
+ gitCommandFactory := git.NewExecCommandFactory(cfg)
+
+ gitalypb.RegisterRefServiceServer(server, ref.NewServer(
+ cfg,
+ config.NewLocator(cfg),
+ gitCommandFactory,
+ transaction.NewManager(cfg, backchannel.NewRegistry()),
+ catfile.NewCache(gitCommandFactory, cfg),
+ ))
return server
}
diff --git a/internal/praefect/info_service_test.go b/internal/praefect/info_service_test.go
index c3ae1d5f7..7c5386673 100644
--- a/internal/praefect/info_service_test.go
+++ b/internal/praefect/info_service_test.go
@@ -32,7 +32,14 @@ func TestInfoService_RepositoryReplicas(t *testing.T) {
}
cfgs = append(cfgs, cfg)
cfgs[i].SocketPath = testserver.RunGitalyServer(t, cfgs[i], nil, func(srv *grpc.Server, deps *service.Dependencies) {
- gitalypb.RegisterRepositoryServiceServer(srv, repository.NewServer(deps.GetCfg(), deps.GetRubyServer(), deps.GetLocator(), deps.GetTxManager(), deps.GetGitCmdFactory()))
+ gitalypb.RegisterRepositoryServiceServer(srv, repository.NewServer(
+ deps.GetCfg(),
+ deps.GetRubyServer(),
+ deps.GetLocator(),
+ deps.GetTxManager(),
+ deps.GetGitCmdFactory(),
+ deps.GetCatfileCache(),
+ ))
}, testserver.WithDisablePraefect())
cfgNodes = append(cfgNodes, &config.Node{
Storage: cfgs[i].Storages[0].Name,
diff --git a/internal/praefect/replicator_test.go b/internal/praefect/replicator_test.go
index 7f39b1473..913ef21f0 100644
--- a/internal/praefect/replicator_test.go
+++ b/internal/praefect/replicator_test.go
@@ -92,7 +92,7 @@ func TestReplMgr_ProcessBacklog(t *testing.T) {
// create object pool on the source
objectPoolPath := gittest.NewObjectPoolName(t)
- pool, err := objectpool.NewObjectPool(primaryCfg, gconfig.NewLocator(primaryCfg), git.NewExecCommandFactory(primaryCfg), testRepo.GetStorageName(), objectPoolPath)
+ pool, err := objectpool.NewObjectPool(primaryCfg, gconfig.NewLocator(primaryCfg), git.NewExecCommandFactory(primaryCfg), nil, testRepo.GetStorageName(), objectPoolPath)
require.NoError(t, err)
poolCtx, cancel := testhelper.Context()
diff --git a/internal/testhelper/testserver/gitaly.go b/internal/testhelper/testserver/gitaly.go
index 4f6cce06c..5be0b5480 100644
--- a/internal/testhelper/testserver/gitaly.go
+++ b/internal/testhelper/testserver/gitaly.go
@@ -17,6 +17,7 @@ import (
"gitlab.com/gitlab-org/gitaly/client"
"gitlab.com/gitlab-org/gitaly/internal/backchannel"
"gitlab.com/gitlab-org/gitaly/internal/git"
+ "gitlab.com/gitlab-org/gitaly/internal/git/catfile"
"gitlab.com/gitlab-org/gitaly/internal/gitaly/config"
"gitlab.com/gitlab-org/gitaly/internal/gitaly/config/auth"
gitalylog "gitlab.com/gitlab-org/gitaly/internal/gitaly/config/log"
@@ -269,6 +270,7 @@ type gitalyServerDeps struct {
gitCmdFactory git.CommandFactory
linguist *linguist.Instance
backchannelReg *backchannel.Registry
+ catfileCache catfile.Cache
}
func (gsd *gitalyServerDeps) createDependencies(t testing.TB, cfg config.Cfg, rubyServer *rubyserver.Server) *service.Dependencies {
@@ -310,6 +312,10 @@ func (gsd *gitalyServerDeps) createDependencies(t testing.TB, cfg config.Cfg, ru
require.NoError(t, err)
}
+ if gsd.catfileCache == nil {
+ gsd.catfileCache = catfile.NewCache(gsd.gitCmdFactory, cfg)
+ }
+
return &service.Dependencies{
Cfg: cfg,
RubyServer: rubyServer,
@@ -321,6 +327,7 @@ func (gsd *gitalyServerDeps) createDependencies(t testing.TB, cfg config.Cfg, ru
Linguist: gsd.linguist,
BackchannelRegistry: gsd.backchannelReg,
GitlabClient: gsd.gitlabClient,
+ CatfileCache: gsd.catfileCache,
}
}
@@ -382,3 +389,11 @@ func WithBackchannelRegistry(backchannelReg *backchannel.Registry) GitalyServerO
return deps
}
}
+
+// WithCatfileCache sets catfile.Cache instance that will be used for gitaly services initialisation.
+func WithCatfileCache(catfileCache catfile.Cache) GitalyServerOpt {
+ return func(deps gitalyServerDeps) gitalyServerDeps {
+ deps.catfileCache = catfileCache
+ return deps
+ }
+}