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-22 14:55:17 +0300
committerSami Hiltunen <shiltunen@gitlab.com>2021-03-22 14:55:17 +0300
commite87f9229084c1bc149bead3bd986f6da2337ee85 (patch)
tree6ea7055d105786eb896ec78758fec7f89d0763f9
parentc5b7288a3f9bd5e6ce16303b21d325f0c945db97 (diff)
parent9d79c7a12d0d80f61dd94fa64ac6bb3e675d2d99 (diff)
Merge branch 'smh-port-update-remote-mirror' into 'master'
Port UpdateRemoteMirror to Go Closes #3316 See merge request gitlab-org/gitaly!3254
-rw-r--r--internal/git/localrepo/remote.go35
-rw-r--r--internal/git/localrepo/remote_test.go127
-rw-r--r--internal/git/ssh.go55
-rw-r--r--internal/git/ssh_test.go85
-rw-r--r--internal/git/subcommand.go3
-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/gitaly/service/repository/fetch_remote.go51
-rw-r--r--internal/metadata/featureflag/feature_flags.go3
-rw-r--r--proto/go/gitalypb/remote.pb.go42
-rw-r--r--proto/remote.proto19
-rw-r--r--ruby/proto/gitaly/remote_services_pb.rb5
12 files changed, 1016 insertions, 70 deletions
diff --git a/internal/git/localrepo/remote.go b/internal/git/localrepo/remote.go
index 3ccadfe87..ac3f4b39b 100644
--- a/internal/git/localrepo/remote.go
+++ b/internal/git/localrepo/remote.go
@@ -4,6 +4,7 @@ import (
"bufio"
"bytes"
"context"
+ "errors"
"fmt"
"io"
"strings"
@@ -272,3 +273,37 @@ func validateNotBlank(val, name string) error {
}
return nil
}
+
+// PushOptions are options that can be configured for a push.
+type PushOptions struct {
+ // SSHCommand is the command line to use for git's SSH invocation. The command line is used
+ // as is and must be verified by the caller to be safe.
+ SSHCommand string
+}
+
+// Push force pushes the refspecs to the remote.
+func (repo *Repo) Push(ctx context.Context, remote string, refspecs []string, options PushOptions) error {
+ if len(refspecs) == 0 {
+ return errors.New("refspecs to push must be explicitly specified")
+ }
+
+ var env []string
+ if options.SSHCommand != "" {
+ env = append(env, "GIT_SSH_COMMAND="+options.SSHCommand)
+ }
+
+ stderr := &bytes.Buffer{}
+ if err := repo.ExecAndWait(ctx,
+ git.SubCmd{
+ Name: "push",
+ Flags: []git.Option{git.Flag{Name: "--force"}},
+ Args: append([]string{remote}, refspecs...),
+ },
+ git.WithStderr(stderr),
+ git.WithEnv(env...),
+ ); err != nil {
+ return fmt.Errorf("git push: %w, stderr: %q", err, stderr)
+ }
+
+ return nil
+}
diff --git a/internal/git/localrepo/remote_test.go b/internal/git/localrepo/remote_test.go
index 453d0b045..adceefe2b 100644
--- a/internal/git/localrepo/remote_test.go
+++ b/internal/git/localrepo/remote_test.go
@@ -3,7 +3,9 @@ package localrepo
import (
"bytes"
"errors"
+ "fmt"
"io/ioutil"
+ "os"
"os/exec"
"path/filepath"
"testing"
@@ -434,3 +436,128 @@ func TestRepo_FetchRemote(t *testing.T) {
require.False(t, containsTags)
})
}
+
+func TestRepo_Push(t *testing.T) {
+ ctx, cancel := testhelper.Context()
+ defer cancel()
+
+ cfg, sourceRepoPb, _ := testcfg.BuildWithRepo(t)
+
+ tmpDir, clean := testhelper.TempDir(t)
+ defer clean()
+
+ gitPath := filepath.Join(tmpDir, "git-hook")
+ envPath := filepath.Join(tmpDir, "GIT_SSH_PATH")
+ require.NoError(t, ioutil.WriteFile(
+ gitPath,
+ []byte(fmt.Sprintf(
+ `#!/usr/bin/env bash
+if [ -z ${GIT_SSH_COMMAND+x} ];then rm -f %q ;else echo -n "$GIT_SSH_COMMAND" > %q; fi
+%s "$@"`,
+ envPath, envPath,
+ cfg.Git.BinPath,
+ )),
+ os.ModePerm),
+ )
+
+ cfg.Git.BinPath = gitPath
+ sourceRepo := New(git.NewExecCommandFactory(cfg), sourceRepoPb, cfg)
+
+ for _, tc := range []struct {
+ desc string
+ invalidRemote bool
+ sshCommand string
+ refspecs []string
+ setupPushRepo func(testing.TB, *Repo)
+ errorMessage string
+ expectedFilter []string
+ }{
+ {
+ desc: "refspecs must be specified",
+ errorMessage: "refspecs to push must be explicitly specified",
+ },
+ {
+ desc: "push two refs",
+ refspecs: []string{"refs/heads/master", "refs/heads/feature"},
+ expectedFilter: []string{"refs/heads/master", "refs/heads/feature"},
+ }, {
+ desc: "push with custom ssh command",
+ sshCommand: "custom --ssh-command",
+ refspecs: []string{"refs/heads/master"},
+ expectedFilter: []string{"refs/heads/master"},
+ },
+ {
+ desc: "force pushes over diverged refs",
+ refspecs: []string{"refs/heads/master"},
+ setupPushRepo: func(t testing.TB, repo *Repo) {
+ // set up master as a divergin ref in push repo
+ sourceMaster, err := sourceRepo.GetReference(ctx, "refs/heads/master")
+ require.NoError(t, err)
+
+ pushRepoPath, err := repo.Path()
+ require.NoError(t, err)
+
+ require.NoError(t, sourceRepo.Push(ctx, pushRepoPath, []string{"refs/*"}, PushOptions{}))
+ divergedMaster := gittest.CreateCommit(t, pushRepoPath, "master", &gittest.CreateCommitOpts{
+ ParentID: sourceMaster.Target,
+ })
+
+ master, err := repo.GetReference(ctx, "refs/heads/master")
+ require.NoError(t, err)
+
+ require.Equal(t, master.Target, divergedMaster)
+ },
+ },
+ {
+ desc: "push all refs",
+ refspecs: []string{"refs/*"},
+ },
+ {
+ desc: "push empty refspec",
+ refspecs: []string{""},
+ errorMessage: `git push: exit status 128, stderr: "fatal: invalid refspec ''\n"`,
+ },
+ {
+ desc: "invalid remote",
+ refspecs: []string{"refs/heads/master"},
+ invalidRemote: true,
+ errorMessage: `git push: exit status 128, stderr: "fatal: no path specified; see 'git help pull' for valid url syntax\n"`,
+ },
+ } {
+ t.Run(tc.desc, func(t *testing.T) {
+ pushRepoPb, pushRepoPath, _ := gittest.InitBareRepoAt(t, cfg.Storages[0])
+ pushRepo := New(git.NewExecCommandFactory(cfg), pushRepoPb, cfg)
+
+ if tc.setupPushRepo != nil {
+ tc.setupPushRepo(t, pushRepo)
+ }
+
+ remote := pushRepoPath
+ if tc.invalidRemote {
+ remote = ""
+ }
+
+ err := sourceRepo.Push(ctx, remote, tc.refspecs, PushOptions{SSHCommand: tc.sshCommand})
+ if tc.errorMessage != "" {
+ require.EqualError(t, err, tc.errorMessage)
+ return
+ }
+ require.NoError(t, err)
+
+ gitSSHCommand, err := ioutil.ReadFile(envPath)
+ if !os.IsNotExist(err) {
+ require.NoError(t, err)
+ }
+
+ require.Equal(t, tc.sshCommand, string(gitSSHCommand))
+
+ actual, err := pushRepo.GetReferences(ctx)
+ require.NoError(t, err)
+
+ expected, err := sourceRepo.GetReferences(ctx, tc.expectedFilter...)
+ require.NoError(t, err)
+
+ require.Equal(t, expected, actual)
+ })
+ }
+}
diff --git a/internal/git/ssh.go b/internal/git/ssh.go
new file mode 100644
index 000000000..c7612d819
--- /dev/null
+++ b/internal/git/ssh.go
@@ -0,0 +1,55 @@
+package git
+
+import (
+ "context"
+ "fmt"
+ "io/ioutil"
+ "os"
+ "path/filepath"
+ "strings"
+
+ "github.com/grpc-ecosystem/go-grpc-middleware/logging/logrus/ctxlogrus"
+)
+
+// BuildSSHInvocation builds a command line to invoke SSH with the provided key and known hosts.
+// Both are optional.
+func BuildSSHInvocation(ctx context.Context, sshKey, knownHosts string) (string, func(), error) {
+ const sshCommand = "ssh"
+ if sshKey == "" && knownHosts == "" {
+ return sshCommand, func() {}, nil
+ }
+
+ tmpDir, err := ioutil.TempDir("", "gitaly-ssh-invocation")
+ if err != nil {
+ return "", func() {}, fmt.Errorf("create temporary directory: %w", err)
+ }
+
+ cleanup := func() {
+ if err := os.RemoveAll(tmpDir); err != nil {
+ ctxlogrus.Extract(ctx).WithError(err).Error("failed to remove tmp directory with ssh key/config")
+ }
+ }
+
+ args := []string{sshCommand}
+ if sshKey != "" {
+ sshKeyFile := filepath.Join(tmpDir, "ssh-key")
+ if err := ioutil.WriteFile(sshKeyFile, []byte(sshKey), 0400); err != nil {
+ cleanup()
+ return "", nil, fmt.Errorf("create ssh key file: %w", err)
+ }
+
+ args = append(args, "-oIdentitiesOnly=yes", "-oIdentityFile="+sshKeyFile)
+ }
+
+ if knownHosts != "" {
+ knownHostsFile := filepath.Join(tmpDir, "known-hosts")
+ if err := ioutil.WriteFile(knownHostsFile, []byte(knownHosts), 0400); err != nil {
+ cleanup()
+ return "", nil, fmt.Errorf("create known hosts file: %w", err)
+ }
+
+ args = append(args, "-oStrictHostKeyChecking=yes", "-oUserKnownHostsFile="+knownHostsFile)
+ }
+
+ return strings.Join(args, " "), cleanup, nil
+}
diff --git a/internal/git/ssh_test.go b/internal/git/ssh_test.go
new file mode 100644
index 000000000..cf6111edc
--- /dev/null
+++ b/internal/git/ssh_test.go
@@ -0,0 +1,85 @@
+package git
+
+import (
+ "io/ioutil"
+ "path/filepath"
+ "regexp"
+ "testing"
+
+ "github.com/stretchr/testify/require"
+ "gitlab.com/gitlab-org/gitaly/internal/testhelper"
+)
+
+func TestBuildSSHInvocation(t *testing.T) {
+ ctx, cancel := testhelper.Context()
+ defer cancel()
+
+ const tmpDirPattern = "=(\\S+)/(ssh-key|known-hosts)"
+ reTmpDir := regexp.MustCompile(tmpDirPattern)
+
+ for _, tc := range []struct {
+ desc string
+ sshKey string
+ knownHosts string
+ }{
+ {
+ desc: "no arguments",
+ },
+ {
+ desc: "ssh key given",
+ sshKey: "ssh-key-content",
+ },
+ {
+ desc: "known hosts given",
+ knownHosts: "known-hosts-content",
+ },
+ {
+ desc: "both given",
+ sshKey: "ssh-key-content",
+ knownHosts: "known-hosts-content",
+ },
+ } {
+ t.Run(tc.desc, func(t *testing.T) {
+ sshCommand, clean, err := BuildSSHInvocation(ctx, tc.sshKey, tc.knownHosts)
+ require.NoError(t, err)
+ defer clean()
+
+ var tmpDir string
+ if tc.sshKey != "" || tc.knownHosts != "" {
+ matches := reTmpDir.FindStringSubmatch(sshCommand)
+ require.Greater(t, len(matches), 1, "expected to find at least one file configured")
+ tmpDir = matches[1]
+ require.DirExists(t, tmpDir)
+ } else {
+ require.False(t, reTmpDir.MatchString(sshCommand))
+ }
+
+ sshKeyPath := filepath.Join(tmpDir, "ssh-key")
+ knownHostsPath := filepath.Join(tmpDir, "known-hosts")
+
+ expectedCommand := "ssh"
+ if tc.sshKey != "" {
+ content, err := ioutil.ReadFile(sshKeyPath)
+ require.NoError(t, err)
+ require.Equal(t, tc.sshKey, string(content))
+ expectedCommand += " -oIdentitiesOnly=yes -oIdentityFile=" + sshKeyPath
+ } else {
+ require.NoFileExists(t, sshKeyPath)
+ }
+
+ if tc.knownHosts != "" {
+ content, err := ioutil.ReadFile(knownHostsPath)
+ require.NoError(t, err)
+ require.Equal(t, tc.knownHosts, string(content))
+ expectedCommand += " -oStrictHostKeyChecking=yes -oUserKnownHostsFile=" + knownHostsPath
+ } else {
+ require.NoFileExists(t, knownHostsPath)
+ }
+
+ require.Equal(t, expectedCommand, sshCommand)
+
+ clean()
+ require.NoDirExists(t, tmpDir)
+ })
+ }
+}
diff --git a/internal/git/subcommand.go b/internal/git/subcommand.go
index 473d9cd0e..d68e1a988 100644
--- a/internal/git/subcommand.go
+++ b/internal/git/subcommand.go
@@ -110,6 +110,9 @@ var gitCommands = map[string]gitCommand{
"pack-objects": gitCommand{
flags: scNoRefUpdates | scGeneratesPackfiles,
},
+ "push": gitCommand{
+ flags: scNoRefUpdates | scNoEndOfOptions,
+ },
"receive-pack": gitCommand{
flags: 0,
opts: []GlobalOption{
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/gitaly/service/repository/fetch_remote.go b/internal/gitaly/service/repository/fetch_remote.go
index 215f04344..5a66eedfc 100644
--- a/internal/gitaly/service/repository/fetch_remote.go
+++ b/internal/gitaly/service/repository/fetch_remote.go
@@ -5,10 +5,7 @@ import (
"context"
"fmt"
"io"
- "io/ioutil"
"net/url"
- "os"
- "path/filepath"
"strings"
"time"
@@ -134,13 +131,13 @@ func (s *server) FetchRemote(ctx context.Context, req *gitalypb.FetchRemoteReque
}()
}
} else {
- envGitSSHCommand, cleanup, err := s.configureSSH(ctx, req.GetSshKey(), req.GetKnownHosts())
+ sshCommand, cleanup, err := git.BuildSSHInvocation(ctx, req.GetSshKey(), req.GetKnownHosts())
if err != nil {
return nil, err
}
defer cleanup()
- opts.Env = append(opts.Env, envGitSSHCommand)
+ opts.Env = append(opts.Env, "GIT_SSH_COMMAND="+sshCommand)
}
if req.GetTimeout() > 0 {
@@ -265,47 +262,3 @@ func (s *server) removeRemote(ctx context.Context, repo *localrepo.Repo, name st
return nil
}
-
-func (s *server) configureSSH(ctx context.Context, sshKey, knownHosts string) (string, func(), error) {
- sshKeyPresent := strings.TrimSpace(sshKey) != ""
- knownHostsPresent := strings.TrimSpace(knownHosts) != ""
-
- if !sshKeyPresent && !knownHostsPresent {
- return "", func() {}, nil
- }
-
- tmpdir, err := ioutil.TempDir("", "")
- if err != nil {
- return "", nil, err
- }
-
- cleanup := func() {
- if err := os.RemoveAll(tmpdir); err != nil {
- ctxlogrus.Extract(ctx).WithError(err).Error("failed to remove tmp directory with ssh key/config")
- }
- }
-
- var conf []string
-
- if sshKeyPresent {
- identityFilePath := filepath.Join(tmpdir, "gitlab-shell-key-file")
-
- if err := ioutil.WriteFile(identityFilePath, []byte(sshKey), 0400); err != nil {
- cleanup()
- return "", nil, err
- }
- conf = append(conf, "-oIdentitiesOnly=yes", "-oIdentityFile="+identityFilePath)
- }
-
- if knownHostsPresent {
- hostsFilePath := filepath.Join(tmpdir, "gitlab-shell-known-hosts")
-
- if err := ioutil.WriteFile(hostsFilePath, []byte(knownHosts), 0400); err != nil {
- cleanup()
- return "", nil, err
- }
- conf = append(conf, "-oStrictHostKeyChecking=yes", "-oUserKnownHostsFile="+hostsFilePath)
- }
-
- return "GIT_SSH_COMMAND=ssh " + strings.Join(conf, " "), cleanup, nil
-}
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,
diff --git a/proto/go/gitalypb/remote.pb.go b/proto/go/gitalypb/remote.pb.go
index 761a6ab20..73e669b90 100644
--- a/proto/go/gitalypb/remote.pb.go
+++ b/proto/go/gitalypb/remote.pb.go
@@ -292,15 +292,27 @@ func (m *FetchInternalRemoteResponse) GetResult() bool {
}
type UpdateRemoteMirrorRequest struct {
- Repository *Repository `protobuf:"bytes,1,opt,name=repository,proto3" json:"repository,omitempty"`
- RefName string `protobuf:"bytes,2,opt,name=ref_name,json=refName,proto3" json:"ref_name,omitempty"`
- OnlyBranchesMatching [][]byte `protobuf:"bytes,3,rep,name=only_branches_matching,json=onlyBranchesMatching,proto3" json:"only_branches_matching,omitempty"`
- SshKey string `protobuf:"bytes,4,opt,name=ssh_key,json=sshKey,proto3" json:"ssh_key,omitempty"`
- KnownHosts string `protobuf:"bytes,5,opt,name=known_hosts,json=knownHosts,proto3" json:"known_hosts,omitempty"`
- KeepDivergentRefs bool `protobuf:"varint,6,opt,name=keep_divergent_refs,json=keepDivergentRefs,proto3" json:"keep_divergent_refs,omitempty"`
- XXX_NoUnkeyedLiteral struct{} `json:"-"`
- XXX_unrecognized []byte `json:"-"`
- XXX_sizecache int32 `json:"-"`
+ // repository is the repository whose mirror repository to update.
+ Repository *Repository `protobuf:"bytes,1,opt,name=repository,proto3" json:"repository,omitempty"`
+ // ref_name is actually the remote to update.
+ RefName string `protobuf:"bytes,2,opt,name=ref_name,json=refName,proto3" json:"ref_name,omitempty"`
+ // only_branches_matching contains patterns to match branches against. Only
+ // the matched brances are updated in the remote mirror. If no patterns are
+ // specified, all branches are updated. The patterns should only contain the
+ // branch name without the 'refs/heads/' prefix. "*" can be used as a wildcard
+ // to match anything. only_branches_matching can be streamed to the server over multiple
+ // messages. Optional.
+ OnlyBranchesMatching [][]byte `protobuf:"bytes,3,rep,name=only_branches_matching,json=onlyBranchesMatching,proto3" json:"only_branches_matching,omitempty"`
+ // ssh_key is the SSH key to use for accessing to the mirror repository. Optional.
+ SshKey string `protobuf:"bytes,4,opt,name=ssh_key,json=sshKey,proto3" json:"ssh_key,omitempty"`
+ // known_hosts specifies the identities used for strict host key checking. Optional.
+ KnownHosts string `protobuf:"bytes,5,opt,name=known_hosts,json=knownHosts,proto3" json:"known_hosts,omitempty"`
+ // keep_divergent_refs specifies whether or not to update diverged references in the
+ // mirror repository.
+ KeepDivergentRefs bool `protobuf:"varint,6,opt,name=keep_divergent_refs,json=keepDivergentRefs,proto3" json:"keep_divergent_refs,omitempty"`
+ XXX_NoUnkeyedLiteral struct{} `json:"-"`
+ XXX_unrecognized []byte `json:"-"`
+ XXX_sizecache int32 `json:"-"`
}
func (m *UpdateRemoteMirrorRequest) Reset() { *m = UpdateRemoteMirrorRequest{} }
@@ -371,6 +383,8 @@ func (m *UpdateRemoteMirrorRequest) GetKeepDivergentRefs() bool {
}
type UpdateRemoteMirrorResponse struct {
+ // divergent_refs contains a list of references that had diverged in the mirror from the
+ // source repository.
DivergentRefs [][]byte `protobuf:"bytes,1,rep,name=divergent_refs,json=divergentRefs,proto3" json:"divergent_refs,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
@@ -807,6 +821,11 @@ type RemoteServiceClient interface {
AddRemote(ctx context.Context, in *AddRemoteRequest, opts ...grpc.CallOption) (*AddRemoteResponse, error)
FetchInternalRemote(ctx context.Context, in *FetchInternalRemoteRequest, opts ...grpc.CallOption) (*FetchInternalRemoteResponse, error)
RemoveRemote(ctx context.Context, in *RemoveRemoteRequest, opts ...grpc.CallOption) (*RemoveRemoteResponse, error)
+ // UpdateRemoteMirror compares the references in the target repository and its remote mirror
+ // repository. Any differences in the references are then addressed by pushing the differing
+ // references to the mirror. Created and modified references are updated, removed references are
+ // deleted from the mirror. UpdateRemoteMirror updates all tags. Branches are updated if they match
+ // the patterns specified in the requests.
UpdateRemoteMirror(ctx context.Context, opts ...grpc.CallOption) (RemoteService_UpdateRemoteMirrorClient, error)
FindRemoteRepository(ctx context.Context, in *FindRemoteRepositoryRequest, opts ...grpc.CallOption) (*FindRemoteRepositoryResponse, error)
FindRemoteRootRef(ctx context.Context, in *FindRemoteRootRefRequest, opts ...grpc.CallOption) (*FindRemoteRootRefResponse, error)
@@ -904,6 +923,11 @@ type RemoteServiceServer interface {
AddRemote(context.Context, *AddRemoteRequest) (*AddRemoteResponse, error)
FetchInternalRemote(context.Context, *FetchInternalRemoteRequest) (*FetchInternalRemoteResponse, error)
RemoveRemote(context.Context, *RemoveRemoteRequest) (*RemoveRemoteResponse, error)
+ // UpdateRemoteMirror compares the references in the target repository and its remote mirror
+ // repository. Any differences in the references are then addressed by pushing the differing
+ // references to the mirror. Created and modified references are updated, removed references are
+ // deleted from the mirror. UpdateRemoteMirror updates all tags. Branches are updated if they match
+ // the patterns specified in the requests.
UpdateRemoteMirror(RemoteService_UpdateRemoteMirrorServer) error
FindRemoteRepository(context.Context, *FindRemoteRepositoryRequest) (*FindRemoteRepositoryResponse, error)
FindRemoteRootRef(context.Context, *FindRemoteRootRefRequest) (*FindRemoteRootRefResponse, error)
diff --git a/proto/remote.proto b/proto/remote.proto
index 849a7393d..e1608d234 100644
--- a/proto/remote.proto
+++ b/proto/remote.proto
@@ -24,6 +24,11 @@ service RemoteService {
};
}
+ // UpdateRemoteMirror compares the references in the target repository and its remote mirror
+ // repository. Any differences in the references are then addressed by pushing the differing
+ // references to the mirror. Created and modified references are updated, removed references are
+ // deleted from the mirror. UpdateRemoteMirror updates all tags. Branches are updated if they match
+ // the patterns specified in the requests.
rpc UpdateRemoteMirror(stream UpdateRemoteMirrorRequest) returns (UpdateRemoteMirrorResponse) {
option (op_type) = {
op: MUTATOR
@@ -74,15 +79,29 @@ message FetchInternalRemoteResponse {
}
message UpdateRemoteMirrorRequest {
+ // repository is the repository whose mirror repository to update.
Repository repository = 1 [(target_repository)=true];
+ // ref_name is actually the remote to update.
string ref_name = 2;
+ // only_branches_matching contains patterns to match branches against. Only
+ // the matched brances are updated in the remote mirror. If no patterns are
+ // specified, all branches are updated. The patterns should only contain the
+ // branch name without the 'refs/heads/' prefix. "*" can be used as a wildcard
+ // to match anything. only_branches_matching can be streamed to the server over multiple
+ // messages. Optional.
repeated bytes only_branches_matching = 3;
+ // ssh_key is the SSH key to use for accessing to the mirror repository. Optional.
string ssh_key = 4;
+ // known_hosts specifies the identities used for strict host key checking. Optional.
string known_hosts = 5;
+ // keep_divergent_refs specifies whether or not to update diverged references in the
+ // mirror repository.
bool keep_divergent_refs = 6;
}
message UpdateRemoteMirrorResponse {
+ // divergent_refs contains a list of references that had diverged in the mirror from the
+ // source repository.
repeated bytes divergent_refs = 1;
}
diff --git a/ruby/proto/gitaly/remote_services_pb.rb b/ruby/proto/gitaly/remote_services_pb.rb
index 013c45961..93959d0db 100644
--- a/ruby/proto/gitaly/remote_services_pb.rb
+++ b/ruby/proto/gitaly/remote_services_pb.rb
@@ -17,6 +17,11 @@ module Gitaly
rpc :AddRemote, Gitaly::AddRemoteRequest, Gitaly::AddRemoteResponse
rpc :FetchInternalRemote, Gitaly::FetchInternalRemoteRequest, Gitaly::FetchInternalRemoteResponse
rpc :RemoveRemote, Gitaly::RemoveRemoteRequest, Gitaly::RemoveRemoteResponse
+ # UpdateRemoteMirror compares the references in the target repository and its remote mirror
+ # repository. Any differences in the references are then addressed by pushing the differing
+ # references to the mirror. Created and modified references are updated, removed references are
+ # deleted from the mirror. UpdateRemoteMirror updates all tags. Branches are updated if they match
+ # the patterns specified in the requests.
rpc :UpdateRemoteMirror, stream(Gitaly::UpdateRemoteMirrorRequest), Gitaly::UpdateRemoteMirrorResponse
rpc :FindRemoteRepository, Gitaly::FindRemoteRepositoryRequest, Gitaly::FindRemoteRepositoryResponse
rpc :FindRemoteRootRef, Gitaly::FindRemoteRootRefRequest, Gitaly::FindRemoteRootRefResponse