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-03-15 13:33:32 +0300
committerSami Hiltunen <shiltunen@gitlab.com>2021-03-22 14:11:53 +0300
commit9d79c7a12d0d80f61dd94fa64ac6bb3e675d2d99 (patch)
tree6ea7055d105786eb896ec78758fec7f89d0763f9
parentc98cffd1bb9d7dc4079b77d9f3ba0d890c9610b1 (diff)
port UpdateRemoteMirror into Go
This commit adds a Go port of UpdateRemoteMirror behind a feature flag. UpdateRemoteMirror is an RPC that updates a remote mirror of the repository by pushing to it. It first fetches the remote's refs, compares them to the local ones and pushes to any refs that need to be updated or deleted. Various bugs were discovered while porting the RPC. These bugs have test cases and links to the relevant issues. I've kept the bug compatibility now so the test cases pass for both the original Ruby implementation and the Go port.
-rw-r--r--internal/gitaly/service/remote/update_remote_mirror.go194
-rw-r--r--internal/gitaly/service/remote/update_remote_mirror_test.go467
-rw-r--r--internal/metadata/featureflag/feature_flags.go3
3 files changed, 652 insertions, 12 deletions
diff --git a/internal/gitaly/service/remote/update_remote_mirror.go b/internal/gitaly/service/remote/update_remote_mirror.go
index dfe1deb48..4fdc11dfa 100644
--- a/internal/gitaly/service/remote/update_remote_mirror.go
+++ b/internal/gitaly/service/remote/update_remote_mirror.go
@@ -1,13 +1,23 @@
package remote
import (
+ "errors"
"fmt"
+ "io"
+ "regexp"
+ "strings"
+ "gitlab.com/gitlab-org/gitaly/internal/git"
+ "gitlab.com/gitlab-org/gitaly/internal/git/localrepo"
"gitlab.com/gitlab-org/gitaly/internal/gitaly/rubyserver"
"gitlab.com/gitlab-org/gitaly/internal/helper"
+ "gitlab.com/gitlab-org/gitaly/internal/metadata/featureflag"
"gitlab.com/gitlab-org/gitaly/proto/go/gitalypb"
)
+// pushBatchSize is the maximum number of branches to push in a single push call.
+const pushBatchSize = 10
+
func (s *server) UpdateRemoteMirror(stream gitalypb.RemoteService_UpdateRemoteMirrorServer) error {
firstRequest, err := stream.Recv()
if err != nil {
@@ -18,6 +28,14 @@ func (s *server) UpdateRemoteMirror(stream gitalypb.RemoteService_UpdateRemoteMi
return helper.ErrInvalidArgument(err)
}
+ if featureflag.IsEnabled(stream.Context(), featureflag.GoUpdateRemoteMirror) {
+ if err := s.goUpdateRemoteMirror(stream, firstRequest); err != nil {
+ return helper.ErrInternal(err)
+ }
+
+ return nil
+ }
+
if err := s.updateRemoteMirror(stream, firstRequest); err != nil {
return helper.ErrInternal(err)
}
@@ -73,6 +91,182 @@ func (s *server) updateRemoteMirror(stream gitalypb.RemoteService_UpdateRemoteMi
return nil
}
+func (s *server) goUpdateRemoteMirror(stream gitalypb.RemoteService_UpdateRemoteMirrorServer, firstRequest *gitalypb.UpdateRemoteMirrorRequest) error {
+ ctx := stream.Context()
+
+ branchMatchers := firstRequest.GetOnlyBranchesMatching()
+ for {
+ req, err := stream.Recv()
+ if err != nil {
+ if errors.Is(err, io.EOF) {
+ break
+ }
+
+ return fmt.Errorf("receive: %w", err)
+ }
+
+ branchMatchers = append(branchMatchers, req.GetOnlyBranchesMatching()...)
+ }
+
+ referenceMatcher, err := newReferenceMatcher(branchMatchers)
+ if err != nil {
+ return fmt.Errorf("create reference matcher: %w", err)
+ }
+
+ repo := localrepo.New(s.gitCmdFactory, firstRequest.GetRepository(), s.cfg)
+ remoteRefsSlice, err := repo.GetRemoteReferences(ctx, firstRequest.GetRefName(), "refs/heads/*", "refs/tags/*")
+ if err != nil {
+ return fmt.Errorf("get remote references: %w", err)
+ }
+
+ localRefs, err := repo.GetReferences(ctx, "refs/heads/", "refs/tags/")
+ if err != nil {
+ return fmt.Errorf("get local references: %w", err)
+ }
+
+ if len(localRefs) == 0 {
+ // https://gitlab.com/gitlab-org/gitaly/-/issues/3503
+ return errors.New("close stream to gitaly-ruby: rpc error: code = Unknown desc = NoMethodError: undefined method `id' for nil:NilClass")
+ }
+
+ remoteRefs := make(map[git.ReferenceName]string, len(remoteRefsSlice))
+ for _, ref := range remoteRefsSlice {
+ remoteRefs[ref.Name] = ref.Target
+ }
+
+ var divergentRefs [][]byte
+ toUpdate := map[git.ReferenceName]string{}
+ for _, localRef := range localRefs {
+ remoteTarget, ok := remoteRefs[localRef.Name]
+ if !ok {
+ // ref does not exist on the mirror, it should be created
+ toUpdate[localRef.Name] = localRef.Target
+ delete(remoteRefs, localRef.Name)
+ continue
+ }
+
+ if remoteTarget == localRef.Target {
+ // ref is up to date on the mirror
+ delete(remoteRefs, localRef.Name)
+ continue
+ }
+
+ if firstRequest.GetKeepDivergentRefs() {
+ isAncestor, err := repo.IsAncestor(ctx, git.Revision(remoteTarget), git.Revision(localRef.Target))
+ if err != nil && !errors.Is(err, localrepo.InvalidCommitError(remoteTarget)) {
+ return fmt.Errorf("is ancestor: %w", err)
+ }
+
+ if !isAncestor {
+ // The mirror's reference has diverged from the local ref, or the mirror contains a commit
+ // which is not present in the local repository.
+ divergentRefs = append(divergentRefs, []byte(localRef.Name))
+ delete(remoteRefs, localRef.Name)
+ continue
+ }
+ }
+
+ if localRef.Name == "refs/heads/tag" {
+ // https://gitlab.com/gitlab-org/gitaly/-/issues/3502
+ return errors.New("close stream to gitaly-ruby: rpc error: code = Unknown desc = Gitlab::Git::CommandError: fatal: tag shorthand without <tag>")
+ }
+
+ // the mirror's ref does not match ours, we should update it.
+ toUpdate[localRef.Name] = localRef.Target
+ delete(remoteRefs, localRef.Name)
+ }
+
+ toDelete := remoteRefs
+ if firstRequest.GetKeepDivergentRefs() {
+ toDelete = map[git.ReferenceName]string{}
+ }
+
+ seen := map[string]struct{}{}
+ var refspecs []string
+ for prefix, references := range map[string]map[git.ReferenceName]string{
+ "": toUpdate, ":": toDelete,
+ } {
+ for reference := range references {
+ if !referenceMatcher.MatchString(reference.String()) {
+ continue
+ }
+
+ refspecs = append(refspecs, prefix+reference.String())
+
+ // https://gitlab.com/gitlab-org/gitaly/-/issues/3504
+ name := strings.TrimPrefix(reference.String(), "refs/heads/")
+ if strings.HasPrefix(reference.String(), "refs/tags/") {
+ name = strings.TrimPrefix(reference.String(), "refs/tags/")
+ }
+
+ if _, ok := seen[name]; ok {
+ return errors.New("close stream to gitaly-ruby: rpc error: code = Unknown desc = Gitlab::Git::CommandError: error: src refspec master matches more than one")
+ }
+
+ seen[name] = struct{}{}
+ }
+ }
+
+ if len(refspecs) > 0 {
+ sshCommand, clean, err := git.BuildSSHInvocation(ctx, firstRequest.GetSshKey(), firstRequest.GetKnownHosts())
+ if err != nil {
+ return fmt.Errorf("build ssh invocation: %w", err)
+ }
+ defer clean()
+
+ for len(refspecs) > 0 {
+ batch := refspecs
+ if len(refspecs) > pushBatchSize {
+ batch = refspecs[:pushBatchSize]
+ }
+
+ refspecs = refspecs[len(batch):]
+
+ // The refs could have been modified on the mirror during after we fetched them.
+ // This could cause divergent refs to be force pushed over even with keep_divergent_refs set.
+ // This could be addressed by force pushing only if the current ref still matches what
+ // we received in the original fetch. https://gitlab.com/gitlab-org/gitaly/-/issues/3505
+ if err := repo.Push(ctx, firstRequest.GetRefName(), batch, localrepo.PushOptions{SSHCommand: sshCommand}); err != nil {
+ return fmt.Errorf("push to mirror: %w", err)
+ }
+ }
+ }
+
+ return stream.SendAndClose(&gitalypb.UpdateRemoteMirrorResponse{DivergentRefs: divergentRefs})
+}
+
+// newReferenceMatcher returns a regexp which matches references that should
+// be updated in the mirror repository. Tags are always matched successfully.
+// branchMatchers optionally contain patterns that are used to match branches.
+// The patterns should only include the branch name without the `refs/heads/`
+// prefix. "*" can be used as a wilcard in the patterns. If no branchMatchers
+// are specified, all branches are matched successfully.
+func newReferenceMatcher(branchMatchers [][]byte) (*regexp.Regexp, error) {
+ sb := &strings.Builder{}
+ sb.WriteString("^refs/tags/.+$|^refs/heads/(")
+
+ for i, expression := range branchMatchers {
+ segments := strings.Split(string(expression), "*")
+ for i := range segments {
+ segments[i] = regexp.QuoteMeta(segments[i])
+ }
+
+ sb.WriteString(strings.Join(segments, ".*"))
+
+ if i < len(branchMatchers)-1 {
+ sb.WriteString("|")
+ }
+ }
+
+ if len(branchMatchers) == 0 {
+ sb.WriteString(".+")
+ }
+
+ sb.WriteString(")$")
+
+ return regexp.Compile(sb.String())
+}
+
func validateUpdateRemoteMirrorRequest(req *gitalypb.UpdateRemoteMirrorRequest) error {
if req.GetRepository() == nil {
return fmt.Errorf("empty Repository")
diff --git a/internal/gitaly/service/remote/update_remote_mirror_test.go b/internal/gitaly/service/remote/update_remote_mirror_test.go
index 0f210db04..57948631c 100644
--- a/internal/gitaly/service/remote/update_remote_mirror_test.go
+++ b/internal/gitaly/service/remote/update_remote_mirror_test.go
@@ -1,25 +1,459 @@
package remote
import (
+ "context"
+ "fmt"
+ "path/filepath"
"strings"
"testing"
+ "time"
"github.com/stretchr/testify/require"
+ "gitlab.com/gitlab-org/gitaly/internal/git"
"gitlab.com/gitlab-org/gitaly/internal/git/gittest"
+ "gitlab.com/gitlab-org/gitaly/internal/git2go"
+ "gitlab.com/gitlab-org/gitaly/internal/gitaly/config"
+ "gitlab.com/gitlab-org/gitaly/internal/helper/text"
+ "gitlab.com/gitlab-org/gitaly/internal/metadata/featureflag"
"gitlab.com/gitlab-org/gitaly/internal/testhelper"
"gitlab.com/gitlab-org/gitaly/proto/go/gitalypb"
"google.golang.org/grpc/codes"
)
-func TestSuccessfulUpdateRemoteMirrorRequest(t *testing.T) {
+func TestUpdateRemoteMirror(t *testing.T) {
+ testhelper.NewFeatureSets([]featureflag.FeatureFlag{
+ featureflag.GoUpdateRemoteMirror,
+ }).Run(t, testUpdateRemoteMirror)
+}
+
+func testUpdateRemoteMirror(t *testing.T, ctx context.Context) {
+ tmpDir, cleanTempDir := testhelper.TempDir(t)
+ defer cleanTempDir()
+
+ testhelper.ConfigureGitalyGit2Go(tmpDir)
+
serverSocketPath, stop := RunRemoteServiceServer(t)
defer stop()
client, conn := NewRemoteClient(t, serverSocketPath)
defer conn.Close()
- ctx, cancel := testhelper.Context()
- defer cancel()
+ type refs map[string][]string
+
+ for _, tc := range []struct {
+ desc string
+ sourceRefs refs
+ sourceSymRefs map[string]string
+ mirrorRefs refs
+ mirrorSymRefs map[string]string
+ keepDivergentRefs bool
+ onlyBranchesMatching []string
+ requests []*gitalypb.UpdateRemoteMirrorRequest
+ errorContains string
+ response *gitalypb.UpdateRemoteMirrorResponse
+ expectedMirrorRefs map[string]string
+ }{
+ {
+ // https://gitlab.com/gitlab-org/gitaly/-/issues/3503
+ desc: "empty mirror source fails",
+ mirrorRefs: refs{
+ "refs/heads/tags": {"commit 1"},
+ },
+ errorContains: "rpc error: code = Internal desc = close stream to gitaly-ruby: rpc error: code = Unknown desc = NoMethodError: undefined method `id' for nil:NilClass",
+ },
+ {
+ desc: "mirror is up to date",
+ response: &gitalypb.UpdateRemoteMirrorResponse{},
+ sourceRefs: refs{
+ "refs/heads/master": {"commit 1"},
+ "refs/tags/tag": {"commit 1"},
+ },
+ mirrorRefs: refs{
+ "refs/heads/master": {"commit 1"},
+ "refs/tags/tag": {"commit 1"},
+ },
+ expectedMirrorRefs: map[string]string{
+ "refs/heads/master": "commit 1",
+ "refs/tags/tag": "commit 1",
+ },
+ },
+ {
+ desc: "creates missing references",
+ sourceRefs: refs{
+ "refs/heads/master": {"commit 1"},
+ "refs/tags/tag": {"commit 1"},
+ },
+ response: &gitalypb.UpdateRemoteMirrorResponse{},
+ expectedMirrorRefs: map[string]string{
+ "refs/heads/master": "commit 1",
+ "refs/tags/tag": "commit 1",
+ },
+ },
+ {
+ desc: "updates outdated references",
+ sourceRefs: refs{
+ "refs/heads/master": {"commit 1", "commit 2"},
+ "refs/tags/tag": {"commit 1", "commit 2"},
+ },
+ mirrorRefs: refs{
+ "refs/heads/master": {"commit 1"},
+ "refs/tags/tag": {"commit 1"},
+ },
+ response: &gitalypb.UpdateRemoteMirrorResponse{},
+ expectedMirrorRefs: map[string]string{
+ "refs/heads/master": "commit 2",
+ "refs/tags/tag": "commit 2",
+ },
+ },
+ {
+ desc: "deletes unneeded references",
+ sourceRefs: refs{
+ "refs/heads/master": {"commit 1"},
+ },
+ mirrorRefs: refs{
+ "refs/heads/master": {"commit 1"},
+ "refs/heads/branch": {"commit 1"},
+ "refs/tags/tag": {"commit 1"},
+ },
+ response: &gitalypb.UpdateRemoteMirrorResponse{},
+ expectedMirrorRefs: map[string]string{
+ "refs/heads/master": "commit 1",
+ },
+ },
+ {
+ desc: "deletes unneeded references that match the branch selector",
+ sourceRefs: refs{
+ "refs/heads/master": {"commit 1"},
+ },
+ mirrorRefs: refs{
+ "refs/heads/master": {"commit 1"},
+ "refs/heads/matched": {"commit 1"},
+ "refs/heads/not-matched": {"commit 1"},
+ "refs/tags/tag": {"commit 1"},
+ },
+ onlyBranchesMatching: []string{"matched"},
+ response: &gitalypb.UpdateRemoteMirrorResponse{},
+ expectedMirrorRefs: map[string]string{
+ "refs/heads/master": "commit 1",
+ "refs/heads/not-matched": "commit 1",
+ },
+ },
+ {
+ desc: "does not delete refs with KeepDivergentRefs",
+ sourceRefs: refs{
+ "refs/heads/master": {"commit 1"},
+ },
+ keepDivergentRefs: true,
+ mirrorRefs: refs{
+ "refs/heads/master": {"commit 1"},
+ "refs/heads/branch": {"commit 1"},
+ "refs/tags/tag": {"commit 1"},
+ },
+ response: &gitalypb.UpdateRemoteMirrorResponse{},
+ expectedMirrorRefs: map[string]string{
+ "refs/heads/master": "commit 1",
+ "refs/heads/branch": "commit 1",
+ "refs/tags/tag": "commit 1",
+ },
+ },
+ {
+ // https://gitlab.com/gitlab-org/gitaly/-/issues/3502
+ desc: "updating branch called tag fails",
+ sourceRefs: refs{
+ "refs/heads/tag": {"commit 1", "commit 2"},
+ },
+ mirrorRefs: refs{
+ "refs/heads/tag": {"commit 1"},
+ },
+ errorContains: "rpc error: code = Internal desc = close stream to gitaly-ruby: rpc error: code = Unknown desc = Gitlab::Git::CommandError: fatal: tag shorthand without <tag>",
+ },
+ {
+ // https://gitlab.com/gitlab-org/gitaly/-/issues/3504
+ desc: "fails if tag and branch named the same",
+ sourceRefs: refs{
+ "refs/heads/master": {"commit 1"},
+ "refs/tags/master": {"commit 1"},
+ },
+ errorContains: "rpc error: code = Internal desc = close stream to gitaly-ruby: rpc error: code = Unknown desc = Gitlab::Git::CommandError: error: src refspec master matches more than one",
+ },
+ {
+ desc: "only local branches are considered",
+ sourceRefs: refs{
+ "refs/heads/master": {"commit 1"},
+ "refs/remote/local-remote/branch": {"commit 1"},
+ },
+ mirrorRefs: refs{
+ "refs/remote/mirror-remote/branch": {"commit 1"},
+ },
+ response: &gitalypb.UpdateRemoteMirrorResponse{},
+ expectedMirrorRefs: map[string]string{
+ "refs/heads/master": "commit 1",
+ "refs/remote/mirror-remote/branch": "commit 1",
+ },
+ },
+ {
+ desc: "creates branches matching selector",
+ sourceRefs: refs{
+ "refs/heads/matches": {"commit 1"},
+ "refs/heads/does-not-match": {"commit 2"},
+ "refs/tags/tag": {"commit 3"},
+ },
+ onlyBranchesMatching: []string{"matches"},
+ response: &gitalypb.UpdateRemoteMirrorResponse{},
+ expectedMirrorRefs: map[string]string{
+ "refs/heads/matches": "commit 1",
+ "refs/tags/tag": "commit 3",
+ },
+ },
+ {
+ desc: "updates branches matching selector",
+ sourceRefs: refs{
+ "refs/heads/matches": {"commit 1", "commit 2"},
+ "refs/heads/does-not-match": {"commit 3", "commit 4"},
+ "refs/tags/tag": {"commit 6"},
+ },
+ mirrorRefs: refs{
+ "refs/heads/matches": {"commit 1"},
+ "refs/heads/does-not-match": {"commit 3"},
+ "refs/tags/tag": {"commit 5"},
+ },
+ onlyBranchesMatching: []string{"matches"},
+ response: &gitalypb.UpdateRemoteMirrorResponse{},
+ expectedMirrorRefs: map[string]string{
+ "refs/heads/matches": "commit 2",
+ "refs/heads/does-not-match": "commit 3",
+ "refs/tags/tag": "commit 6",
+ },
+ },
+ {
+ // https://gitlab.com/gitlab-org/gitaly/-/issues/3509
+ desc: "overwrites diverged references without KeepDivergentRefs",
+ sourceRefs: refs{
+ "refs/heads/non-diverged": {"commit 1", "commit 2"},
+ "refs/heads/master": {"commit 2"},
+ "refs/tags/tag-1": {"commit 1"},
+ },
+ mirrorRefs: refs{
+ "refs/heads/non-diverged": {"commit 1"},
+ "refs/heads/master": {"commit 2", "ahead"},
+ "refs/tags/tag-1": {"commit 2"},
+ },
+ response: &gitalypb.UpdateRemoteMirrorResponse{},
+ expectedMirrorRefs: map[string]string{
+ "refs/heads/non-diverged": "commit 2",
+ "refs/heads/master": "commit 2",
+ "refs/tags/tag-1": "commit 1",
+ },
+ },
+ {
+ // https://gitlab.com/gitlab-org/gitaly/-/issues/3509
+ desc: "keeps diverged references with KeepDivergentRefs",
+ sourceRefs: refs{
+ "refs/heads/non-diverged": {"commit 1", "commit 2"},
+ "refs/heads/master": {"commit 2"},
+ "refs/tags/tag-1": {"commit 1"},
+ },
+ mirrorRefs: refs{
+ "refs/heads/non-diverged": {"commit 1"},
+ "refs/heads/master": {"commit 2", "ahead"},
+ "refs/tags/tag-1": {"commit 2"},
+ },
+ keepDivergentRefs: true,
+ response: &gitalypb.UpdateRemoteMirrorResponse{
+ DivergentRefs: [][]byte{
+ []byte("refs/heads/master"),
+ []byte("refs/tags/tag-1"),
+ },
+ },
+ expectedMirrorRefs: map[string]string{
+ "refs/heads/non-diverged": "commit 2",
+ "refs/heads/master": "ahead",
+ "refs/tags/tag-1": "commit 2",
+ },
+ },
+ {
+ // https://gitlab.com/gitlab-org/gitaly/-/issues/3508
+ desc: "mirror is up to date with symbolic reference",
+ sourceRefs: refs{
+ "refs/heads/master": {"commit 1"},
+ },
+ sourceSymRefs: map[string]string{
+ "refs/heads/symbolic-reference": "refs/heads/master",
+ },
+ mirrorRefs: refs{
+ "refs/heads/master": {"commit 1"},
+ },
+ response: &gitalypb.UpdateRemoteMirrorResponse{},
+ expectedMirrorRefs: map[string]string{
+ "refs/heads/master": "commit 1",
+ },
+ },
+ {
+ // https://gitlab.com/gitlab-org/gitaly/-/issues/3508
+ desc: "updates branch pointed to by symbolic reference",
+ sourceRefs: refs{
+ "refs/heads/master": {"commit 1"},
+ },
+ sourceSymRefs: map[string]string{
+ "refs/heads/symbolic-reference": "refs/heads/master",
+ },
+ onlyBranchesMatching: []string{"symbolic-reference"},
+ response: &gitalypb.UpdateRemoteMirrorResponse{},
+ expectedMirrorRefs: map[string]string{
+ "refs/heads/master": "commit 1",
+ },
+ },
+ {
+ // https://gitlab.com/gitlab-org/gitaly/-/issues/3508
+ //
+ // refs/heads/master gets removed but and a broken sym ref is left in
+ // refs/heads/symbolic-reference
+ desc: "removes symbolic ref target from mirror if not symbolic ref is not present locally",
+ sourceRefs: refs{
+ "refs/heads/master": {"commit 1"},
+ },
+ mirrorRefs: refs{
+ "refs/heads/master": {"commit 1"},
+ },
+ mirrorSymRefs: map[string]string{
+ "refs/heads/symbolic-reference": "refs/heads/master",
+ },
+ response: &gitalypb.UpdateRemoteMirrorResponse{},
+ expectedMirrorRefs: map[string]string{},
+ },
+ {
+ // https://gitlab.com/gitlab-org/gitaly/-/issues/3508
+ desc: "fails with symbolic reference and target in the same push",
+ sourceRefs: refs{
+ "refs/heads/master": {"commit 1"},
+ },
+ sourceSymRefs: map[string]string{
+ "refs/heads/symbolic-reference": "refs/heads/master",
+ },
+ errorContains: "remote: error: cannot lock ref 'refs/heads/master': reference already exists",
+ },
+ {
+ desc: "push batching works",
+ sourceRefs: func() refs {
+ out := refs{}
+ for i := 0; i < 2*pushBatchSize+1; i++ {
+ out[fmt.Sprintf("refs/heads/branch-%d", i)] = []string{"commit 1"}
+ }
+ return out
+ }(),
+ response: &gitalypb.UpdateRemoteMirrorResponse{},
+ expectedMirrorRefs: func() map[string]string {
+ out := map[string]string{}
+ for i := 0; i < 2*pushBatchSize+1; i++ {
+ out[fmt.Sprintf("refs/heads/branch-%d", i)] = "commit 1"
+ }
+ return out
+ }(),
+ },
+ } {
+ t.Run(tc.desc, func(t *testing.T) {
+ _, mirrorRepoPath, cleanMirrorRepo := gittest.InitBareRepo(t)
+ defer cleanMirrorRepo()
+
+ sourceRepoPb, sourceRepoPath, cleanSourceRepo := gittest.InitBareRepo(t)
+ defer cleanSourceRepo()
+
+ // configure the mirror repository as a remote in the source
+ testhelper.MustRunCommand(t, nil, "git", "-C", sourceRepoPath, "remote", "add", "mirror", mirrorRepoPath)
+
+ // create identical commits in both repositories so we can use them for
+ // the references
+ commitSignature := git2go.NewSignature("Test Author", "author@example.com", time.Now())
+ executor := git2go.New(filepath.Join(tmpDir, "gitaly-git2go"), config.Config.Git.BinPath)
+
+ // construct the starting state of the repositories
+ for repoPath, references := range map[string]refs{
+ sourceRepoPath: tc.sourceRefs,
+ mirrorRepoPath: tc.mirrorRefs,
+ } {
+ for reference, commits := range references {
+ var commitOID git.ObjectID
+ for _, commit := range commits {
+ var err error
+ commitOID, err = executor.Commit(ctx, git2go.CommitParams{
+ Repository: repoPath,
+ Author: commitSignature,
+ Committer: commitSignature,
+ Message: commit,
+ Parent: commitOID.String(),
+ })
+ require.NoError(t, err)
+ }
+
+ testhelper.MustRunCommand(t, nil, "git", "-C", repoPath, "update-ref", reference, commitOID.String())
+ }
+ }
+ for repoPath, symRefs := range map[string]map[string]string{
+ sourceRepoPath: tc.sourceSymRefs,
+ mirrorRepoPath: tc.mirrorSymRefs,
+ } {
+ for symRef, targetRef := range symRefs {
+ testhelper.MustRunCommand(t, nil, "git", "-C", repoPath, "symbolic-ref", symRef, targetRef)
+ }
+ }
+
+ stream, err := client.UpdateRemoteMirror(ctx)
+ require.NoError(t, err)
+
+ require.NoError(t, stream.Send(&gitalypb.UpdateRemoteMirrorRequest{
+ Repository: sourceRepoPb,
+ RefName: "mirror",
+ KeepDivergentRefs: tc.keepDivergentRefs,
+ }))
+
+ for _, pattern := range tc.onlyBranchesMatching {
+ require.NoError(t, stream.Send(&gitalypb.UpdateRemoteMirrorRequest{
+ OnlyBranchesMatching: [][]byte{[]byte(pattern)},
+ }))
+ }
+
+ resp, err := stream.CloseAndRecv()
+ if tc.errorContains != "" {
+ testhelper.RequireGrpcError(t, err, codes.Internal)
+ require.Contains(t, err.Error(), tc.errorContains)
+ return
+ }
+
+ require.NoError(t, err)
+ require.Equal(t, tc.response, resp)
+
+ // Check that the refs on the mirror now refer to the correct commits.
+ // This is done by checking the commit messages as the commits are otherwise
+ // the same.
+ actualMirrorRefs := map[string]string{}
+
+ refLines := strings.Split(text.ChompBytes(testhelper.MustRunCommand(t, nil, "git", "-C", mirrorRepoPath, "for-each-ref", "--format=%(refname)%00%(contents:subject)")), "\n")
+ for _, line := range refLines {
+ if line == "" {
+ continue
+ }
+
+ split := strings.Split(line, "\000")
+ actualMirrorRefs[split[0]] = split[1]
+ }
+
+ require.Equal(t, tc.expectedMirrorRefs, actualMirrorRefs)
+ })
+ }
+}
+
+func TestSuccessfulUpdateRemoteMirrorRequest(t *testing.T) {
+ testhelper.NewFeatureSets([]featureflag.FeatureFlag{
+ featureflag.GoUpdateRemoteMirror,
+ }).Run(t, testSuccessfulUpdateRemoteMirrorRequest)
+}
+
+func testSuccessfulUpdateRemoteMirrorRequest(t *testing.T, ctx context.Context) {
+ serverSocketPath, stop := RunRemoteServiceServer(t)
+ defer stop()
+
+ client, conn := NewRemoteClient(t, serverSocketPath)
+ defer conn.Close()
testRepo, testRepoPath, cleanupFn := gittest.CloneRepo(t)
defer cleanupFn()
@@ -106,15 +540,18 @@ func TestSuccessfulUpdateRemoteMirrorRequest(t *testing.T) {
}
func TestSuccessfulUpdateRemoteMirrorRequestWithWildcards(t *testing.T) {
+ testhelper.NewFeatureSets([]featureflag.FeatureFlag{
+ featureflag.GoUpdateRemoteMirror,
+ }).Run(t, testSuccessfulUpdateRemoteMirrorRequestWithWildcards)
+}
+
+func testSuccessfulUpdateRemoteMirrorRequestWithWildcards(t *testing.T, ctx context.Context) {
serverSocketPath, stop := RunRemoteServiceServer(t)
defer stop()
client, conn := NewRemoteClient(t, serverSocketPath)
defer conn.Close()
- ctx, cancel := testhelper.Context()
- defer cancel()
-
testRepo, testRepoPath, cleanupFn := gittest.CloneRepo(t)
defer cleanupFn()
@@ -184,15 +621,18 @@ func TestSuccessfulUpdateRemoteMirrorRequestWithWildcards(t *testing.T) {
}
func TestSuccessfulUpdateRemoteMirrorRequestWithKeepDivergentRefs(t *testing.T) {
+ testhelper.NewFeatureSets([]featureflag.FeatureFlag{
+ featureflag.GoUpdateRemoteMirror,
+ }).Run(t, testSuccessfulUpdateRemoteMirrorRequestWithKeepDivergentRefs)
+}
+
+func testSuccessfulUpdateRemoteMirrorRequestWithKeepDivergentRefs(t *testing.T, ctx context.Context) {
serverSocketPath, stop := RunRemoteServiceServer(t)
defer stop()
client, conn := NewRemoteClient(t, serverSocketPath)
defer conn.Close()
- ctx, cancel := testhelper.Context()
- defer cancel()
-
testRepo, testRepoPath, cleanupFn := gittest.CloneRepo(t)
defer cleanupFn()
@@ -264,6 +704,12 @@ func TestSuccessfulUpdateRemoteMirrorRequestWithKeepDivergentRefs(t *testing.T)
}
func TestFailedUpdateRemoteMirrorRequestDueToValidation(t *testing.T) {
+ testhelper.NewFeatureSets([]featureflag.FeatureFlag{
+ featureflag.GoUpdateRemoteMirror,
+ }).Run(t, testFailedUpdateRemoteMirrorRequestDueToValidation)
+}
+
+func testFailedUpdateRemoteMirrorRequestDueToValidation(t *testing.T, ctx context.Context) {
serverSocketPath, stop := RunRemoteServiceServer(t)
defer stop()
@@ -295,9 +741,6 @@ func TestFailedUpdateRemoteMirrorRequestDueToValidation(t *testing.T) {
for _, tc := range testCases {
t.Run(tc.desc, func(t *testing.T) {
- ctx, cancel := testhelper.Context()
- defer cancel()
-
stream, err := client.UpdateRemoteMirror(ctx)
require.NoError(t, err)
require.NoError(t, stream.Send(tc.request))
diff --git a/internal/metadata/featureflag/feature_flags.go b/internal/metadata/featureflag/feature_flags.go
index 61771fd24..80edee9ff 100644
--- a/internal/metadata/featureflag/feature_flags.go
+++ b/internal/metadata/featureflag/feature_flags.go
@@ -38,6 +38,8 @@ var (
UploadPackGitalyHooks = FeatureFlag{Name: "upload_pack_gitaly_hooks", OnByDefault: false}
// LFSPointersUseBitmapIndex enables the use of bitmap indices when searching LFS pointers.
LFSPointersUseBitmapIndex = FeatureFlag{Name: "lfs_pointers_use_bitmap_index", OnByDefault: false}
+ // GoUpdateRemoteMirror enables the Go implementation of UpdateRemoteMirror
+ GoUpdateRemoteMirror = FeatureFlag{Name: "go_update_remote_mirror", OnByDefault: false}
// TxApplyBfgObjectMapStream enables transactions for ApplyBfgObjectMapStream
TxApplyBfgObjectMapStream = FeatureFlag{Name: "tx_apply_bfg_object_map_stream", OnByDefault: true}
@@ -120,6 +122,7 @@ var All = []FeatureFlag{
GoGetLFSPointers,
GoGetNewLFSPointers,
LFSPointersUseBitmapIndex,
+ GoUpdateRemoteMirror,
TxApplyBfgObjectMapStream,
TxApplyGitattributes,
TxResolveConflicts,