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:
authorNick Thomas <nick@gitlab.com>2020-11-03 17:54:12 +0300
committerNick Thomas <nick@gitlab.com>2020-12-07 21:04:14 +0300
commit1b6e7628d32bd5bf135efa97040f6ca671450fa6 (patch)
tree4d2a6a747c817d4fc997ff8ee86ba85557364f8f
parent08dc6b04bd1fe8a9c96432f6993453660a63e277 (diff)
Implement RepositoryService.FetchRemoteWithStatusfetch-remote-with-status
This RPC is identical to FetchRemote, but returns the list of updated references in a response stream. Useful if you want to know what was updated, and what was left alone, by the RPC.
-rw-r--r--internal/git/fetch_scanner.go159
-rw-r--r--internal/git/fetch_scanner_test.go81
-rw-r--r--internal/git/repository.go8
-rw-r--r--internal/gitaly/service/repository/fetch_remote.go114
-rw-r--r--internal/gitaly/service/repository/fetch_remote_test.go105
-rw-r--r--internal/praefect/coordinator.go1
-rw-r--r--internal/praefect/protoregistry/protoregistry_test.go1
7 files changed, 462 insertions, 7 deletions
diff --git a/internal/git/fetch_scanner.go b/internal/git/fetch_scanner.go
new file mode 100644
index 000000000..c3bfd980f
--- /dev/null
+++ b/internal/git/fetch_scanner.go
@@ -0,0 +1,159 @@
+package git
+
+import (
+ "bufio"
+ "bytes"
+ "io"
+ "strings"
+)
+
+type RefUpdateType byte
+
+// type FetchStatusLine represents a line of status output from `git fetch`, as
+// documented here: https://git-scm.com/docs/git-fetch/2.11.4#_output
+//
+// Note that the content of the `from` and `to` lines may be affected by the
+// refspecs given to the `git fetch` command
+type FetchStatusLine struct {
+ Type RefUpdateType
+ Summary string
+ From string
+ To string
+ Reason string
+}
+
+const (
+ RefUpdateTypeFastForwardUpdate RefUpdateType = ' '
+ RefUpdateTypeForcedUpdate RefUpdateType = '+'
+ RefUpdateTypePruned RefUpdateType = '-'
+ RefUpdateTypeTagUpdate RefUpdateType = 't'
+ RefUpdateTypeFetched RefUpdateType = '*'
+ RefUpdateTypeUpdateFailed RefUpdateType = '!'
+ RefUpdateTypeUnchanged RefUpdateType = '='
+)
+
+var (
+ validRefUpdateTypes = []RefUpdateType{
+ RefUpdateTypeFastForwardUpdate,
+ RefUpdateTypeForcedUpdate,
+ RefUpdateTypePruned,
+ RefUpdateTypeTagUpdate,
+ RefUpdateTypeFetched,
+ RefUpdateTypeUpdateFailed,
+ RefUpdateTypeUnchanged,
+ }
+)
+
+func (t RefUpdateType) Valid() bool {
+ for _, cmp := range validRefUpdateTypes {
+ if t == cmp {
+ return true
+ }
+ }
+
+ return false
+}
+
+// type FetchScanner scans the output of `git fetch`, allowing information about
+// the updated refs to be gathered
+type FetchScanner struct {
+ scanner *bufio.Scanner
+ lastErr error
+ lastLine FetchStatusLine
+}
+
+func NewFetchScanner(r io.Reader) *FetchScanner {
+ return &FetchScanner{scanner: bufio.NewScanner(r)}
+}
+
+func (f *FetchScanner) Scan() bool {
+ if f.lastErr != nil {
+ return false
+ }
+
+ for f.scanner.Scan() {
+ // Silently ignore non-matching lines
+ line, ok := parseFetchStatusLine(f.scanner.Bytes())
+ if !ok {
+ continue
+ }
+
+ f.lastLine = line
+ return true
+ }
+
+ f.lastErr = f.scanner.Err()
+ return false
+}
+
+func (f *FetchScanner) Err() error {
+ return f.lastErr
+}
+
+func (f *FetchScanner) StatusLine() FetchStatusLine {
+ return f.lastLine
+}
+
+// line has this format: " <flag> <summary> <from> -> <to> [<reason>]"
+func parseFetchStatusLine(line []byte) (FetchStatusLine, bool) {
+ var blank FetchStatusLine
+ var out FetchStatusLine
+
+ // Handle the flag very strictly, since status and non-status text mingle
+ if len(line) < 4 || line[0] != ' ' || line[2] != ' ' {
+ return blank, false
+ }
+
+ out.Type, line = RefUpdateType(line[1]), line[3:]
+ if !out.Type.Valid() {
+ return blank, false
+ }
+
+ // Get the summary, which may be composed of multiple words
+ if line[0] == '[' {
+ end := bytes.IndexByte(line, ']')
+ if end < 0 || len(line) <= end+2 {
+ return blank, false
+ }
+
+ out.Summary, line = string(line[0:end+1]), line[end+1:]
+ } else {
+ end := bytes.IndexByte(line, ' ')
+ if end < 0 || len(line) <= end+1 {
+ return blank, false
+ }
+
+ out.Summary, line = string(line[0:end]), line[end:]
+ }
+
+ // Now we can scan by word for a bit
+ scanner := bufio.NewScanner(bytes.NewReader(line))
+ scanner.Split(bufio.ScanWords)
+
+ // From field
+ if !scanner.Scan() {
+ return blank, false
+ }
+ out.From = scanner.Text()
+
+ // Hardcoded -> delimeter
+ if !scanner.Scan() || !bytes.Equal(scanner.Bytes(), []byte("->")) {
+ return blank, false
+ }
+
+ // To field
+ if !scanner.Scan() {
+ return blank, false
+ }
+ out.To = scanner.Text()
+
+ // Reason field - optional, the rest of the line. This implementation will
+ // squeeze multiple spaces into one, but that shouldn't be a big problem
+ var reason []string
+ for scanner.Scan() {
+ reason = append(reason, scanner.Text())
+ }
+ out.Reason = strings.Join(reason, " ")
+
+ return out, true
+}
diff --git a/internal/git/fetch_scanner_test.go b/internal/git/fetch_scanner_test.go
new file mode 100644
index 000000000..e1bc15e34
--- /dev/null
+++ b/internal/git/fetch_scanner_test.go
@@ -0,0 +1,81 @@
+package git
+
+import (
+ "strconv"
+ "strings"
+ "testing"
+
+ "github.com/stretchr/testify/require"
+)
+
+func TestFetchScannerScan(t *testing.T) {
+ blank := FetchStatusLine{}
+
+ for i, tc := range []struct {
+ data string
+ expected FetchStatusLine
+ success bool
+ }{
+ {"", blank, false},
+ {" ", blank, false},
+ {"****", blank, false},
+ {"* [new branch] foo -> upstream/foo", blank, false},
+ {" * [new branch] foo -> upstream/foo", blank, false},
+ {" * [new branch foo -> upstream/foo", blank, false},
+ {" * new branch foo -> upstream/foo", blank, false},
+ {" * [new branch] foo upstream/foo", blank, false},
+ {" * [new branch] foo upstream/foo (some reason)", blank, false},
+ {
+ " * [new branch] foo -> upstream/foo",
+ FetchStatusLine{RefUpdateTypeFetched, "[new branch]", "foo", "upstream/foo", ""},
+ true,
+ },
+ {
+ " * [new branch] 面 -> upstream/面",
+ FetchStatusLine{RefUpdateTypeFetched, "[new branch]", "面", "upstream/面", ""},
+ true,
+ },
+ {
+ " + d8b96a36c...d2a598d09 cgroups-impl -> upstream/cgroups-impl (forced update)",
+ FetchStatusLine{RefUpdateTypeForcedUpdate, "d8b96a36c...d2a598d09", "cgroups-impl", "upstream/cgroups-impl", "(forced update)"},
+ true,
+ },
+ {
+ " * [new tag] v13.7.0-rc1 -> v13.7.0-rc1",
+ FetchStatusLine{RefUpdateTypeFetched, "[new tag]", "v13.7.0-rc1", "v13.7.0-rc1", ""},
+ true,
+ },
+ {
+ " 87daf9d2e..1504b30e1 master -> upstream/master",
+ FetchStatusLine{RefUpdateTypeFastForwardUpdate, "87daf9d2e..1504b30e1", "master", "upstream/master", ""},
+ true,
+ },
+ {
+ " - [deleted] (none) -> upstream/foo",
+ FetchStatusLine{RefUpdateTypePruned, "[deleted]", "(none)", "upstream/foo", ""},
+ true,
+ },
+ {
+ " t d8b96a36c...d2a598d09 v1.2.3 -> v1.2.3",
+ FetchStatusLine{RefUpdateTypeTagUpdate, "d8b96a36c...d2a598d09", "v1.2.3", "v1.2.3", ""},
+ true,
+ },
+ {
+ " ! d8b96a36c...d2a598d09 foo -> upstream/foo (update hook failed)",
+ FetchStatusLine{RefUpdateTypeUpdateFailed, "d8b96a36c...d2a598d09", "foo", "upstream/foo", "(update hook failed)"},
+ true,
+ },
+ {
+ " = [up to date] foo -> upstream/foo",
+ FetchStatusLine{RefUpdateTypeUnchanged, "[up to date]", "foo", "upstream/foo", ""},
+ true,
+ },
+ } {
+ t.Run(strconv.Itoa(i), func(t *testing.T) {
+ // Regular run
+ scanner := NewFetchScanner(strings.NewReader(tc.data))
+ require.Equal(t, tc.success, scanner.Scan())
+ require.Equal(t, tc.expected, scanner.StatusLine())
+ })
+ }
+}
diff --git a/internal/git/repository.go b/internal/git/repository.go
index e72f2e69d..b9bcd360f 100644
--- a/internal/git/repository.go
+++ b/internal/git/repository.go
@@ -64,6 +64,10 @@ type FetchOpts struct {
// doesn't have the previous commit as an ancestor.
// https://git-scm.com/docs/git-fetch#Documentation/git-fetch.txt---force
Force bool
+ // Verbose controls how much information is written to stderr. The list of
+ // refs updated by the fetch will only be listed if verbose is true.
+ // https://git-scm.com/docs/git-fetch#Documentation/git-fetch.txt---verbose
+ Verbose bool
// Tags controls whether tags will be fetched as part of the remote or not.
// https://git-scm.com/docs/git-fetch#Documentation/git-fetch.txt---tags
// https://git-scm.com/docs/git-fetch#Documentation/git-fetch.txt---no-tags
@@ -83,6 +87,10 @@ func (opts FetchOpts) buildFlags() []Option {
flags = append(flags, Flag{Name: "--force"})
}
+ if opts.Verbose {
+ flags = append(flags, Flag{Name: "--verbose"})
+ }
+
if opts.Tags != FetchOptsTagsDefault {
flags = append(flags, Flag{Name: opts.Tags.String()})
}
diff --git a/internal/gitaly/service/repository/fetch_remote.go b/internal/gitaly/service/repository/fetch_remote.go
index 47f24072d..995fa518a 100644
--- a/internal/gitaly/service/repository/fetch_remote.go
+++ b/internal/gitaly/service/repository/fetch_remote.go
@@ -4,6 +4,7 @@ import (
"bytes"
"context"
"fmt"
+ "io"
"io/ioutil"
"net/url"
"os"
@@ -11,6 +12,7 @@ import (
"strings"
"time"
+ "github.com/golang/protobuf/proto"
"github.com/grpc-ecosystem/go-grpc-middleware/logging/logrus/ctxlogrus"
"github.com/prometheus/client_golang/prometheus"
"github.com/sirupsen/logrus"
@@ -19,6 +21,7 @@ import (
"gitlab.com/gitlab-org/gitaly/internal/git"
"gitlab.com/gitlab-org/gitaly/internal/gitaly/rubyserver"
"gitlab.com/gitlab-org/gitaly/internal/helper"
+ "gitlab.com/gitlab-org/gitaly/internal/helper/chunk"
"gitlab.com/gitlab-org/gitaly/internal/metadata/featureflag"
"gitlab.com/gitlab-org/gitaly/proto/go/gitalypb"
"google.golang.org/grpc/status"
@@ -38,8 +41,55 @@ func init() {
prometheus.MustRegister(fetchRemoteImplCounter)
}
+func (s *server) FetchRemoteWithStatus(in *gitalypb.FetchRemoteRequest, stream gitalypb.RepositoryService_FetchRemoteWithStatusServer) error {
+ ctx, cancel := context.WithCancel(stream.Context())
+ defer cancel()
+
+ sender := chunk.New(&fetchRemoteWithStatusSender{stream: stream})
+ _, err := s.fetchRemote(ctx, in, func(line git.FetchStatusLine) {
+ updateType, ok := convertRefUpdateType(line.Type)
+ if !ok {
+ // FIXME: log an error here
+ cancel()
+ return
+ }
+
+ // Only report refs that have changed for now
+ if updateType == gitalypb.FetchRemoteWithStatusResponse_UNCHANGED {
+ return
+ }
+
+ update := &gitalypb.FetchRemoteWithStatusResponse_Update{
+ UpdateType: updateType,
+ Summary: line.Summary,
+ FromRef: line.From,
+ ToRef: line.To,
+ Reason: line.Reason,
+ }
+
+ if err := sender.Send(&gitalypb.FetchRemoteWithStatusResponse{RefUpdates: []*gitalypb.FetchRemoteWithStatusResponse_Update{update}}); err != nil {
+ // FIXME: log an error here
+ cancel()
+ }
+ })
+
+ if err != nil {
+ return err
+ }
+
+ return sender.Flush()
+}
+
func (s *server) FetchRemote(ctx context.Context, req *gitalypb.FetchRemoteRequest) (*gitalypb.FetchRemoteResponse, error) {
+ return s.fetchRemote(ctx, req, nil)
+}
+
+func (s *server) fetchRemote(ctx context.Context, req *gitalypb.FetchRemoteRequest, refUpdatesFunc func(git.FetchStatusLine)) (*gitalypb.FetchRemoteResponse, error) {
if featureflag.IsDisabled(ctx, featureflag.GoFetchRemote) {
+ if refUpdatesFunc != nil {
+ return nil, helper.Unimplemented
+ }
+
fetchRemoteImplCounter.WithLabelValues("ruby").Inc()
client, err := s.ruby.RepositoryServiceClient(ctx)
@@ -62,7 +112,33 @@ func (s *server) FetchRemote(ctx context.Context, req *gitalypb.FetchRemoteReque
}
var stderr bytes.Buffer
- opts := git.FetchOpts{Stderr: &stderr, Force: req.Force, Prune: true, Tags: git.FetchOptsTagsAll}
+ opts := git.FetchOpts{
+ Stderr: &stderr,
+ Force: req.Force,
+ Prune: true,
+ Tags: git.FetchOptsTagsAll,
+ }
+
+ if refUpdatesFunc != nil {
+ pr, pw := io.Pipe()
+
+ opts.Stderr = io.MultiWriter(&stderr, pw)
+ opts.Verbose = true
+
+ go func() {
+ defer pr.Close()
+ defer pw.Close()
+
+ scanner := git.NewFetchScanner(pr)
+ for scanner.Scan() {
+ refUpdatesFunc(scanner.StatusLine())
+ }
+
+ if err := scanner.Err(); err != nil {
+ logrus.Warnf("Scanner failed: %v", err)
+ }
+ }()
+ }
if req.GetNoTags() {
opts.Tags = git.FetchOptsTagsNone
@@ -297,3 +373,39 @@ func (s *server) configureSSH(ctx context.Context, sshKey, knownHosts string) (s
return "GIT_SSH_COMMAND=ssh " + strings.Join(conf, " "), cleanup, nil
}
+
+type fetchRemoteWithStatusSender struct {
+ stream gitalypb.RepositoryService_FetchRemoteWithStatusServer
+ response *gitalypb.FetchRemoteWithStatusResponse
+}
+
+func (s *fetchRemoteWithStatusSender) Reset() { s.response = &gitalypb.FetchRemoteWithStatusResponse{} }
+func (s *fetchRemoteWithStatusSender) Send() error { return s.stream.Send(s.response) }
+func (s *fetchRemoteWithStatusSender) Append(m proto.Message) {
+ s.response.RefUpdates = append(s.response.RefUpdates, m.(*gitalypb.FetchRemoteWithStatusResponse).RefUpdates...)
+}
+
+func convertRefUpdateType(t git.RefUpdateType) (gitalypb.FetchRemoteWithStatusResponse_UpdateType, bool) {
+ var updateType gitalypb.FetchRemoteWithStatusResponse_UpdateType
+
+ switch t {
+ case git.RefUpdateTypeFastForwardUpdate:
+ updateType = gitalypb.FetchRemoteWithStatusResponse_FAST_FORWARD_UPDATE
+ case git.RefUpdateTypeForcedUpdate:
+ updateType = gitalypb.FetchRemoteWithStatusResponse_FORCED_UPDATE
+ case git.RefUpdateTypePruned:
+ updateType = gitalypb.FetchRemoteWithStatusResponse_PRUNED
+ case git.RefUpdateTypeTagUpdate:
+ updateType = gitalypb.FetchRemoteWithStatusResponse_TAG_UPDATE
+ case git.RefUpdateTypeFetched:
+ updateType = gitalypb.FetchRemoteWithStatusResponse_FETCHED
+ case git.RefUpdateTypeUpdateFailed:
+ updateType = gitalypb.FetchRemoteWithStatusResponse_UPDATE_FAILED
+ case git.RefUpdateTypeUnchanged:
+ updateType = gitalypb.FetchRemoteWithStatusResponse_UNCHANGED
+ default:
+ return updateType, false
+ }
+
+ return updateType, true
+}
diff --git a/internal/gitaly/service/repository/fetch_remote_test.go b/internal/gitaly/service/repository/fetch_remote_test.go
index 586bd6e72..3d5f9f420 100644
--- a/internal/gitaly/service/repository/fetch_remote_test.go
+++ b/internal/gitaly/service/repository/fetch_remote_test.go
@@ -3,6 +3,7 @@ package repository
import (
"context"
"fmt"
+ "io"
"io/ioutil"
"net/http"
"net/http/httptest"
@@ -14,6 +15,7 @@ import (
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
+ "gitlab.com/gitlab-org/gitaly/internal/git"
"gitlab.com/gitlab-org/gitaly/internal/gitaly/config"
"gitlab.com/gitlab-org/gitaly/internal/helper/text"
"gitlab.com/gitlab-org/gitaly/internal/metadata/featureflag"
@@ -24,7 +26,7 @@ import (
"google.golang.org/grpc/metadata"
)
-func copyRepoWithNewRemote(t *testing.T, repo *gitalypb.Repository, locator storage.Locator, remote string) *gitalypb.Repository {
+func copyRepoWithNewRemote(t *testing.T, repo *gitalypb.Repository, locator storage.Locator, remote string) (*gitalypb.Repository, string) {
repoPath, err := locator.GetRepoPath(repo)
require.NoError(t, err)
@@ -37,7 +39,7 @@ func copyRepoWithNewRemote(t *testing.T, repo *gitalypb.Repository, locator stor
testhelper.MustRunCommand(t, nil, "git", "-C", clonePath, "remote", "add", remote, repoPath)
- return cloneRepo
+ return cloneRepo, clonePath
}
func TestFetchRemoteSuccess(t *testing.T) {
@@ -54,11 +56,9 @@ func TestFetchRemoteSuccess(t *testing.T) {
testRepo, _, cleanupFn := testhelper.NewTestRepo(t)
defer cleanupFn()
- cloneRepo := copyRepoWithNewRemote(t, testRepo, locator, "my-remote")
+ cloneRepo, cloneRepoPath := copyRepoWithNewRemote(t, testRepo, locator, "my-remote")
defer func() {
- path, err := locator.GetRepoPath(cloneRepo)
- require.NoError(t, err)
- require.NoError(t, os.RemoveAll(path))
+ require.NoError(t, os.RemoveAll(cloneRepoPath))
}()
resp, err := client.FetchRemote(ctx, &gitalypb.FetchRemoteRequest{
@@ -71,6 +71,99 @@ func TestFetchRemoteSuccess(t *testing.T) {
})
}
+func TestFetchRemoteWithStatusSuccess(t *testing.T) {
+ locator := config.NewLocator(config.Config)
+ serverSocketPath, stop := runRepoServer(t, locator, testhelper.WithInternalSocket(config.Config))
+ defer stop()
+
+ client, conn := newRepositoryClient(t, serverSocketPath)
+ defer conn.Close()
+
+ testRepo, testRepoPath, cleanupFn := testhelper.NewTestRepo(t)
+ defer cleanupFn()
+
+ cloneRepo, cloneRepoPath := copyRepoWithNewRemote(t, testRepo, locator, "my-remote")
+ defer func() {
+ require.NoError(t, os.RemoveAll(cloneRepoPath))
+ }()
+
+ oldMasterSHA := "1e292f8"
+
+ // Set up state for refs that will be overwritten
+ testhelper.CreateTag(t, cloneRepoPath, "modified-tag", "master", nil)
+ testhelper.CreateRemoteBranch(t, cloneRepoPath, "my-remote", "master", "master")
+
+ // Make some updates to the source repo that we want to be pulled
+ newMasterSHA := testhelper.CreateCommit(t, testRepoPath, "master", nil)
+ newBranchSHA, newBranchName := testhelper.CreateCommitOnNewBranch(t, testRepoPath)
+ newTagSHA := testhelper.CreateTag(t, testRepoPath, "new-tag", newBranchSHA, nil)
+ modifiedTagSHA := testhelper.CreateTag(t, testRepoPath, "modified-tag", newBranchSHA, nil)
+
+ stream, err := client.FetchRemoteWithStatus(context.Background(), &gitalypb.FetchRemoteRequest{
+ Repository: cloneRepo,
+ Force: true,
+ Remote: "my-remote",
+ Timeout: 120,
+ })
+ require.NoError(t, err)
+
+ // Collect all status updates
+ var updates []*gitalypb.FetchRemoteWithStatusResponse_Update
+ for {
+ rsp, err := stream.Recv()
+ if rsp != nil {
+ updates = append(updates, rsp.GetRefUpdates()...)
+ }
+
+ if err == io.EOF {
+ break
+ }
+
+ require.NoError(t, err)
+ }
+
+ gitCloneRepo := git.NewRepository(cloneRepo)
+
+ ctx, cancel := context.WithCancel(context.Background())
+ defer cancel()
+
+ newTagRef, err := gitCloneRepo.GetReference(ctx, "refs/tags/new-tag")
+ require.NoError(t, err, "New tag wasn't mirrored")
+ assert.Equal(t, newTagSHA, newTagRef.Target, "New tag has the wrong SHA")
+ assert.Contains(t, updates, refUpdateStruct('*', "[new tag]", "new-tag", "new-tag", ""))
+
+ modifiedTagRef, err := gitCloneRepo.GetReference(ctx, "refs/tags/modified-tag")
+ require.NoError(t, err, "Modified tag wasn't mirrored")
+ assert.Equal(t, modifiedTagSHA, modifiedTagRef.Target, "Modified tag has the wrong SHA")
+ assert.Contains(t, updates, refUpdateStruct('t', "[tag update]", "modified-tag", "modified-tag", ""))
+
+ masterRef, err := gitCloneRepo.GetReference(ctx, "refs/remotes/my-remote/master")
+ require.NoError(t, err, "Update to master wasn't mirrored")
+ assert.Equal(t, newMasterSHA, masterRef.Target)
+ assert.Contains(t, updates, refUpdateStruct('+', shortRevRange(oldMasterSHA, newMasterSHA), "master", "my-remote/master", "(forced update)"))
+
+ newBranchRef, err := gitCloneRepo.GetReference(ctx, "refs/remotes/my-remote/"+newBranchName)
+ require.NoError(t, err, "New branch "+newBranchName+" wasn't mirrored")
+ assert.Equal(t, newBranchSHA, newBranchRef.Target)
+ assert.Contains(t, updates, refUpdateStruct('*', "[new branch]", newBranchName, "my-remote/"+newBranchName, ""))
+}
+
+func refUpdateStruct(updateType byte, summary, from, to, reason string) *gitalypb.FetchRemoteWithStatusResponse_Update {
+ realUpdateType, _ := convertRefUpdateType(git.RefUpdateType(updateType))
+
+ return &gitalypb.FetchRemoteWithStatusResponse_Update{
+ UpdateType: realUpdateType,
+ Summary: summary,
+ FromRef: from,
+ ToRef: to,
+ Reason: reason,
+ }
+}
+
+func shortRevRange(from, to string) string {
+ return fmt.Sprintf("%s...%s", from[0:7], to[0:7])
+}
+
func TestFetchRemoteFailure(t *testing.T) {
repo, _, cleanup := testhelper.NewTestRepo(t)
defer cleanup()
diff --git a/internal/praefect/coordinator.go b/internal/praefect/coordinator.go
index ad549f75e..3884b144d 100644
--- a/internal/praefect/coordinator.go
+++ b/internal/praefect/coordinator.go
@@ -68,6 +68,7 @@ var transactionRPCs = map[string]transactionsCondition{
"/gitaly.RepositoryService/CreateRepositoryFromSnapshot": transactionsFlag(featureflag.TxCreateRepositoryFromSnapshot),
"/gitaly.RepositoryService/CreateRepositoryFromURL": transactionsFlag(featureflag.TxCreateRepositoryFromURL),
"/gitaly.RepositoryService/FetchRemote": transactionsFlag(featureflag.TxFetchRemote),
+ "/gitaly.RepositoryService/FetchRemoteWithStatus": transactionsFlag(featureflag.TxFetchRemote),
"/gitaly.RepositoryService/FetchSourceBranch": transactionsFlag(featureflag.TxFetchSourceBranch),
"/gitaly.RepositoryService/ReplicateRepository": transactionsFlag(featureflag.TxReplicateRepository),
"/gitaly.RepositoryService/WriteRef": transactionsFlag(featureflag.TxWriteRef),
diff --git a/internal/praefect/protoregistry/protoregistry_test.go b/internal/praefect/protoregistry/protoregistry_test.go
index 795ab3df9..52e0e33cb 100644
--- a/internal/praefect/protoregistry/protoregistry_test.go
+++ b/internal/praefect/protoregistry/protoregistry_test.go
@@ -122,6 +122,7 @@ func TestNewProtoRegistry(t *testing.T) {
"RepositorySize": protoregistry.OpAccessor,
"ApplyGitattributes": protoregistry.OpMutator,
"FetchRemote": protoregistry.OpMutator,
+ "FetchRemoteWithStatus": protoregistry.OpMutator,
"CreateRepository": protoregistry.OpMutator,
"GetArchive": protoregistry.OpAccessor,
"HasLocalBranches": protoregistry.OpAccessor,