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:
authorPatrick Steinhardt <psteinhardt@gitlab.com>2022-07-15 10:38:29 +0300
committerPatrick Steinhardt <psteinhardt@gitlab.com>2022-07-15 10:38:29 +0300
commit9c2d53c9d31da39b0e34c38b930c8b878fbd23e8 (patch)
tree51993b4df4ef9a78876ab33746438cdfc79ca6f6
parent339277bb1945df868accc7a5d3b43e98c7ba2a57 (diff)
parent410295810f9b3e7d94d0ce07b1a3ad1682023746 (diff)
Merge branch 'pks-object-pools-optimize-repository' into 'master'
objectpool: Switch to use OptimizeRepository for pool repos Closes #4338 See merge request gitlab-org/gitaly!4708
-rw-r--r--internal/git/gittest/delta_islands.go76
-rw-r--r--internal/git/gittest/repo.go34
-rw-r--r--internal/git/housekeeping/object_pool.go14
-rw-r--r--internal/git/housekeeping/object_pool_test.go51
-rw-r--r--internal/git/housekeeping/objects.go22
-rw-r--r--internal/git/housekeeping/objects_test.go13
-rw-r--r--internal/git/housekeeping/optimize_repository.go37
-rw-r--r--internal/git/housekeeping/optimize_repository_test.go291
-rw-r--r--internal/git/housekeeping/testhelper_test.go14
-rw-r--r--internal/git/objectpool/fetch.go46
-rw-r--r--internal/git/objectpool/fetch_test.go92
-rw-r--r--internal/git/objectpool/pool.go8
-rw-r--r--internal/git/objectpool/testhelper_test.go5
-rw-r--r--internal/git/reference.go4
-rw-r--r--internal/gitaly/service/objectpool/fetch_into_object_pool_test.go38
-rw-r--r--internal/gitaly/service/objectpool/testhelper_test.go8
-rw-r--r--internal/gitaly/service/repository/gc.go2
-rw-r--r--internal/gitaly/service/repository/gc_test.go2
-rw-r--r--internal/gitaly/service/repository/midx.go2
-rw-r--r--internal/gitaly/service/repository/repack_test.go2
-rw-r--r--internal/metadata/featureflag/ff_fetch_into_object_pool_optimize_repository.go10
-rw-r--r--internal/praefect/delete_object_pool.go2
-rw-r--r--internal/praefect/praefectutil/replica_path.go8
-rw-r--r--internal/praefect/praefectutil/replica_path_test.go42
-rw-r--r--internal/praefect/router_per_repository.go6
25 files changed, 572 insertions, 257 deletions
diff --git a/internal/git/gittest/delta_islands.go b/internal/git/gittest/delta_islands.go
index 3096d72a2..b54b72c73 100644
--- a/internal/git/gittest/delta_islands.go
+++ b/internal/git/gittest/delta_islands.go
@@ -1,56 +1,60 @@
package gittest
import (
- "crypto/rand"
- "io"
"strings"
"testing"
- "github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
"gitlab.com/gitlab-org/gitaly/v15/internal/git"
"gitlab.com/gitlab-org/gitaly/v15/internal/gitaly/config"
+ "gitlab.com/gitlab-org/gitaly/v15/internal/helper/text"
)
-// TestDeltaIslands is based on the tests in
-// https://github.com/git/git/blob/master/t/t5320-delta-islands.sh .
-func TestDeltaIslands(t *testing.T, cfg config.Cfg, repoPath string, repack func() error) {
+// TestDeltaIslands checks whether functions that repack objects in a repository correctly set up
+// delta islands. Based on https://github.com/git/git/blob/master/t/t5320-delta-islands.sh.
+func TestDeltaIslands(t *testing.T, cfg config.Cfg, repoPath string, isPoolRepo bool, repack func() error) {
t.Helper()
// Create blobs that we expect Git to use delta compression on.
- blob1, err := io.ReadAll(io.LimitReader(rand.Reader, 100000))
- require.NoError(t, err)
-
- blob2 := append(blob1, "\nblob 2"...)
-
- // Assume Git prefers the largest blob as the delta base.
- badBlob := append(blob2, "\nbad blob"...)
-
- blob1ID := commitBlob(t, cfg, repoPath, "refs/heads/branch1", blob1)
- blob2ID := commitBlob(t, cfg, repoPath, "refs/tags/tag2", blob2)
-
- // The bad blob will only be reachable via a non-standard ref. Because of
- // that it should be excluded from delta chains in the main island.
- badBlobID := commitBlob(t, cfg, repoPath, "refs/bad/ref3", badBlob)
-
- // So far we have create blobs and commits but they will be in loose
- // object files; we want them to be delta compressed. Run repack to make
- // that happen.
+ blob1 := strings.Repeat("X", 100000)
+ blob2 := blob1 + "\nblob 2"
+ // Create another, third blob that is longer than the second blob. Git prefers the largest
+ // blob as delta base, which means that it should in theory pick this blob. But we will make
+ // it reachable via a reference that is not part of the delta island, and thus it should not
+ // be used as delta base.
+ badBlob := blob2 + "\nbad blob"
+
+ refsPrefix := "refs"
+ if isPoolRepo {
+ // Pool repositories use different references for their delta islands, so we need to
+ // adapt accordingly.
+ refsPrefix = git.ObjectPoolRefNamespace
+ }
+
+ // Make the first two blobs reachable via references that are part of the delta island.
+ blob1ID := commitBlob(t, cfg, repoPath, refsPrefix+"/heads/branch1", blob1)
+ blob2ID := commitBlob(t, cfg, repoPath, refsPrefix+"/tags/tag2", blob2)
+
+ // The bad blob will only be reachable via a reference that is not covered by a delta
+ // island. Because of that it should be excluded from delta chains in the main island.
+ badBlobID := commitBlob(t, cfg, repoPath, refsPrefix+"/bad/ref3", badBlob)
+
+ // Repack all objects into a single pack so that we can verify that delta chains are built
+ // by Git as expected. Most notably, we don't use the delta islands here yet and thus the
+ // delta base for both blob1 and blob2 should be the bad blob.
Exec(t, cfg, "-C", repoPath, "repack", "-ad")
+ require.Equal(t, badBlobID, deltaBase(t, cfg, repoPath, blob1ID), "expect blob 1 delta base to be bad blob after test setup")
+ require.Equal(t, badBlobID, deltaBase(t, cfg, repoPath, blob2ID), "expect blob 2 delta base to be bad blob after test setup")
- assert.Equal(t, badBlobID, deltaBase(t, cfg, repoPath, blob1ID), "expect blob 1 delta base to be bad blob after test setup")
- assert.Equal(t, badBlobID, deltaBase(t, cfg, repoPath, blob2ID), "expect blob 2 delta base to be bad blob after test setup")
-
+ // Now we run the repacking function provided to us by the caller. We expect it to use delta
+ // chains, and thus neither of the two blobs should use the bad blob as delta base.
require.NoError(t, repack(), "repack after delta island setup")
-
- assert.Equal(t, blob2ID, deltaBase(t, cfg, repoPath, blob1ID), "blob 1 delta base should be blob 2 after repack")
-
- // blob2 is the bigger of the two so it should be the delta base
- assert.Equal(t, git.ZeroOID.String(), deltaBase(t, cfg, repoPath, blob2ID), "blob 2 should not be delta compressed after repack")
+ require.Equal(t, blob2ID, deltaBase(t, cfg, repoPath, blob1ID), "blob 1 delta base should be blob 2 after repack")
+ require.Equal(t, git.ZeroOID.String(), deltaBase(t, cfg, repoPath, blob2ID), "blob 2 should not be delta compressed after repack")
}
-func commitBlob(t *testing.T, cfg config.Cfg, repoPath, ref string, content []byte) string {
- blobID := WriteBlob(t, cfg, repoPath, content)
+func commitBlob(t *testing.T, cfg config.Cfg, repoPath, ref string, content string) string {
+ blobID := WriteBlob(t, cfg, repoPath, []byte(content))
// No parent, that means this will be an initial commit. Not very
// realistic but it doesn't matter for delta compression.
@@ -71,7 +75,5 @@ func deltaBase(t *testing.T, cfg config.Cfg, repoPath string, blobID string) str
"-C", repoPath, "cat-file", "--batch-check=%(deltabase)",
)
- return chompToString(catfileOut)
+ return text.ChompBytes(catfileOut)
}
-
-func chompToString(s []byte) string { return strings.TrimSuffix(string(s), "\n") }
diff --git a/internal/git/gittest/repo.go b/internal/git/gittest/repo.go
index c34f24548..ff833b5ed 100644
--- a/internal/git/gittest/repo.go
+++ b/internal/git/gittest/repo.go
@@ -371,3 +371,37 @@ func AddWorktreeArgs(repoPath, worktreeName string) []string {
func AddWorktree(t testing.TB, cfg config.Cfg, repoPath string, worktreeName string) {
Exec(t, cfg, AddWorktreeArgs(repoPath, worktreeName)...)
}
+
+// FixGitLabTestRepoForCommitGraphs fixes the "gitlab-test.git" repository so that it can be used in
+// the context of commit-graphs. The test repository contains the commit ba3343b (Weird commit date,
+// 292278994-08-17). As you can already see, this commit has a commit year of 292278994, which is
+// not exactly a realistic commit date to have in normal repositories. Unfortunately, this commit
+// date causes commit-graphs to become corrupt with the following error that's likely caused by
+// an overflow:
+//
+// commit date for commit ba3343bc4fa403a8dfbfcab7fc1a8c29ee34bd69 in commit-graph is 15668040695 != 9223372036854775
+//
+// This is not a new error, but something that has existed for quite a while already in Git. And
+// while the bug can also be easily hit in Gitaly because we do write commit-graphs in pool
+// repositories, until now we haven't because we never exercised this.
+//
+// Unfortunately, we're between a rock and a hard place: this error will be hit when running
+// git-fsck(1) to find dangling objects, which we do to rescue objects. git-fsck(1) will by default
+// verify the commit-graphs to be consistent even with `--connectivity-only`, which causes the
+// error. But while we could in theory just disable the usage of commit-graphs by passing
+// `core.commitGraph=0`, the end result would be that the connectivity check itself may become a lot
+// slower.
+//
+// So for now we just bail on this whole topic: it's not a new bug and we can't do much about it
+// given it could regress performance. The pool members would be broken in the same way, even though
+// less visibly so because we don't git-fsck(1) in "normal" RPCs. But to make our tests work we
+// delete the reference for this specific commit so that it doesn't cause our tests to break.
+//
+// You can easily test whether this bug still exists via the following commands:
+//
+// $ git clone _build/testrepos/gitlab-test.git
+// $ git -C gitlab-test commit-graph write
+// $ git -C gitlab-test commit-graph verify
+func FixGitLabTestRepoForCommitGraphs(tb testing.TB, cfg config.Cfg, repoPath string) {
+ Exec(tb, cfg, "-C", repoPath, "update-ref", "-d", "refs/heads/spooky-stuff", "ba3343bc4fa403a8dfbfcab7fc1a8c29ee34bd69")
+}
diff --git a/internal/git/housekeeping/object_pool.go b/internal/git/housekeeping/object_pool.go
index 6af1a9973..732045e09 100644
--- a/internal/git/housekeeping/object_pool.go
+++ b/internal/git/housekeeping/object_pool.go
@@ -4,15 +4,16 @@ import (
"regexp"
"strings"
+ "gitlab.com/gitlab-org/gitaly/v15/internal/git/repository"
"gitlab.com/gitlab-org/gitaly/v15/internal/praefect/praefectutil"
)
// railsPoolDirRegexp is used to validate object pool directory structure and name as generated by Rails.
var railsPoolDirRegexp = regexp.MustCompile(`^@pools/([0-9a-f]{2})/([0-9a-f]{2})/([0-9a-f]{64})\.git$`)
-// IsRailsPoolPath returns whether the relative path indicates this is a pool path generated by Rails.
-func IsRailsPoolPath(relativePath string) bool {
- matches := railsPoolDirRegexp.FindStringSubmatch(relativePath)
+// IsRailsPoolRepository returns whether the repository is a pool repository generated by Rails.
+func IsRailsPoolRepository(repo repository.GitRepo) bool {
+ matches := railsPoolDirRegexp.FindStringSubmatch(repo.GetRelativePath())
if matches == nil || !strings.HasPrefix(matches[3], matches[1]+matches[2]) {
return false
}
@@ -20,8 +21,7 @@ func IsRailsPoolPath(relativePath string) bool {
return true
}
-// IsPoolPath returns whether the relative path indicates the repository is an object
-// pool.
-func IsPoolPath(relativePath string) bool {
- return IsRailsPoolPath(relativePath) || praefectutil.IsPoolPath(relativePath)
+// IsPoolRepository returns whether the repository is an object pool.
+func IsPoolRepository(repo repository.GitRepo) bool {
+ return IsRailsPoolRepository(repo) || praefectutil.IsPoolRepository(repo)
}
diff --git a/internal/git/housekeeping/object_pool_test.go b/internal/git/housekeeping/object_pool_test.go
index 7c0bf5422..daf134505 100644
--- a/internal/git/housekeeping/object_pool_test.go
+++ b/internal/git/housekeeping/object_pool_test.go
@@ -6,42 +6,57 @@ import (
"github.com/stretchr/testify/require"
"gitlab.com/gitlab-org/gitaly/v15/internal/git/gittest"
"gitlab.com/gitlab-org/gitaly/v15/internal/praefect/praefectutil"
+ "gitlab.com/gitlab-org/gitaly/v15/proto/go/gitalypb"
)
-func TestIsPoolPath(t *testing.T) {
+func TestIsPoolRepository(t *testing.T) {
for _, tc := range []struct {
- desc string
- relativePath string
- isPoolPath bool
+ desc string
+ repo *gitalypb.Repository
+ isPoolPath bool
}{
{
- desc: "rails pool directory",
- relativePath: gittest.NewObjectPoolName(t),
- isPoolPath: true,
+ desc: "rails pool directory",
+ repo: &gitalypb.Repository{
+ RelativePath: gittest.NewObjectPoolName(t),
+ },
+ isPoolPath: true,
},
{
- desc: "praefect pool path",
- relativePath: praefectutil.DerivePoolPath(1),
- isPoolPath: true,
+ desc: "praefect pool path",
+ repo: &gitalypb.Repository{
+ RelativePath: praefectutil.DerivePoolPath(1),
+ },
+ isPoolPath: true,
},
{
- desc: "praefect replica path",
- relativePath: praefectutil.DeriveReplicaPath(1),
+ desc: "praefect replica path",
+ repo: &gitalypb.Repository{
+ RelativePath: praefectutil.DeriveReplicaPath(1),
+ },
},
{
- desc: "empty string",
+ desc: "missing repository",
},
{
- desc: "rails path first to subdirs dont match full hash",
- relativePath: "@pools/aa/bb/ffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff.git",
+ desc: "empty repository",
+ repo: &gitalypb.Repository{},
},
{
- desc: "normal repos dont match",
- relativePath: "@hashed/" + gittest.NewRepositoryName(t, true),
+ desc: "rails path first to subdirs dont match full hash",
+ repo: &gitalypb.Repository{
+ RelativePath: "@pools/aa/bb/ffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff.git",
+ },
+ },
+ {
+ desc: "normal repos dont match",
+ repo: &gitalypb.Repository{
+ RelativePath: "@hashed/" + gittest.NewRepositoryName(t, true),
+ },
},
} {
t.Run(tc.desc, func(t *testing.T) {
- require.Equal(t, tc.isPoolPath, IsPoolPath(tc.relativePath))
+ require.Equal(t, tc.isPoolPath, IsPoolRepository(tc.repo))
})
}
}
diff --git a/internal/git/housekeeping/objects.go b/internal/git/housekeeping/objects.go
index 084e115c1..7ce675c6e 100644
--- a/internal/git/housekeeping/objects.go
+++ b/internal/git/housekeeping/objects.go
@@ -5,6 +5,7 @@ import (
"gitlab.com/gitlab-org/gitaly/v15/internal/git"
"gitlab.com/gitlab-org/gitaly/v15/internal/git/localrepo"
+ "gitlab.com/gitlab-org/gitaly/v15/internal/git/repository"
"gitlab.com/gitlab-org/gitaly/v15/internal/git/stats"
)
@@ -47,7 +48,7 @@ func RepackObjects(ctx context.Context, repo *localrepo.Repo, cfg RepackObjectsC
// details.
git.Flag{Name: "-n"},
}, options...),
- }, git.WithConfig(GetRepackGitConfig(ctx, cfg.WriteBitmap)...)); err != nil {
+ }, git.WithConfig(GetRepackGitConfig(ctx, repo, cfg.WriteBitmap)...)); err != nil {
return err
}
@@ -61,14 +62,25 @@ func RepackObjects(ctx context.Context, repo *localrepo.Repo, cfg RepackObjectsC
}
// GetRepackGitConfig returns configuration suitable for Git commands which write new packfiles.
-func GetRepackGitConfig(ctx context.Context, bitmap bool) []git.ConfigPair {
+func GetRepackGitConfig(ctx context.Context, repo repository.GitRepo, bitmap bool) []git.ConfigPair {
config := []git.ConfigPair{
- {Key: "pack.island", Value: "r(e)fs/heads"},
- {Key: "pack.island", Value: "r(e)fs/tags"},
- {Key: "pack.islandCore", Value: "e"},
{Key: "repack.useDeltaIslands", Value: "true"},
}
+ if IsPoolRepository(repo) {
+ config = append(config,
+ git.ConfigPair{Key: "pack.island", Value: git.ObjectPoolRefNamespace + "/he(a)ds"},
+ git.ConfigPair{Key: "pack.island", Value: git.ObjectPoolRefNamespace + "/t(a)gs"},
+ git.ConfigPair{Key: "pack.islandCore", Value: "a"},
+ )
+ } else {
+ config = append(config,
+ git.ConfigPair{Key: "pack.island", Value: "r(e)fs/heads"},
+ git.ConfigPair{Key: "pack.island", Value: "r(e)fs/tags"},
+ git.ConfigPair{Key: "pack.islandCore", Value: "e"},
+ )
+ }
+
if bitmap {
config = append(config, git.ConfigPair{Key: "repack.writeBitmaps", Value: "true"})
config = append(config, git.ConfigPair{Key: "pack.writeBitmapHashCache", Value: "true"})
diff --git a/internal/git/housekeeping/objects_test.go b/internal/git/housekeeping/objects_test.go
index 0d65e60cd..f6b079198 100644
--- a/internal/git/housekeeping/objects_test.go
+++ b/internal/git/housekeeping/objects_test.go
@@ -51,4 +51,17 @@ func TestRepackObjects(t *testing.T) {
require.NoFileExists(t, filepath.Join(repoPath, "info", "refs"))
require.NoFileExists(t, filepath.Join(repoPath, "objects", "info", "packs"))
})
+
+ testRepoAndPool(t, "delta islands", func(t *testing.T, relativePath string) {
+ repoProto, repoPath := gittest.InitRepo(t, cfg, cfg.Storages[0], gittest.InitRepoOpts{
+ WithRelativePath: relativePath,
+ })
+ repo := localrepo.NewTestRepo(t, cfg, repoProto)
+
+ gittest.TestDeltaIslands(t, cfg, repoPath, IsPoolRepository(repoProto), func() error {
+ return RepackObjects(ctx, repo, RepackObjectsConfig{
+ FullRepack: true,
+ })
+ })
+ })
}
diff --git a/internal/git/housekeeping/optimize_repository.go b/internal/git/housekeeping/optimize_repository.go
index 0d79db70e..ef74f8f0d 100644
--- a/internal/git/housekeeping/optimize_repository.go
+++ b/internal/git/housekeeping/optimize_repository.go
@@ -215,25 +215,36 @@ func needsRepacking(repo *localrepo.Repo) (bool, RepackObjectsConfig, error) {
// relative though: for small repositories it's fine to do full repacks regularly, but for
// large repositories we need to be more careful. We thus use a heuristic of "repository
// largeness": we take the biggest packfile that exists, and then the maximum allowed number
- // of packfiles is `log(largestpackfile_size_in_mb) / log(1.3)`. This gives the following
- // allowed number of packfiles:
+ // of packfiles is `log(largestpackfile_size_in_mb) / log(1.3)` for normal repositories and
+ // `log(largestpackfile_size_in_mb) / log(10.0)` for pools. This gives the following allowed
+ // number of packfiles:
//
- // - No packfile: 5 packfile. This is a special case.
- // - 10MB packfile: 8 packfiles.
- // - 100MB packfile: 17 packfiles.
- // - 500MB packfile: 23 packfiles.
- // - 1GB packfile: 26 packfiles.
- // - 5GB packfile: 32 packfiles.
- // - 10GB packfile: 35 packfiles.
- // - 100GB packfile: 43 packfiles.
+ // -------------------------------------------------------------------------------------
+ // | largest packfile size | allowed packfiles for repos | allowed packfiles for pools |
+ // -------------------------------------------------------------------------------------
+ // | none or <10MB | 5 | 2 |
+ // | 10MB | 8 | 2 |
+ // | 100MB | 17 | 2 |
+ // | 500MB | 23 | 2 |
+ // | 1GB | 26 | 3 |
+ // | 5GB | 32 | 3 |
+ // | 10GB | 35 | 4 |
+ // | 100GB | 43 | 5 |
+ // -------------------------------------------------------------------------------------
//
// The goal is to have a comparatively quick ramp-up of allowed packfiles as the repository
// size grows, but then slow down such that we're effectively capped and don't end up with
- // an excessive amount of packfiles.
+ // an excessive amount of packfiles. On the other hand, pool repositories are potentially
+ // reused as basis for many forks and should thus be packed much more aggressively.
//
// This is a heuristic and thus imperfect by necessity. We may tune it as we gain experience
// with the way it behaves.
- if int64(math.Max(5, math.Log(float64(largestPackfileSize))/math.Log(1.3))) < packfileCount {
+ lowerLimit, log := 5.0, 1.3
+ if IsPoolRepository(repo) {
+ lowerLimit, log = 2.0, 10.0
+ }
+
+ if int64(math.Max(lowerLimit, math.Log(float64(largestPackfileSize))/math.Log(log))) <= packfileCount {
return true, RepackObjectsConfig{
FullRepack: true,
WriteBitmap: !hasAlternate,
@@ -359,7 +370,7 @@ func estimateLooseObjectCount(repo *localrepo.Repo, cutoffDate time.Time) (int64
func pruneIfNeeded(ctx context.Context, repo *localrepo.Repo) (bool, error) {
// Pool repositories must never prune any objects, or otherwise we may corrupt members of
// that pool if they still refer to that object.
- if IsPoolPath(repo.GetRelativePath()) {
+ if IsPoolRepository(repo) {
return false, nil
}
diff --git a/internal/git/housekeeping/optimize_repository_test.go b/internal/git/housekeeping/optimize_repository_test.go
index 473c17341..939dab9b1 100644
--- a/internal/git/housekeeping/optimize_repository_test.go
+++ b/internal/git/housekeeping/optimize_repository_test.go
@@ -4,7 +4,6 @@ import (
"bytes"
"context"
"fmt"
- "io"
"os"
"path/filepath"
"strings"
@@ -26,15 +25,6 @@ import (
"gitlab.com/gitlab-org/gitaly/v15/proto/go/gitalypb"
)
-type infiniteReader struct{}
-
-func (r infiniteReader) Read(b []byte) (int, error) {
- for i := range b {
- b[i] = '\000'
- }
- return len(b), nil
-}
-
func TestNeedsRepacking(t *testing.T) {
t.Parallel()
@@ -42,22 +32,26 @@ func TestNeedsRepacking(t *testing.T) {
for _, tc := range []struct {
desc string
- setup func(t *testing.T) *gitalypb.Repository
+ setup func(t *testing.T, relativePath string) *gitalypb.Repository
expectedErr error
expectedNeeded bool
expectedConfig RepackObjectsConfig
}{
{
desc: "empty repo does nothing",
- setup: func(t *testing.T) *gitalypb.Repository {
- repoProto, _ := gittest.InitRepo(t, cfg, cfg.Storages[0])
+ setup: func(t *testing.T, relativePath string) *gitalypb.Repository {
+ repoProto, _ := gittest.InitRepo(t, cfg, cfg.Storages[0], gittest.InitRepoOpts{
+ WithRelativePath: relativePath,
+ })
return repoProto
},
},
{
desc: "missing bitmap",
- setup: func(t *testing.T) *gitalypb.Repository {
- repoProto, _ := gittest.CloneRepo(t, cfg, cfg.Storages[0])
+ setup: func(t *testing.T, relativePath string) *gitalypb.Repository {
+ repoProto, _ := gittest.CloneRepo(t, cfg, cfg.Storages[0], gittest.CloneRepoOpts{
+ RelativePath: relativePath,
+ })
return repoProto
},
expectedNeeded: true,
@@ -68,8 +62,10 @@ func TestNeedsRepacking(t *testing.T) {
},
{
desc: "missing bitmap with alternate",
- setup: func(t *testing.T) *gitalypb.Repository {
- repoProto, repoPath := gittest.CloneRepo(t, cfg, cfg.Storages[0])
+ setup: func(t *testing.T, relativePath string) *gitalypb.Repository {
+ repoProto, repoPath := gittest.CloneRepo(t, cfg, cfg.Storages[0], gittest.CloneRepoOpts{
+ RelativePath: relativePath,
+ })
// Create the alternates file. If it exists, then we shouldn't try
// to generate a bitmap.
@@ -85,8 +81,10 @@ func TestNeedsRepacking(t *testing.T) {
},
{
desc: "missing commit-graph",
- setup: func(t *testing.T) *gitalypb.Repository {
- repoProto, repoPath := gittest.CloneRepo(t, cfg, cfg.Storages[0])
+ setup: func(t *testing.T, relativePath string) *gitalypb.Repository {
+ repoProto, repoPath := gittest.CloneRepo(t, cfg, cfg.Storages[0], gittest.CloneRepoOpts{
+ RelativePath: relativePath,
+ })
gittest.Exec(t, cfg, "-C", repoPath, "repack", "-Ad", "--write-bitmap-index")
@@ -100,8 +98,10 @@ func TestNeedsRepacking(t *testing.T) {
},
{
desc: "commit-graph without bloom filters",
- setup: func(t *testing.T) *gitalypb.Repository {
- repoProto, repoPath := gittest.CloneRepo(t, cfg, cfg.Storages[0])
+ setup: func(t *testing.T, relativePath string) *gitalypb.Repository {
+ repoProto, repoPath := gittest.CloneRepo(t, cfg, cfg.Storages[0], gittest.CloneRepoOpts{
+ RelativePath: relativePath,
+ })
gittest.Exec(t, cfg, "-C", repoPath, "repack", "-Ad", "--write-bitmap-index")
gittest.Exec(t, cfg, "-C", repoPath, "commit-graph", "write")
@@ -116,8 +116,10 @@ func TestNeedsRepacking(t *testing.T) {
},
{
desc: "no repack needed",
- setup: func(t *testing.T) *gitalypb.Repository {
- repoProto, repoPath := gittest.CloneRepo(t, cfg, cfg.Storages[0])
+ setup: func(t *testing.T, relativePath string) *gitalypb.Repository {
+ repoProto, repoPath := gittest.CloneRepo(t, cfg, cfg.Storages[0], gittest.CloneRepoOpts{
+ RelativePath: relativePath,
+ })
gittest.Exec(t, cfg, "-C", repoPath, "repack", "-Ad", "--write-bitmap-index")
gittest.Exec(t, cfg, "-C", repoPath, "commit-graph", "write", "--changed-paths", "--split")
@@ -131,54 +133,66 @@ func TestNeedsRepacking(t *testing.T) {
t.Run(tc.desc, func(t *testing.T) {
t.Parallel()
- repoProto := tc.setup(t)
- repo := localrepo.NewTestRepo(t, cfg, repoProto)
+ testRepoAndPool(t, tc.desc, func(t *testing.T, relativePath string) {
+ repoProto := tc.setup(t, relativePath)
+ repo := localrepo.NewTestRepo(t, cfg, repoProto)
- repackNeeded, repackCfg, err := needsRepacking(repo)
- require.Equal(t, tc.expectedErr, err)
- require.Equal(t, tc.expectedNeeded, repackNeeded)
- require.Equal(t, tc.expectedConfig, repackCfg)
+ repackNeeded, repackCfg, err := needsRepacking(repo)
+ require.Equal(t, tc.expectedErr, err)
+ require.Equal(t, tc.expectedNeeded, repackNeeded)
+ require.Equal(t, tc.expectedConfig, repackCfg)
+ })
})
}
const megaByte = 1024 * 1024
for _, tc := range []struct {
- packfileSize int64
- requiredPackfiles int
+ packfileSize int64
+ requiredPackfiles int
+ requiredPackfilesForPool int
}{
{
- packfileSize: 1,
- requiredPackfiles: 5,
+ packfileSize: 1,
+ requiredPackfiles: 5,
+ requiredPackfilesForPool: 2,
},
{
- packfileSize: 5 * megaByte,
- requiredPackfiles: 6,
+ packfileSize: 5 * megaByte,
+ requiredPackfiles: 6,
+ requiredPackfilesForPool: 2,
},
{
- packfileSize: 10 * megaByte,
- requiredPackfiles: 8,
+ packfileSize: 10 * megaByte,
+ requiredPackfiles: 8,
+ requiredPackfilesForPool: 2,
},
{
- packfileSize: 50 * megaByte,
- requiredPackfiles: 14,
+ packfileSize: 50 * megaByte,
+ requiredPackfiles: 14,
+ requiredPackfilesForPool: 2,
},
{
- packfileSize: 100 * megaByte,
- requiredPackfiles: 17,
+ packfileSize: 100 * megaByte,
+ requiredPackfiles: 17,
+ requiredPackfilesForPool: 2,
},
{
- packfileSize: 500 * megaByte,
- requiredPackfiles: 23,
+ packfileSize: 500 * megaByte,
+ requiredPackfiles: 23,
+ requiredPackfilesForPool: 2,
},
{
- packfileSize: 1000 * megaByte,
- requiredPackfiles: 26,
+ packfileSize: 1001 * megaByte,
+ requiredPackfiles: 26,
+ requiredPackfilesForPool: 3,
},
// Let's not go any further than this, we're thrashing the temporary directory.
} {
- t.Run(fmt.Sprintf("packfile with %d bytes", tc.packfileSize), func(t *testing.T) {
- repoProto, repoPath := gittest.InitRepo(t, cfg, cfg.Storages[0])
+ testRepoAndPool(t, fmt.Sprintf("packfile with %d bytes", tc.packfileSize), func(t *testing.T, relativePath string) {
+ repoProto, repoPath := gittest.InitRepo(t, cfg, cfg.Storages[0], gittest.InitRepoOpts{
+ WithRelativePath: relativePath,
+ })
repo := localrepo.NewTestRepo(t, cfg, repoProto)
packDir := filepath.Join(repoPath, "objects", "pack")
@@ -193,16 +207,19 @@ func TestNeedsRepacking(t *testing.T) {
// We first create a single big packfile which is used to determine the
// boundary of when we repack.
- bigPackfile, err := os.OpenFile(filepath.Join(packDir, "big.pack"), os.O_CREATE|os.O_EXCL|os.O_WRONLY, 0o644)
- require.NoError(t, err)
- defer testhelper.MustClose(t, bigPackfile)
- _, err = io.Copy(bigPackfile, io.LimitReader(infiniteReader{}, tc.packfileSize))
- require.NoError(t, err)
+ bigPackPath := filepath.Join(packDir, "big.pack")
+ require.NoError(t, os.WriteFile(bigPackPath, nil, 0o644))
+ require.NoError(t, os.Truncate(bigPackPath, tc.packfileSize))
+
+ requiredPackfiles := tc.requiredPackfiles
+ if IsPoolRepository(repoProto) {
+ requiredPackfiles = tc.requiredPackfilesForPool
+ }
// And then we create one less packfile than we need to hit the boundary.
// This is done to assert that we indeed don't repack before hitting the
// boundary.
- for i := 0; i < tc.requiredPackfiles-1; i++ {
+ for i := 0; i < requiredPackfiles-2; i++ {
additionalPackfile, err := os.Create(filepath.Join(packDir, fmt.Sprintf("%d.pack", i)))
require.NoError(t, err)
testhelper.MustClose(t, additionalPackfile)
@@ -284,8 +301,10 @@ func TestNeedsRepacking(t *testing.T) {
expectedRepack: true,
},
} {
- t.Run(tc.desc, func(t *testing.T) {
- repoProto, repoPath := gittest.InitRepo(t, cfg, cfg.Storages[0])
+ testRepoAndPool(t, tc.desc, func(t *testing.T, relativePath string) {
+ repoProto, repoPath := gittest.InitRepo(t, cfg, cfg.Storages[0], gittest.InitRepoOpts{
+ WithRelativePath: relativePath,
+ })
repo := localrepo.NewTestRepo(t, cfg, repoProto)
// Emulate the existence of a bitmap and a commit-graph with bloom filters.
@@ -360,8 +379,10 @@ func TestPackRefsIfNeeded(t *testing.T) {
requiredRefs: 99,
},
} {
- t.Run(fmt.Sprintf("packed-refs with %d bytes", tc.packedRefsSize), func(t *testing.T) {
- repoProto, repoPath := gittest.InitRepo(t, cfg, cfg.Storages[0])
+ testRepoAndPool(t, fmt.Sprintf("packed-refs with %d bytes", tc.packedRefsSize), func(t *testing.T, relativePath string) {
+ repoProto, repoPath := gittest.InitRepo(t, cfg, cfg.Storages[0], gittest.InitRepoOpts{
+ WithRelativePath: relativePath,
+ })
repo := localrepo.NewTestRepo(t, cfg, repoProto)
// Write an empty commit such that we can create valid refs.
@@ -495,15 +516,18 @@ func TestOptimizeRepository(t *testing.T) {
txManager := transaction.NewManager(cfg, backchannel.NewRegistry())
for _, tc := range []struct {
- desc string
- setup func(t *testing.T) *gitalypb.Repository
- expectedErr error
- expectedMetrics string
+ desc string
+ setup func(t *testing.T, relativePath string) *gitalypb.Repository
+ expectedErr error
+ expectedMetrics string
+ expectedMetricsForPool string
}{
{
desc: "empty repository does nothing",
- setup: func(t *testing.T) *gitalypb.Repository {
- repo, _ := gittest.InitRepo(t, cfg, cfg.Storages[0])
+ setup: func(t *testing.T, relativePath string) *gitalypb.Repository {
+ repo, _ := gittest.InitRepo(t, cfg, cfg.Storages[0], gittest.InitRepoOpts{
+ WithRelativePath: relativePath,
+ })
return repo
},
expectedMetrics: `# HELP gitaly_housekeeping_tasks_total Total number of housekeeping tasks performed in the repository
@@ -513,8 +537,10 @@ gitaly_housekeeping_tasks_total{housekeeping_task="total", status="success"} 1
},
{
desc: "repository without bitmap repacks objects",
- setup: func(t *testing.T) *gitalypb.Repository {
- repo, _ := gittest.CloneRepo(t, cfg, cfg.Storages[0])
+ setup: func(t *testing.T, relativePath string) *gitalypb.Repository {
+ repo, _ := gittest.CloneRepo(t, cfg, cfg.Storages[0], gittest.CloneRepoOpts{
+ RelativePath: relativePath,
+ })
return repo
},
expectedMetrics: `# HELP gitaly_housekeeping_tasks_total Total number of housekeeping tasks performed in the repository
@@ -526,13 +552,40 @@ gitaly_housekeeping_tasks_total{housekeeping_task="total", status="success"} 1
},
{
desc: "repository without commit-graph repacks objects",
- setup: func(t *testing.T) *gitalypb.Repository {
- repo, repoPath := gittest.CloneRepo(t, cfg, cfg.Storages[0])
+ setup: func(t *testing.T, relativePath string) *gitalypb.Repository {
+ repo, repoPath := gittest.CloneRepo(t, cfg, cfg.Storages[0], gittest.CloneRepoOpts{
+ RelativePath: relativePath,
+ })
+ gittest.Exec(t, cfg, "-C", repoPath, "repack", "-A", "-d", "--write-bitmap-index")
+ return repo
+ },
+ expectedMetrics: `# HELP gitaly_housekeeping_tasks_total Total number of housekeeping tasks performed in the repository
+# TYPE gitaly_housekeeping_tasks_total counter
+gitaly_housekeeping_tasks_total{housekeeping_task="packed_objects_full", status="success"} 1
+gitaly_housekeeping_tasks_total{housekeeping_task="written_bitmap", status="success"} 1
+gitaly_housekeeping_tasks_total{housekeeping_task="total", status="success"} 1
+`,
+ },
+ {
+ desc: "repository with multiple packfiles packs only for object pool",
+ setup: func(t *testing.T, relativePath string) *gitalypb.Repository {
+ repo, repoPath := gittest.CloneRepo(t, cfg, cfg.Storages[0], gittest.CloneRepoOpts{
+ RelativePath: relativePath,
+ })
+
+ // Note: git-repack(1) without "-d" will _not_ delete the old
+ // packfile and thus end up with two packfiles.
gittest.Exec(t, cfg, "-C", repoPath, "repack", "-A", "--write-bitmap-index")
+ gittest.Exec(t, cfg, "-C", repoPath, "commit-graph", "write", "--split", "--changed-paths")
+
return repo
},
expectedMetrics: `# HELP gitaly_housekeeping_tasks_total Total number of housekeeping tasks performed in the repository
# TYPE gitaly_housekeeping_tasks_total counter
+gitaly_housekeeping_tasks_total{housekeeping_task="total", status="success"} 1
+`,
+ expectedMetricsForPool: `# HELP gitaly_housekeeping_tasks_total Total number of housekeeping tasks performed in the repository
+# TYPE gitaly_housekeeping_tasks_total counter
gitaly_housekeeping_tasks_total{housekeeping_task="packed_objects_full", status="success"} 1
gitaly_housekeeping_tasks_total{housekeeping_task="written_bitmap", status="success"} 1
gitaly_housekeeping_tasks_total{housekeeping_task="total", status="success"} 1
@@ -540,9 +593,11 @@ gitaly_housekeeping_tasks_total{housekeeping_task="total", status="success"} 1
},
{
desc: "well-packed repository does not optimize",
- setup: func(t *testing.T) *gitalypb.Repository {
- repo, repoPath := gittest.CloneRepo(t, cfg, cfg.Storages[0])
- gittest.Exec(t, cfg, "-C", repoPath, "repack", "-A", "--write-bitmap-index")
+ setup: func(t *testing.T, relativePath string) *gitalypb.Repository {
+ repo, repoPath := gittest.CloneRepo(t, cfg, cfg.Storages[0], gittest.CloneRepoOpts{
+ RelativePath: relativePath,
+ })
+ gittest.Exec(t, cfg, "-C", repoPath, "repack", "-A", "-d", "--write-bitmap-index")
gittest.Exec(t, cfg, "-C", repoPath, "commit-graph", "write", "--split", "--changed-paths")
return repo
},
@@ -553,9 +608,11 @@ gitaly_housekeeping_tasks_total{housekeeping_task="total", status="success"} 1
},
{
desc: "recent loose objects don't get pruned",
- setup: func(t *testing.T) *gitalypb.Repository {
- repo, repoPath := gittest.CloneRepo(t, cfg, cfg.Storages[0])
- gittest.Exec(t, cfg, "-C", repoPath, "repack", "-A", "--write-bitmap-index")
+ setup: func(t *testing.T, relativePath string) *gitalypb.Repository {
+ repo, repoPath := gittest.CloneRepo(t, cfg, cfg.Storages[0], gittest.CloneRepoOpts{
+ RelativePath: relativePath,
+ })
+ gittest.Exec(t, cfg, "-C", repoPath, "repack", "-A", "-d", "--write-bitmap-index")
gittest.Exec(t, cfg, "-C", repoPath, "commit-graph", "write", "--split", "--changed-paths")
// The repack won't repack the following objects because they're
@@ -583,9 +640,11 @@ gitaly_housekeeping_tasks_total{housekeeping_task="total", status="success"} 1
},
{
desc: "old loose objects get pruned",
- setup: func(t *testing.T) *gitalypb.Repository {
- repo, repoPath := gittest.CloneRepo(t, cfg, cfg.Storages[0])
- gittest.Exec(t, cfg, "-C", repoPath, "repack", "-A", "--write-bitmap-index")
+ setup: func(t *testing.T, relativePath string) *gitalypb.Repository {
+ repo, repoPath := gittest.CloneRepo(t, cfg, cfg.Storages[0], gittest.CloneRepoOpts{
+ RelativePath: relativePath,
+ })
+ gittest.Exec(t, cfg, "-C", repoPath, "repack", "-A", "-d", "--write-bitmap-index")
gittest.Exec(t, cfg, "-C", repoPath, "commit-graph", "write", "--split", "--changed-paths")
// The repack won't repack the following objects because they're
@@ -608,11 +667,19 @@ gitaly_housekeeping_tasks_total{housekeeping_task="packed_objects_incremental",
gitaly_housekeeping_tasks_total{housekeeping_task="pruned_objects",status="success"} 1
gitaly_housekeeping_tasks_total{housekeeping_task="total", status="success"} 1
`,
+ // Object pools never prune objects.
+ expectedMetricsForPool: `# HELP gitaly_housekeeping_tasks_total Total number of housekeeping tasks performed in the repository
+# TYPE gitaly_housekeeping_tasks_total counter
+gitaly_housekeeping_tasks_total{housekeeping_task="packed_objects_incremental", status="success"} 1
+gitaly_housekeeping_tasks_total{housekeeping_task="total", status="success"} 1
+`,
},
{
desc: "loose refs get packed",
- setup: func(t *testing.T) *gitalypb.Repository {
- repo, repoPath := gittest.InitRepo(t, cfg, cfg.Storages[0])
+ setup: func(t *testing.T, relativePath string) *gitalypb.Repository {
+ repo, repoPath := gittest.InitRepo(t, cfg, cfg.Storages[0], gittest.InitRepoOpts{
+ WithRelativePath: relativePath,
+ })
for i := 0; i < 16; i++ {
gittest.WriteCommit(t, cfg, repoPath, gittest.WithParents(), gittest.WithBranch(fmt.Sprintf("branch-%d", i)))
@@ -630,10 +697,10 @@ gitaly_housekeeping_tasks_total{housekeeping_task="total", status="success"} 1
`,
},
} {
- t.Run(tc.desc, func(t *testing.T) {
+ testRepoAndPool(t, tc.desc, func(t *testing.T, relativePath string) {
ctx := testhelper.Context(t)
- repoProto := tc.setup(t)
+ repoProto := tc.setup(t, relativePath)
repo := localrepo.NewTestRepo(t, cfg, repoProto)
manager := NewManager(cfg.Prometheus, txManager)
@@ -641,9 +708,14 @@ gitaly_housekeeping_tasks_total{housekeeping_task="total", status="success"} 1
err := manager.OptimizeRepository(ctx, repo)
require.Equal(t, tc.expectedErr, err)
- assert.NoError(t, testutil.CollectAndCompare(
+ expectedMetrics := tc.expectedMetrics
+ if IsPoolRepository(repoProto) && tc.expectedMetricsForPool != "" {
+ expectedMetrics = tc.expectedMetricsForPool
+ }
+
+ require.NoError(t, testutil.CollectAndCompare(
manager.tasksTotal,
- bytes.NewBufferString(tc.expectedMetrics),
+ bytes.NewBufferString(expectedMetrics),
"gitaly_housekeeping_tasks_total",
))
})
@@ -776,8 +848,10 @@ func TestPruneIfNeeded(t *testing.T) {
ctx := testhelper.Context(t)
cfg := testcfg.Build(t)
- t.Run("empty repo does not prune", func(t *testing.T) {
- repoProto, _ := gittest.InitRepo(t, cfg, cfg.Storages[0])
+ testRepoAndPool(t, "empty repo does not prune", func(t *testing.T, relativePath string) {
+ repoProto, _ := gittest.InitRepo(t, cfg, cfg.Storages[0], gittest.InitRepoOpts{
+ WithRelativePath: relativePath,
+ })
repo := localrepo.NewTestRepo(t, cfg, repoProto)
didPrune, err := pruneIfNeeded(ctx, repo)
@@ -785,8 +859,10 @@ func TestPruneIfNeeded(t *testing.T) {
require.False(t, didPrune)
})
- t.Run("repo with single object does not prune", func(t *testing.T) {
- repoProto, repoPath := gittest.InitRepo(t, cfg, cfg.Storages[0])
+ testRepoAndPool(t, "repo with single object does not prune", func(t *testing.T, relativePath string) {
+ repoProto, repoPath := gittest.InitRepo(t, cfg, cfg.Storages[0], gittest.InitRepoOpts{
+ WithRelativePath: relativePath,
+ })
repo := localrepo.NewTestRepo(t, cfg, repoProto)
gittest.WriteBlob(t, cfg, repoPath, []byte("something"))
@@ -796,8 +872,10 @@ func TestPruneIfNeeded(t *testing.T) {
require.False(t, didPrune)
})
- t.Run("repo with single 17-prefixed objects does not prune", func(t *testing.T) {
- repoProto, repoPath := gittest.InitRepo(t, cfg, cfg.Storages[0])
+ testRepoAndPool(t, "repo with single 17-prefixed objects does not prune", func(t *testing.T, relativePath string) {
+ repoProto, repoPath := gittest.InitRepo(t, cfg, cfg.Storages[0], gittest.InitRepoOpts{
+ WithRelativePath: relativePath,
+ })
repo := localrepo.NewTestRepo(t, cfg, repoProto)
blobID := gittest.WriteBlob(t, cfg, repoPath, []byte("32"))
@@ -808,8 +886,10 @@ func TestPruneIfNeeded(t *testing.T) {
require.False(t, didPrune)
})
- t.Run("repo with four 17-prefixed objects does not prune", func(t *testing.T) {
- repoProto, repoPath := gittest.InitRepo(t, cfg, cfg.Storages[0])
+ testRepoAndPool(t, "repo with four 17-prefixed objects does not prune", func(t *testing.T, relativePath string) {
+ repoProto, repoPath := gittest.InitRepo(t, cfg, cfg.Storages[0], gittest.InitRepoOpts{
+ WithRelativePath: relativePath,
+ })
repo := localrepo.NewTestRepo(t, cfg, repoProto)
for _, contents := range []string{"32", "119", "334", "782"} {
@@ -822,8 +902,10 @@ func TestPruneIfNeeded(t *testing.T) {
require.False(t, didPrune)
})
- t.Run("repo with five 17-prefixed objects does prune after grace period", func(t *testing.T) {
- repoProto, repoPath := gittest.InitRepo(t, cfg, cfg.Storages[0])
+ testRepoAndPool(t, "repo with five 17-prefixed objects does prune after grace period", func(t *testing.T, relativePath string) {
+ repoProto, repoPath := gittest.InitRepo(t, cfg, cfg.Storages[0], gittest.InitRepoOpts{
+ WithRelativePath: relativePath,
+ })
repo := localrepo.NewTestRepo(t, cfg, repoProto)
objectPath := func(oid git.ObjectID) string {
@@ -864,15 +946,24 @@ func TestPruneIfNeeded(t *testing.T) {
// and thus we would still want to prune here.
didPrune, err = pruneIfNeeded(ctx, repo)
require.NoError(t, err)
- require.True(t, didPrune)
- // But this time the objects shouldn't exist anymore because they were older than
- // the grace period.
- for _, blob := range blobs {
- require.NoFileExists(t, objectPath(blob))
- }
+ if IsPoolRepository(repoProto) {
+ // Object pools mustn't ever prune objects.
+ require.False(t, didPrune)
+ for _, blob := range append(blobs, recentBlob) {
+ require.FileExists(t, objectPath(blob))
+ }
+ } else {
+ require.True(t, didPrune)
- // The recent blob should continue to exist though.
- require.FileExists(t, objectPath(recentBlob))
+ // But this time the objects shouldn't exist anymore because they were older than
+ // the grace period.
+ for _, blob := range blobs {
+ require.NoFileExists(t, objectPath(blob))
+ }
+
+ // The recent blob should continue to exist though.
+ require.FileExists(t, objectPath(recentBlob))
+ }
})
}
diff --git a/internal/git/housekeeping/testhelper_test.go b/internal/git/housekeeping/testhelper_test.go
index 3de1870cd..ccb0e1a28 100644
--- a/internal/git/housekeeping/testhelper_test.go
+++ b/internal/git/housekeeping/testhelper_test.go
@@ -3,9 +3,23 @@ package housekeeping
import (
"testing"
+ "gitlab.com/gitlab-org/gitaly/v15/internal/git/gittest"
"gitlab.com/gitlab-org/gitaly/v15/internal/testhelper"
)
func TestMain(m *testing.M) {
testhelper.Run(m)
}
+
+func testRepoAndPool(t *testing.T, desc string, testFunc func(t *testing.T, relativePath string)) {
+ t.Helper()
+ t.Run(desc, func(t *testing.T) {
+ t.Run("normal repository", func(t *testing.T) {
+ testFunc(t, gittest.NewRepositoryName(t, true))
+ })
+
+ t.Run("object pool", func(t *testing.T) {
+ testFunc(t, gittest.NewObjectPoolName(t))
+ })
+ })
+}
diff --git a/internal/git/objectpool/fetch.go b/internal/git/objectpool/fetch.go
index 549522809..2639cc48e 100644
--- a/internal/git/objectpool/fetch.go
+++ b/internal/git/objectpool/fetch.go
@@ -18,31 +18,29 @@ import (
"gitlab.com/gitlab-org/gitaly/v15/internal/git/localrepo"
"gitlab.com/gitlab-org/gitaly/v15/internal/git/repository"
"gitlab.com/gitlab-org/gitaly/v15/internal/git/updateref"
- "gitlab.com/gitlab-org/gitaly/v15/internal/helper"
+ "gitlab.com/gitlab-org/gitaly/v15/internal/metadata/featureflag"
)
-const sourceRefNamespace = "refs/remotes/origin"
-
// FetchFromOrigin initializes the pool and fetches the objects from its origin repository
func (o *ObjectPool) FetchFromOrigin(ctx context.Context, origin *localrepo.Repo) error {
if err := o.Init(ctx); err != nil {
- return err
+ return fmt.Errorf("initializing object pool: %w", err)
}
originPath, err := origin.Path()
if err != nil {
- return err
+ return fmt.Errorf("computing origin repo's path: %w", err)
}
if err := o.housekeepingManager.CleanStaleData(ctx, o.Repo); err != nil {
- return err
+ return fmt.Errorf("cleaning stale data: %w", err)
}
if err := o.logStats(ctx, "before fetch"); err != nil {
- return err
+ return fmt.Errorf("computing stats before fetch: %w", err)
}
- refSpec := fmt.Sprintf("+refs/*:%s/*", sourceRefNamespace)
+ refSpec := fmt.Sprintf("+refs/*:%s/*", git.ObjectPoolRefNamespace)
var stderr bytes.Buffer
if err := o.Repo.ExecAndWait(ctx,
git.SubCmd{
@@ -64,26 +62,36 @@ func (o *ObjectPool) FetchFromOrigin(ctx context.Context, origin *localrepo.Repo
git.WithRefTxHook(o.Repo),
git.WithStderr(&stderr),
); err != nil {
- return helper.ErrInternalf("fetch into object pool: %w, stderr: %q", err,
+ return fmt.Errorf("fetch into object pool: %w, stderr: %q", err,
stderr.String())
}
if err := o.rescueDanglingObjects(ctx); err != nil {
- return err
+ return fmt.Errorf("rescuing dangling objects: %w", err)
}
if err := o.logStats(ctx, "after fetch"); err != nil {
- return err
+ return fmt.Errorf("computing stats after fetch: %w", err)
}
- if err := o.Repo.ExecAndWait(ctx, git.SubCmd{
- Name: "pack-refs",
- Flags: []git.Option{git.Flag{Name: "--all"}},
- }); err != nil {
- return err
+ if featureflag.FetchIntoObjectPoolOptimizeRepository.IsEnabled(ctx) {
+ if err := o.housekeepingManager.OptimizeRepository(ctx, o.Repo); err != nil {
+ return fmt.Errorf("optimizing pool repo: %w", err)
+ }
+ } else {
+ if err := o.Repo.ExecAndWait(ctx, git.SubCmd{
+ Name: "pack-refs",
+ Flags: []git.Option{git.Flag{Name: "--all"}},
+ }); err != nil {
+ return fmt.Errorf("packing pool refs: %w", err)
+ }
+
+ if err := o.repackPool(ctx, o); err != nil {
+ return fmt.Errorf("repacking pool: %w", err)
+ }
}
- return o.repackPool(ctx, o)
+ return nil
}
const danglingObjectNamespace = "refs/dangling/"
@@ -140,8 +148,8 @@ func (o *ObjectPool) rescueDanglingObjects(ctx context.Context) error {
func (o *ObjectPool) repackPool(ctx context.Context, pool repository.GitRepo) error {
config := []git.ConfigPair{
- {Key: "pack.island", Value: sourceRefNamespace + "/he(a)ds"},
- {Key: "pack.island", Value: sourceRefNamespace + "/t(a)gs"},
+ {Key: "pack.island", Value: git.ObjectPoolRefNamespace + "/he(a)ds"},
+ {Key: "pack.island", Value: git.ObjectPoolRefNamespace + "/t(a)gs"},
{Key: "pack.islandCore", Value: "a"},
{Key: "pack.writeBitmapHashCache", Value: "true"},
}
diff --git a/internal/git/objectpool/fetch_test.go b/internal/git/objectpool/fetch_test.go
index f45abce77..92f741bc5 100644
--- a/internal/git/objectpool/fetch_test.go
+++ b/internal/git/objectpool/fetch_test.go
@@ -1,9 +1,11 @@
package objectpool
import (
+ "context"
"fmt"
"os"
"path/filepath"
+ "strconv"
"strings"
"testing"
@@ -13,11 +15,17 @@ import (
"gitlab.com/gitlab-org/gitaly/v15/internal/git/localrepo"
"gitlab.com/gitlab-org/gitaly/v15/internal/gitaly/config"
"gitlab.com/gitlab-org/gitaly/v15/internal/helper/text"
+ "gitlab.com/gitlab-org/gitaly/v15/internal/metadata/featureflag"
"gitlab.com/gitlab-org/gitaly/v15/internal/testhelper"
)
-func TestFetchFromOriginDangling(t *testing.T) {
- ctx := testhelper.Context(t)
+func TestFetchFromOrigin_dangling(t *testing.T) {
+ t.Parallel()
+ testhelper.NewFeatureSets(featureflag.FetchIntoObjectPoolOptimizeRepository).Run(t, testFetchFromOriginDangling)
+}
+
+func testFetchFromOriginDangling(t *testing.T, ctx context.Context) {
+ t.Parallel()
cfg, pool, repoProto := setupObjectPool(t, ctx)
repo := localrepo.NewTestRepo(t, cfg, repoProto)
@@ -86,8 +94,13 @@ func TestFetchFromOriginDangling(t *testing.T) {
require.NoFileExists(t, filepath.Join(pool.FullPath(), "objects", "info", "packs"))
}
-func TestFetchFromOriginFsck(t *testing.T) {
- ctx := testhelper.Context(t)
+func TestFetchFromOrigin_fsck(t *testing.T) {
+ t.Parallel()
+ testhelper.NewFeatureSets(featureflag.FetchIntoObjectPoolOptimizeRepository).Run(t, testFetchFromOriginFsck)
+}
+
+func testFetchFromOriginFsck(t *testing.T, ctx context.Context) {
+ t.Parallel()
cfg, pool, repoProto := setupObjectPool(t, ctx)
repo := localrepo.NewTestRepo(t, cfg, repoProto)
@@ -110,31 +123,44 @@ func TestFetchFromOriginFsck(t *testing.T) {
require.Contains(t, err.Error(), "duplicateEntries: contains duplicate file entries")
}
-func TestFetchFromOriginDeltaIslands(t *testing.T) {
- ctx := testhelper.Context(t)
+func TestFetchFromOrigin_deltaIslands(t *testing.T) {
+ t.Parallel()
+ testhelper.NewFeatureSets(featureflag.FetchIntoObjectPoolOptimizeRepository).Run(t, testFetchFromOriginDeltaIslands)
+}
+
+func testFetchFromOriginDeltaIslands(t *testing.T, ctx context.Context) {
+ t.Parallel()
cfg, pool, repoProto := setupObjectPool(t, ctx)
+
repo := localrepo.NewTestRepo(t, cfg, repoProto)
- repoPath := filepath.Join(cfg.Storages[0].Path, repo.GetRelativePath())
+ repoPath, err := repo.Path()
+ require.NoError(t, err)
require.NoError(t, pool.FetchFromOrigin(ctx, repo), "seed pool")
require.NoError(t, pool.Link(ctx, repo))
- gittest.TestDeltaIslands(t, cfg, repoPath, func() error {
- // This should create a new packfile with good delta chains in the pool
- if err := pool.FetchFromOrigin(ctx, repo); err != nil {
- return err
- }
-
- // Make sure the old packfile, with bad delta chains, is deleted from the source repo
- gittest.Exec(t, cfg, "-C", repoPath, "repack", "-ald")
-
- return nil
+ gittest.TestDeltaIslands(t, cfg, pool.FullPath(), true, func() error {
+ // The first fetch has already fetched all objects into the pool repository, so
+ // there is nothing new to fetch anymore. Consequentially, FetchFromOrigin doesn't
+ // alter the object database and thus OptimizeRepository would notice that nothing
+ // needs to be optimized.
+ //
+ // We thus write a new commit into the pool member's repository so that we end up
+ // with two packfiles after the fetch.
+ gittest.WriteCommit(t, cfg, repoPath, gittest.WithBranch("changed-ref"))
+
+ return pool.FetchFromOrigin(ctx, repo)
})
}
-func TestFetchFromOriginBitmapHashCache(t *testing.T) {
- ctx := testhelper.Context(t)
+func TestFetchFromOrigin_bitmapHashCache(t *testing.T) {
+ t.Parallel()
+ testhelper.NewFeatureSets(featureflag.FetchIntoObjectPoolOptimizeRepository).Run(t, testFetchFromOriginBitmapHashCache)
+}
+
+func testFetchFromOriginBitmapHashCache(t *testing.T, ctx context.Context) {
+ t.Parallel()
cfg, pool, repoProto := setupObjectPool(t, ctx)
repo := localrepo.NewTestRepo(t, cfg, repoProto)
@@ -158,8 +184,13 @@ func TestFetchFromOriginBitmapHashCache(t *testing.T) {
gittest.TestBitmapHasHashcache(t, bitmap)
}
-func TestFetchFromOriginRefUpdates(t *testing.T) {
- ctx := testhelper.Context(t)
+func TestFetchFromOrigin_refUpdates(t *testing.T) {
+ t.Parallel()
+ testhelper.NewFeatureSets(featureflag.FetchIntoObjectPoolOptimizeRepository).Run(t, testFetchFromOriginRefUpdates)
+}
+
+func testFetchFromOriginRefUpdates(t *testing.T, ctx context.Context) {
+ t.Parallel()
cfg, pool, repoProto := setupObjectPool(t, ctx)
repo := localrepo.NewTestRepo(t, cfg, repoProto)
@@ -184,11 +215,19 @@ func TestFetchFromOriginRefUpdates(t *testing.T) {
"tags/v1.1.0": "646ece5cfed840eca0a4feb21bcd6a81bb19bda3",
}
- for ref, newOid := range newRefs {
- require.NotEqual(t, newOid, oldRefs[ref], "sanity check of new refs")
+ // Create a bunch of additional references. This is to trigger OptimizeRepository to indeed
+ // repack the loose references as we expect it to in this test. It's debatable whether we
+ // should test this at all here given that this is business of the housekeeping package. But
+ // it's easy enough to do, so it doesn't hurt.
+ for i := 0; i < 32; i++ {
+ newRefs[fmt.Sprintf("heads/branch-%d", i)] = gittest.WriteCommit(t, cfg, repoPath,
+ gittest.WithParents(),
+ gittest.WithMessage(strconv.Itoa(i)),
+ ).String()
}
for ref, oid := range newRefs {
+ require.NotEqual(t, oid, oldRefs[ref], "sanity check of new refs")
gittest.Exec(t, cfg, "-C", repoPath, "update-ref", "refs/"+ref, oid)
require.Equal(t, oid, resolveRef(t, cfg, repoPath, "refs/"+ref), "look up %q in source after update", ref)
}
@@ -204,7 +243,12 @@ func TestFetchFromOriginRefUpdates(t *testing.T) {
}
func TestFetchFromOrigin_refs(t *testing.T) {
- ctx := testhelper.Context(t)
+ t.Parallel()
+ testhelper.NewFeatureSets(featureflag.FetchIntoObjectPoolOptimizeRepository).Run(t, testFetchFromOriginRefs)
+}
+
+func testFetchFromOriginRefs(t *testing.T, ctx context.Context) {
+ t.Parallel()
cfg, pool, _ := setupObjectPool(t, ctx)
poolPath := pool.FullPath()
diff --git a/internal/git/objectpool/pool.go b/internal/git/objectpool/pool.go
index 1a454e67e..d1676c2af 100644
--- a/internal/git/objectpool/pool.go
+++ b/internal/git/objectpool/pool.go
@@ -57,10 +57,6 @@ func NewObjectPool(
return nil, err
}
- if !housekeeping.IsPoolPath(relativePath) {
- return nil, ErrInvalidPoolDir
- }
-
pool := &ObjectPool{
gitCmdFactory: gitCmdFactory,
txManager: txManager,
@@ -71,6 +67,10 @@ func NewObjectPool(
}
pool.Repo = localrepo.New(locator, gitCmdFactory, catfileCache, pool)
+ if !housekeeping.IsPoolRepository(pool) {
+ return nil, ErrInvalidPoolDir
+ }
+
return pool, nil
}
diff --git a/internal/git/objectpool/testhelper_test.go b/internal/git/objectpool/testhelper_test.go
index 20c02c65e..6254ce2c1 100644
--- a/internal/git/objectpool/testhelper_test.go
+++ b/internal/git/objectpool/testhelper_test.go
@@ -24,7 +24,10 @@ func TestMain(m *testing.M) {
func setupObjectPool(t *testing.T, ctx context.Context) (config.Cfg, *ObjectPool, *gitalypb.Repository) {
t.Helper()
- cfg, repo, _ := testcfg.BuildWithRepo(t)
+ cfg, repo, repoPath := testcfg.BuildWithRepo(t)
+
+ gittest.FixGitLabTestRepoForCommitGraphs(t, cfg, repoPath)
+
gitCommandFactory := gittest.NewCommandFactory(t, cfg, git.WithSkipHooks())
catfileCache := catfile.NewCache(cfg)
diff --git a/internal/git/reference.go b/internal/git/reference.go
index 186bb89bb..c63230b06 100644
--- a/internal/git/reference.go
+++ b/internal/git/reference.go
@@ -19,6 +19,10 @@ var InternalRefPrefixes = [...]string{
"refs/tmp/",
}
+// ObjectPoolRefNamespace is the namespace used for the references of the primary pool member part
+// of an object pool.
+const ObjectPoolRefNamespace = "refs/remotes/origin"
+
// Revision represents anything that resolves to either a commit, multiple
// commits or to an object different than a commit. This could be e.g.
// "master", "master^{commit}", an object hash or similar. See gitrevisions(1)
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 c6539541c..df1951dba 100644
--- a/internal/gitaly/service/objectpool/fetch_into_object_pool_test.go
+++ b/internal/gitaly/service/objectpool/fetch_into_object_pool_test.go
@@ -1,6 +1,7 @@
package objectpool
import (
+ "context"
"os"
"path/filepath"
"testing"
@@ -17,6 +18,7 @@ import (
"gitlab.com/gitlab-org/gitaly/v15/internal/git/housekeeping"
"gitlab.com/gitlab-org/gitaly/v15/internal/gitaly/config"
"gitlab.com/gitlab-org/gitaly/v15/internal/gitaly/transaction"
+ "gitlab.com/gitlab-org/gitaly/v15/internal/metadata/featureflag"
"gitlab.com/gitlab-org/gitaly/v15/internal/testhelper"
"gitlab.com/gitlab-org/gitaly/v15/internal/testhelper/testcfg"
"gitlab.com/gitlab-org/gitaly/v15/internal/testhelper/testserver"
@@ -29,7 +31,13 @@ import (
)
func TestFetchIntoObjectPool_Success(t *testing.T) {
- ctx := testhelper.Context(t)
+ t.Parallel()
+ testhelper.NewFeatureSets(featureflag.FetchIntoObjectPoolOptimizeRepository).Run(t, testFetchIntoObjectPoolSuccess)
+}
+
+func testFetchIntoObjectPoolSuccess(t *testing.T, ctx context.Context) {
+ t.Parallel()
+
cfg, repo, repoPath, locator, client := setup(ctx, t)
repoCommit := gittest.WriteCommit(t, cfg, repoPath, gittest.WithBranch(t.Name()))
@@ -57,12 +65,21 @@ func TestFetchIntoObjectPool_Success(t *testing.T) {
// No problems
gittest.Exec(t, cfg, "-C", pool.FullPath(), "fsck")
- packFiles, err := filepath.Glob(filepath.Join(pool.FullPath(), "objects", "pack", "pack-*.pack"))
- require.NoError(t, err)
- require.Len(t, packFiles, 1, "ensure commits got packed")
+ // Verify that the newly written commit exists in the repository now.
+ gittest.Exec(t, cfg, "-C", pool.FullPath(), "rev-parse", "--verify", repoCommit.String())
+
+ if featureflag.FetchIntoObjectPoolOptimizeRepository.IsDisabled(ctx) {
+ // We used to verify here how objects have been packed. This doesn't make a lot of
+ // sense anymore in the new world though, where we rely on the housekeeping package
+ // to do repository maintenance for us. We thus only check for this when the feature
+ // flag is disabled.
+ packFiles, err := filepath.Glob(filepath.Join(pool.FullPath(), "objects", "pack", "pack-*.pack"))
+ require.NoError(t, err)
+ require.Len(t, packFiles, 1, "ensure commits got packed")
- packContents := gittest.Exec(t, cfg, "-C", pool.FullPath(), "verify-pack", "-v", packFiles[0])
- require.Contains(t, string(packContents), repoCommit)
+ packContents := gittest.Exec(t, cfg, "-C", pool.FullPath(), "verify-pack", "-v", packFiles[0])
+ require.Contains(t, string(packContents), repoCommit.String())
+ }
_, err = client.FetchIntoObjectPool(ctx, req)
require.NoError(t, err, "calling FetchIntoObjectPool twice should be OK")
@@ -124,8 +141,14 @@ func TestFetchIntoObjectPool_hooks(t *testing.T) {
}
func TestFetchIntoObjectPool_CollectLogStatistics(t *testing.T) {
- cfg := testcfg.Build(t)
+ t.Parallel()
+ testhelper.NewFeatureSets(featureflag.FetchIntoObjectPoolOptimizeRepository).Run(t, testFetchIntoObjectPoolCollectLogStatistics)
+}
+func testFetchIntoObjectPoolCollectLogStatistics(t *testing.T, ctx context.Context) {
+ t.Parallel()
+
+ cfg := testcfg.Build(t)
testcfg.BuildGitalyHooks(t, cfg)
locator := config.NewLocator(cfg)
@@ -133,7 +156,6 @@ func TestFetchIntoObjectPool_CollectLogStatistics(t *testing.T) {
logger, hook := test.NewNullLogger()
cfg.SocketPath = runObjectPoolServer(t, cfg, locator, logger)
- ctx := testhelper.Context(t)
ctx = ctxlogrus.ToContext(ctx, log.WithField("test", "logging"))
repo, _ := gittest.CreateRepository(ctx, t, cfg, gittest.CreateRepositoryConfig{
Seed: gittest.SeedGitLabTest,
diff --git a/internal/gitaly/service/objectpool/testhelper_test.go b/internal/gitaly/service/objectpool/testhelper_test.go
index bb8c17f05..09c116974 100644
--- a/internal/gitaly/service/objectpool/testhelper_test.go
+++ b/internal/gitaly/service/objectpool/testhelper_test.go
@@ -61,6 +61,14 @@ func setup(ctx context.Context, t *testing.T, opts ...testserver.GitalyServerOpt
Seed: gittest.SeedGitLabTest,
})
+ // For the time being we need to fix up the repository to make it work in the context of
+ // commit-graphs. Because initializing the pool from the repository will just copy over
+ // objects 1:1 we also need to repack the repository, or otherwise it would still have a
+ // reference to the deleted commit. And because we make sure to keep alive dangling objects
+ // via keep-around references we'd thus make the broken commit reachable again.
+ gittest.FixGitLabTestRepoForCommitGraphs(t, cfg, repoPath)
+ gittest.Exec(t, cfg, "-C", repoPath, "repack", "-a", "-d")
+
return cfg, repo, repoPath, locator, clientWithConn{ObjectPoolServiceClient: gitalypb.NewObjectPoolServiceClient(conn), conn: conn}
}
diff --git a/internal/gitaly/service/repository/gc.go b/internal/gitaly/service/repository/gc.go
index 15fea86c0..8bce85540 100644
--- a/internal/gitaly/service/repository/gc.go
+++ b/internal/gitaly/service/repository/gc.go
@@ -61,7 +61,7 @@ func (s *server) GarbageCollect(ctx context.Context, in *gitalypb.GarbageCollect
}
func (s *server) gc(ctx context.Context, in *gitalypb.GarbageCollectRequest) error {
- config := append(housekeeping.GetRepackGitConfig(ctx, in.CreateBitmap), git.ConfigPair{Key: "gc.writeCommitGraph", Value: "false"})
+ config := append(housekeeping.GetRepackGitConfig(ctx, in.GetRepository(), in.CreateBitmap), git.ConfigPair{Key: "gc.writeCommitGraph", Value: "false"})
var flags []git.Option
if in.Prune {
diff --git a/internal/gitaly/service/repository/gc_test.go b/internal/gitaly/service/repository/gc_test.go
index 4c28c5058..3bd186a8c 100644
--- a/internal/gitaly/service/repository/gc_test.go
+++ b/internal/gitaly/service/repository/gc_test.go
@@ -497,7 +497,7 @@ func TestGarbageCollectDeltaIslands(t *testing.T) {
ctx := testhelper.Context(t)
cfg, repo, repoPath, client := setupRepositoryService(ctx, t)
- gittest.TestDeltaIslands(t, cfg, repoPath, func() error {
+ gittest.TestDeltaIslands(t, cfg, repoPath, false, func() error {
//nolint:staticcheck
_, err := client.GarbageCollect(ctx, &gitalypb.GarbageCollectRequest{Repository: repo})
return err
diff --git a/internal/gitaly/service/repository/midx.go b/internal/gitaly/service/repository/midx.go
index 151c6901b..b72cbd3a5 100644
--- a/internal/gitaly/service/repository/midx.go
+++ b/internal/gitaly/service/repository/midx.go
@@ -186,7 +186,7 @@ func (s *server) midxRepack(ctx context.Context, repo repository.GitRepo) error
git.ValueFlag{Name: "--batch-size", Value: strconv.FormatInt(batchSize, 10)},
},
},
- git.WithConfig(housekeeping.GetRepackGitConfig(ctx, false)...),
+ git.WithConfig(housekeeping.GetRepackGitConfig(ctx, repo, false)...),
)
if err != nil {
return err
diff --git a/internal/gitaly/service/repository/repack_test.go b/internal/gitaly/service/repository/repack_test.go
index 6a90cbdb5..705134269 100644
--- a/internal/gitaly/service/repository/repack_test.go
+++ b/internal/gitaly/service/repository/repack_test.go
@@ -298,7 +298,7 @@ func TestRepackFullDeltaIslands(t *testing.T) {
ctx := testhelper.Context(t)
cfg, repo, repoPath, client := setupRepositoryService(ctx, t)
- gittest.TestDeltaIslands(t, cfg, repoPath, func() error {
+ gittest.TestDeltaIslands(t, cfg, repoPath, false, func() error {
//nolint:staticcheck
_, err := client.RepackFull(ctx, &gitalypb.RepackFullRequest{Repository: repo})
return err
diff --git a/internal/metadata/featureflag/ff_fetch_into_object_pool_optimize_repository.go b/internal/metadata/featureflag/ff_fetch_into_object_pool_optimize_repository.go
new file mode 100644
index 000000000..0d47c50df
--- /dev/null
+++ b/internal/metadata/featureflag/ff_fetch_into_object_pool_optimize_repository.go
@@ -0,0 +1,10 @@
+package featureflag
+
+// FetchIntoObjectPoolOptimizeRepository will cause FetchIntoObjectPool to use OptimizeRepository to
+// maintain the object pool instead of manually performing repository maintenance.
+var FetchIntoObjectPoolOptimizeRepository = NewFeatureFlag(
+ "fetch_into_object_pool_optimize_repository",
+ "v15.2.0",
+ "https://gitlab.com/gitlab-org/gitaly/-/issues/4342",
+ false,
+)
diff --git a/internal/praefect/delete_object_pool.go b/internal/praefect/delete_object_pool.go
index 83d8ab9dd..c10449d42 100644
--- a/internal/praefect/delete_object_pool.go
+++ b/internal/praefect/delete_object_pool.go
@@ -29,7 +29,7 @@ func DeleteObjectPoolHandler(rs datastore.RepositoryStore, conns Connections) gr
return nil, err
}
- if !housekeeping.IsRailsPoolPath(repo.GetRelativePath()) {
+ if !housekeeping.IsRailsPoolRepository(repo) {
return nil, helper.ErrInvalidArgument(objectpool.ErrInvalidPoolDir)
}
diff --git a/internal/praefect/praefectutil/replica_path.go b/internal/praefect/praefectutil/replica_path.go
index 576762d22..1bf34b33b 100644
--- a/internal/praefect/praefectutil/replica_path.go
+++ b/internal/praefect/praefectutil/replica_path.go
@@ -6,14 +6,16 @@ import (
"path/filepath"
"strconv"
"strings"
+
+ "gitlab.com/gitlab-org/gitaly/v15/internal/git/repository"
)
// poolPathPrefix is the prefix directory where Praefect places object pools.
const poolPathPrefix = "@cluster/pools/"
-// IsPoolPath returns whether the relative path indicates this is a Praefect generated object pool path.
-func IsPoolPath(relativePath string) bool {
- return strings.HasPrefix(relativePath, poolPathPrefix)
+// IsPoolRepository returns whether the repository is a Praefect generated object pool repository.
+func IsPoolRepository(repo repository.GitRepo) bool {
+ return strings.HasPrefix(repo.GetRelativePath(), poolPathPrefix)
}
// DeriveReplicaPath derives a repository's disk storage path from its repository ID. The repository ID
diff --git a/internal/praefect/praefectutil/replica_path_test.go b/internal/praefect/praefectutil/replica_path_test.go
index be9f42e24..d168778c1 100644
--- a/internal/praefect/praefectutil/replica_path_test.go
+++ b/internal/praefect/praefectutil/replica_path_test.go
@@ -5,6 +5,7 @@ import (
"github.com/stretchr/testify/require"
"gitlab.com/gitlab-org/gitaly/v15/internal/git/gittest"
+ "gitlab.com/gitlab-org/gitaly/v15/proto/go/gitalypb"
)
func TestDeriveReplicaPath(t *testing.T) {
@@ -17,28 +18,45 @@ func TestDerivePoolPath(t *testing.T) {
require.Equal(t, "@cluster/pools/d4/73/2", DerivePoolPath(2))
}
-func TestIsPoolPath(t *testing.T) {
+func TestIsPoolRepository(t *testing.T) {
for _, tc := range []struct {
- desc string
- relativePath string
- isPoolPath bool
+ desc string
+ repo *gitalypb.Repository
+ isPoolPath bool
}{
{
- desc: "praefect pool path",
- relativePath: DerivePoolPath(1),
- isPoolPath: true,
+ desc: "missing repository",
+ isPoolPath: false,
},
{
- desc: "praefect replica path",
- relativePath: DeriveReplicaPath(1),
+ desc: "empty string",
+ repo: &gitalypb.Repository{
+ RelativePath: "",
+ },
+ isPoolPath: false,
},
{
- desc: "rails pool path",
- relativePath: gittest.NewObjectPoolName(t),
+ desc: "praefect pool path",
+ repo: &gitalypb.Repository{
+ RelativePath: DerivePoolPath(1),
+ },
+ isPoolPath: true,
+ },
+ {
+ desc: "praefect replica path",
+ repo: &gitalypb.Repository{
+ RelativePath: DeriveReplicaPath(1),
+ },
+ },
+ {
+ desc: "rails pool path",
+ repo: &gitalypb.Repository{
+ RelativePath: gittest.NewObjectPoolName(t),
+ },
},
} {
t.Run(tc.desc, func(t *testing.T) {
- require.Equal(t, tc.isPoolPath, IsPoolPath(tc.relativePath))
+ require.Equal(t, tc.isPoolPath, IsPoolRepository(tc.repo))
})
}
}
diff --git a/internal/praefect/router_per_repository.go b/internal/praefect/router_per_repository.go
index ea3d60118..287d8d33b 100644
--- a/internal/praefect/router_per_repository.go
+++ b/internal/praefect/router_per_repository.go
@@ -10,6 +10,7 @@ import (
"gitlab.com/gitlab-org/gitaly/v15/internal/praefect/datastore"
"gitlab.com/gitlab-org/gitaly/v15/internal/praefect/nodes"
"gitlab.com/gitlab-org/gitaly/v15/internal/praefect/praefectutil"
+ "gitlab.com/gitlab-org/gitaly/v15/proto/go/gitalypb"
"google.golang.org/grpc"
)
@@ -313,7 +314,10 @@ func (r *PerRepositoryRouter) RouteRepositoryCreation(ctx context.Context, virtu
replicaPath := relativePath
if featureflag.PraefectGeneratedReplicaPaths.IsEnabled(ctx) {
replicaPath = praefectutil.DeriveReplicaPath(id)
- if housekeeping.IsRailsPoolPath(relativePath) {
+ if housekeeping.IsRailsPoolRepository(&gitalypb.Repository{
+ StorageName: virtualStorage,
+ RelativePath: relativePath,
+ }) {
replicaPath = praefectutil.DerivePoolPath(id)
}
}