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:
authorPavlo Strokov <pstrokov@gitlab.com>2020-07-22 10:35:16 +0300
committerPavlo Strokov <pstrokov@gitlab.com>2020-07-22 10:35:16 +0300
commitd758ff48fbf8392e08626b60685d037373347d72 (patch)
treee0b8515bfe931b7cd24b1ef52e78f4e6ffb27d35
parentf6960a449e57bd890d7400f09e35facc55b982ec (diff)
parent5a4956b35d88054afc62ee1897fc112f7a3d63a8 (diff)
Merge branch 'ps-rpc-scope-storage' into 'master'
Praefect: storage scoped RPCs not handled properly Closes #2442 See merge request gitlab-org/gitaly!2234
-rw-r--r--changelogs/unreleased/ps-rpc-scope-storage.yml5
-rw-r--r--internal/praefect/auth_test.go7
-rw-r--r--internal/praefect/coordinator.go121
-rw-r--r--internal/praefect/coordinator_test.go261
-rw-r--r--internal/praefect/protoregistry/find_oid.go10
-rw-r--r--internal/praefect/protoregistry/find_oid_test.go45
-rw-r--r--internal/praefect/protoregistry/protoregistry.go25
-rw-r--r--internal/service/remote/remotes_test.go4
-rw-r--r--proto/go/gitalypb/remote.pb.go102
-rw-r--r--proto/remote.proto4
10 files changed, 513 insertions, 71 deletions
diff --git a/changelogs/unreleased/ps-rpc-scope-storage.yml b/changelogs/unreleased/ps-rpc-scope-storage.yml
new file mode 100644
index 000000000..65f46f23b
--- /dev/null
+++ b/changelogs/unreleased/ps-rpc-scope-storage.yml
@@ -0,0 +1,5 @@
+---
+title: 'Praefect: storage scoped RPCs not handled properly'
+merge_request: 2234
+author:
+type: fixed
diff --git a/internal/praefect/auth_test.go b/internal/praefect/auth_test.go
index b112a716f..df57ea87b 100644
--- a/internal/praefect/auth_test.go
+++ b/internal/praefect/auth_test.go
@@ -18,6 +18,7 @@ import (
"gitlab.com/gitlab-org/gitaly/internal/praefect/transactions"
"gitlab.com/gitlab-org/gitaly/internal/testhelper"
"gitlab.com/gitlab-org/gitaly/internal/testhelper/promtest"
+ "gitlab.com/gitlab-org/gitaly/proto/go/gitalypb"
"google.golang.org/grpc"
"google.golang.org/grpc/codes"
)
@@ -111,11 +112,9 @@ func TestAuthSuccess(t *testing.T) {
require.NoError(t, err, tc.desc)
defer conn.Close()
- cli := mock.NewSimpleServiceClient(conn)
+ cli := gitalypb.NewServerServiceClient(conn)
- _, err = cli.ServerAccessor(ctx, &mock.SimpleRequest{
- Value: 1,
- })
+ _, err = cli.ServerInfo(ctx, &gitalypb.ServerInfoRequest{})
assert.NoError(t, err, tc.desc)
})
diff --git a/internal/praefect/coordinator.go b/internal/praefect/coordinator.go
index df00fe4a8..2e7ab294d 100644
--- a/internal/praefect/coordinator.go
+++ b/internal/praefect/coordinator.go
@@ -21,8 +21,6 @@ import (
"gitlab.com/gitlab-org/gitaly/proto/go/gitalypb"
"gitlab.com/gitlab-org/labkit/correlation"
"golang.org/x/sync/errgroup"
- "google.golang.org/grpc/codes"
- "google.golang.org/grpc/status"
)
type ReadOnlyStorageError string
@@ -368,22 +366,113 @@ func (c *Coordinator) StreamDirector(ctx context.Context, fullMethodName string,
return sp, nil
}
- // TODO: remove the need to handle non repository scoped RPCs. The only remaining one is FindRemoteRepository.
- // https://gitlab.com/gitlab-org/gitaly/issues/2442. One this issue is resolved, we can explicitly require that
- // any RPC that gets proxied through praefect must be repository scoped.
- shard, err := c.nodeMgr.GetShard(c.conf.VirtualStorages[0].Name)
+ if mi.Scope == protoregistry.ScopeStorage {
+ return c.directStorageScopedMessage(ctx, mi, m)
+ }
+
+ // TODO: please refer to https://gitlab.com/gitlab-org/gitaly/-/issues/2974
+ if mi.Scope == protoregistry.ScopeServer {
+ shard, err := c.nodeMgr.GetShard(c.conf.VirtualStorages[0].Name)
+ if err != nil {
+ if errors.Is(err, nodes.ErrVirtualStorageNotExist) {
+ return nil, helper.ErrInvalidArgument(err)
+ }
+ return nil, err
+ }
+
+ return proxy.NewStreamParameters(proxy.Destination{
+ Ctx: helper.IncomingToOutgoing(ctx),
+ Conn: shard.Primary.GetConnection(),
+ Msg: payload,
+ }, nil, func() error { return nil }, nil), nil
+ }
+
+ return nil, helper.ErrInternalf("rpc with undefined scope %q", mi.Scope)
+}
+
+func (c *Coordinator) directStorageScopedMessage(ctx context.Context, mi protoregistry.MethodInfo, msg proto.Message) (*proxy.StreamParameters, error) {
+ virtualStorage, err := mi.Storage(msg)
+ if err != nil {
+ return nil, helper.ErrInvalidArgument(err)
+ }
+
+ if virtualStorage == "" {
+ return nil, helper.ErrInvalidArgumentf("storage scoped: target storage is invalid")
+ }
+
+ var ps *proxy.StreamParameters
+ switch mi.Operation {
+ case protoregistry.OpAccessor:
+ ps, err = c.accessorStorageStreamParameters(ctx, mi, msg, virtualStorage)
+ case protoregistry.OpMutator:
+ ps, err = c.mutatorStorageStreamParameters(ctx, mi, msg, virtualStorage)
+ default:
+ err = fmt.Errorf("storage scope: unknown operation type: %v", mi.Operation)
+ }
+ return ps, err
+}
+
+func (c *Coordinator) accessorStorageStreamParameters(ctx context.Context, mi protoregistry.MethodInfo, msg proto.Message, virtualStorage string) (*proxy.StreamParameters, error) {
+ shard, err := c.nodeMgr.GetShard(virtualStorage)
if err != nil {
if errors.Is(err, nodes.ErrVirtualStorageNotExist) {
- return nil, status.Errorf(codes.InvalidArgument, err.Error())
+ return nil, helper.ErrInvalidArgument(err)
}
- return nil, err
+ return nil, helper.ErrInternalf("accessor storage scoped: get shard %q: %w", virtualStorage, err)
}
- return proxy.NewStreamParameters(proxy.Destination{
- Ctx: helper.IncomingToOutgoing(ctx),
+ primaryStorage := shard.Primary.GetStorage()
+
+ b, err := rewrittenStorageMessage(mi, msg, primaryStorage)
+ if err != nil {
+ return nil, helper.ErrInvalidArgument(fmt.Errorf("accessor storage scoped: %w", err))
+ }
+
+ // As this is a read operation it could be routed to another storage (not only primary) if it meets constraints
+ // such as: it is healthy, it belongs to the same virtual storage bundle, etc.
+ // https://gitlab.com/gitlab-org/gitaly/-/issues/2972
+ primaryDest := proxy.Destination{
+ Ctx: ctx,
+ Conn: shard.Primary.GetConnection(),
+ Msg: b,
+ }
+
+ return proxy.NewStreamParameters(primaryDest, nil, func() error { return nil }, nil), nil
+}
+
+func (c *Coordinator) mutatorStorageStreamParameters(ctx context.Context, mi protoregistry.MethodInfo, msg proto.Message, virtualStorage string) (*proxy.StreamParameters, error) {
+ shard, err := c.nodeMgr.GetShard(virtualStorage)
+ if err != nil {
+ if errors.Is(err, nodes.ErrVirtualStorageNotExist) {
+ return nil, helper.ErrInvalidArgument(err)
+ }
+ return nil, helper.ErrInternalf("mutator storage scoped: get shard %q: %w", virtualStorage, err)
+ }
+
+ primaryStorage := shard.Primary.GetStorage()
+
+ b, err := rewrittenStorageMessage(mi, msg, primaryStorage)
+ if err != nil {
+ return nil, helper.ErrInvalidArgument(fmt.Errorf("mutator storage scoped: %w", err))
+ }
+
+ primaryDest := proxy.Destination{
+ Ctx: ctx,
Conn: shard.Primary.GetConnection(),
- Msg: payload,
- }, nil, func() error { return nil }, nil), nil
+ Msg: b,
+ }
+
+ secondaries := shard.GetHealthySecondaries()
+ secondaryDests := make([]proxy.Destination, len(secondaries))
+ for i, secondary := range secondaries {
+ b, err := rewrittenStorageMessage(mi, msg, secondary.GetStorage())
+ if err != nil {
+ return nil, helper.ErrInvalidArgument(fmt.Errorf("mutator storage scoped: %w", err))
+ }
+ secondaryDests[i] = proxy.Destination{Ctx: ctx, Conn: secondary.GetConnection(), Msg: b}
+ }
+
+ return proxy.NewStreamParameters(primaryDest, secondaryDests, func() error { return nil }, nil), nil
}
func rewrittenRepositoryMessage(mi protoregistry.MethodInfo, m proto.Message, storage string) ([]byte, error) {
@@ -412,6 +501,14 @@ func rewrittenRepositoryMessage(mi protoregistry.MethodInfo, m proto.Message, st
return b, nil
}
+func rewrittenStorageMessage(mi protoregistry.MethodInfo, m proto.Message, storage string) ([]byte, error) {
+ if err := mi.SetStorage(m, storage); err != nil {
+ return nil, helper.ErrInvalidArgument(err)
+ }
+
+ return proxy.NewCodec().Marshal(m)
+}
+
func protoMessage(mi protoregistry.MethodInfo, frame []byte) (proto.Message, error) {
m, err := mi.UnmarshalRequestProto(frame)
if err != nil {
diff --git a/internal/praefect/coordinator_test.go b/internal/praefect/coordinator_test.go
index f1557dd41..8ae128287 100644
--- a/internal/praefect/coordinator_test.go
+++ b/internal/praefect/coordinator_test.go
@@ -13,6 +13,7 @@ import (
"github.com/golang/protobuf/proto"
"github.com/golang/protobuf/ptypes/empty"
"github.com/sirupsen/logrus"
+ "github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
"gitlab.com/gitlab-org/gitaly/internal/metadata/featureflag"
"gitlab.com/gitlab-org/gitaly/internal/middleware/metadatahandler"
@@ -30,6 +31,7 @@ import (
"google.golang.org/grpc/codes"
"google.golang.org/grpc/health/grpc_health_v1"
"google.golang.org/grpc/metadata"
+ "google.golang.org/grpc/status"
)
var testLogger = logrus.New()
@@ -772,3 +774,262 @@ func TestCoordinatorEnqueueFailure(t *testing.T) {
require.Error(t, err)
require.Equal(t, err.Error(), "rpc error: code = Unknown desc = enqueue replication event: "+expectErrMsg)
}
+
+func TestStreamDirectorServerScope(t *testing.T) {
+ gz := proto.FileDescriptor("mock.proto")
+ fd, err := protoregistry.ExtractFileDescriptor(gz)
+ require.NoError(t, err)
+
+ registry, err := protoregistry.New(fd)
+ require.NoError(t, err)
+
+ conf := config.Config{VirtualStorages: []*config.VirtualStorage{{Name: "praefect"}, {Name: "another"}}}
+ mgr := &nodes.MockManager{GetShardFunc: func(s string) (nodes.Shard, error) {
+ require.Equal(t, "praefect", s)
+ return nodes.Shard{Primary: &nodes.MockNode{}}, nil
+ }}
+ coordinator := NewCoordinator(nil, mgr, nil, conf, registry)
+
+ fullMethod := "/mock.SimpleService/ServerAccessor"
+ requireScopeOperation(t, coordinator.registry, fullMethod, protoregistry.ScopeServer, protoregistry.OpAccessor)
+
+ frame, err := proto.Marshal(&mock.SimpleRequest{})
+ require.NoError(t, err)
+
+ ctx, cancel := testhelper.Context()
+ defer cancel()
+
+ sp, err := coordinator.StreamDirector(ctx, fullMethod, &mockPeeker{frame})
+ require.NoError(t, err)
+ require.NotNil(t, sp.Primary())
+ require.Empty(t, sp.Secondaries())
+}
+
+func TestStreamDirectorServerScope_Error(t *testing.T) {
+ gz := proto.FileDescriptor("mock.proto")
+ fd, err := protoregistry.ExtractFileDescriptor(gz)
+ require.NoError(t, err)
+
+ registry, err := protoregistry.New(fd)
+ require.NoError(t, err)
+
+ conf := config.Config{VirtualStorages: []*config.VirtualStorage{{Name: "fake"}, {Name: "another"}}}
+
+ t.Run("unknown storage provided", func(t *testing.T) {
+ mgr := &nodes.MockManager{
+ GetShardFunc: func(s string) (nodes.Shard, error) {
+ require.Equal(t, "fake", s)
+ return nodes.Shard{}, nodes.ErrVirtualStorageNotExist
+ },
+ }
+ coordinator := NewCoordinator(nil, mgr, nil, conf, registry)
+
+ fullMethod := "/mock.SimpleService/ServerAccessor"
+ requireScopeOperation(t, coordinator.registry, fullMethod, protoregistry.ScopeServer, protoregistry.OpAccessor)
+
+ frame, err := proto.Marshal(&mock.SimpleRequest{})
+ require.NoError(t, err)
+
+ ctx, cancel := testhelper.Context()
+ defer cancel()
+
+ _, err = coordinator.StreamDirector(ctx, fullMethod, &mockPeeker{frame})
+ require.Error(t, err)
+ result, ok := status.FromError(err)
+ require.True(t, ok)
+ require.Equal(t, codes.InvalidArgument, result.Code())
+ require.Equal(t, "virtual storage does not exist", result.Message())
+ })
+
+ t.Run("primary not healthy", func(t *testing.T) {
+ mgr := &nodes.MockManager{
+ GetShardFunc: func(s string) (nodes.Shard, error) {
+ require.Equal(t, "fake", s)
+ return nodes.Shard{}, nodes.ErrPrimaryNotHealthy
+ },
+ }
+ coordinator := NewCoordinator(nil, mgr, nil, conf, registry)
+
+ fullMethod := "/mock.SimpleService/ServerAccessor"
+ requireScopeOperation(t, coordinator.registry, fullMethod, protoregistry.ScopeServer, protoregistry.OpAccessor)
+
+ frame, err := proto.Marshal(&mock.SimpleRequest{})
+ require.NoError(t, err)
+
+ ctx, cancel := testhelper.Context()
+ defer cancel()
+
+ _, err = coordinator.StreamDirector(ctx, fullMethod, &mockPeeker{frame})
+ require.Error(t, err)
+ require.Equal(t, nodes.ErrPrimaryNotHealthy, err)
+ })
+}
+
+func TestStreamDirectorStorageScope(t *testing.T) {
+ // stubs health-check requests because nodes.NewManager establishes connection on creation
+ gitalySocket0, gitalySocket1 := testhelper.GetTemporaryGitalySocketFileName(), testhelper.GetTemporaryGitalySocketFileName()
+ srv1, _ := testhelper.NewServerWithHealth(t, gitalySocket0)
+ defer srv1.Stop()
+ srv2, _ := testhelper.NewServerWithHealth(t, gitalySocket1)
+ defer srv2.Stop()
+
+ primaryAddress, secondaryAddress := "unix://"+gitalySocket0, "unix://"+gitalySocket1
+ primaryGitaly := &config.Node{Address: primaryAddress, Storage: "gitaly-1", DefaultPrimary: true}
+ secondaryGitaly := &config.Node{Address: secondaryAddress, Storage: "gitaly-2"}
+ conf := config.Config{
+ Failover: config.Failover{Enabled: true},
+ VirtualStorages: []*config.VirtualStorage{{
+ Name: "praefect",
+ Nodes: []*config.Node{primaryGitaly, secondaryGitaly},
+ }}}
+
+ nodeMgr, err := nodes.NewManager(testhelper.DiscardTestEntry(t), conf, nil, nil, promtest.NewMockHistogramVec())
+ require.NoError(t, err)
+ nodeMgr.Start(0, time.Second)
+ coordinator := NewCoordinator(nil, nodeMgr, nil, conf, protoregistry.GitalyProtoPreregistered)
+
+ ctx, cancel := testhelper.Context()
+ defer cancel()
+
+ t.Run("mutator", func(t *testing.T) {
+ fullMethod := "/gitaly.NamespaceService/RemoveNamespace"
+ requireScopeOperation(t, coordinator.registry, fullMethod, protoregistry.ScopeStorage, protoregistry.OpMutator)
+
+ frame, err := proto.Marshal(&gitalypb.RemoveNamespaceRequest{
+ StorageName: conf.VirtualStorages[0].Name,
+ Name: "stub",
+ })
+ require.NoError(t, err)
+
+ streamParams, err := coordinator.StreamDirector(ctx, fullMethod, &mockPeeker{frame})
+ require.NoError(t, err)
+
+ require.Equal(t, primaryAddress, streamParams.Primary().Conn.Target(), "stream director didn't redirect to gitaly storage")
+
+ rewritten := gitalypb.RemoveNamespaceRequest{}
+ require.NoError(t, proto.Unmarshal(streamParams.Primary().Msg, &rewritten))
+ require.Equal(t, primaryGitaly.Storage, rewritten.StorageName, "stream director didn't rewrite storage")
+ })
+
+ t.Run("accessor", func(t *testing.T) {
+ fullMethod := "/gitaly.NamespaceService/NamespaceExists"
+ requireScopeOperation(t, coordinator.registry, fullMethod, protoregistry.ScopeStorage, protoregistry.OpAccessor)
+
+ frame, err := proto.Marshal(&gitalypb.NamespaceExistsRequest{
+ StorageName: conf.VirtualStorages[0].Name,
+ Name: "stub",
+ })
+ require.NoError(t, err)
+
+ streamParams, err := coordinator.StreamDirector(ctx, fullMethod, &mockPeeker{frame})
+ require.NoError(t, err)
+
+ require.Equal(t, primaryAddress, streamParams.Primary().Conn.Target(), "stream director didn't redirect to gitaly storage")
+
+ rewritten := gitalypb.RemoveNamespaceRequest{}
+ require.NoError(t, proto.Unmarshal(streamParams.Primary().Msg, &rewritten))
+ require.Equal(t, primaryGitaly.Storage, rewritten.StorageName, "stream director didn't rewrite storage")
+ })
+}
+
+func TestStreamDirectorStorageScopeError(t *testing.T) {
+ ctx, cancel := testhelper.Context()
+ defer cancel()
+
+ t.Run("no storage provided", func(t *testing.T) {
+ mgr := &nodes.MockManager{
+ GetShardFunc: func(s string) (nodes.Shard, error) {
+ require.FailNow(t, "validation of input was not executed")
+ return nodes.Shard{}, assert.AnError
+ },
+ }
+ coordinator := NewCoordinator(nil, mgr, nil, config.Config{}, protoregistry.GitalyProtoPreregistered)
+
+ frame, err := proto.Marshal(&gitalypb.RemoveNamespaceRequest{StorageName: "", Name: "stub"})
+ require.NoError(t, err)
+
+ _, err = coordinator.StreamDirector(ctx, "/gitaly.NamespaceService/RemoveNamespace", &mockPeeker{frame})
+ require.Error(t, err)
+ result, ok := status.FromError(err)
+ require.True(t, ok)
+ require.Equal(t, codes.InvalidArgument, result.Code())
+ require.Equal(t, "storage scoped: target storage is invalid", result.Message())
+ })
+
+ t.Run("unknown storage provided", func(t *testing.T) {
+ mgr := &nodes.MockManager{
+ GetShardFunc: func(s string) (nodes.Shard, error) {
+ require.Equal(t, "fake", s)
+ return nodes.Shard{}, nodes.ErrVirtualStorageNotExist
+ },
+ }
+ coordinator := NewCoordinator(nil, mgr, nil, config.Config{}, protoregistry.GitalyProtoPreregistered)
+
+ frame, err := proto.Marshal(&gitalypb.RemoveNamespaceRequest{StorageName: "fake", Name: "stub"})
+ require.NoError(t, err)
+
+ _, err = coordinator.StreamDirector(ctx, "/gitaly.NamespaceService/RemoveNamespace", &mockPeeker{frame})
+ require.Error(t, err)
+ result, ok := status.FromError(err)
+ require.True(t, ok)
+ require.Equal(t, codes.InvalidArgument, result.Code())
+ require.Equal(t, "virtual storage does not exist", result.Message())
+ })
+
+ t.Run("primary is not healthy", func(t *testing.T) {
+ t.Run("accessor", func(t *testing.T) {
+ mgr := &nodes.MockManager{
+ GetShardFunc: func(s string) (nodes.Shard, error) {
+ require.Equal(t, "fake", s)
+ return nodes.Shard{}, nodes.ErrPrimaryNotHealthy
+ },
+ }
+ coordinator := NewCoordinator(nil, mgr, nil, config.Config{}, protoregistry.GitalyProtoPreregistered)
+
+ fullMethod := "/gitaly.NamespaceService/NamespaceExists"
+ requireScopeOperation(t, coordinator.registry, fullMethod, protoregistry.ScopeStorage, protoregistry.OpAccessor)
+
+ frame, err := proto.Marshal(&gitalypb.NamespaceExistsRequest{StorageName: "fake", Name: "stub"})
+ require.NoError(t, err)
+
+ _, err = coordinator.StreamDirector(ctx, fullMethod, &mockPeeker{frame})
+ require.Error(t, err)
+ result, ok := status.FromError(err)
+ require.True(t, ok)
+ require.Equal(t, codes.Internal, result.Code())
+ require.Equal(t, `accessor storage scoped: get shard "fake": primary is not healthy`, result.Message())
+ })
+
+ t.Run("mutator", func(t *testing.T) {
+ mgr := &nodes.MockManager{
+ GetShardFunc: func(s string) (nodes.Shard, error) {
+ require.Equal(t, "fake", s)
+ return nodes.Shard{}, nodes.ErrPrimaryNotHealthy
+ },
+ }
+ coordinator := NewCoordinator(nil, mgr, nil, config.Config{}, protoregistry.GitalyProtoPreregistered)
+
+ fullMethod := "/gitaly.NamespaceService/RemoveNamespace"
+ requireScopeOperation(t, coordinator.registry, fullMethod, protoregistry.ScopeStorage, protoregistry.OpMutator)
+
+ frame, err := proto.Marshal(&gitalypb.RemoveNamespaceRequest{StorageName: "fake", Name: "stub"})
+ require.NoError(t, err)
+
+ _, err = coordinator.StreamDirector(ctx, fullMethod, &mockPeeker{frame})
+ require.Error(t, err)
+ result, ok := status.FromError(err)
+ require.True(t, ok)
+ require.Equal(t, codes.Internal, result.Code())
+ require.Equal(t, `mutator storage scoped: get shard "fake": primary is not healthy`, result.Message())
+ })
+ })
+}
+
+func requireScopeOperation(t *testing.T, registry *protoregistry.Registry, fullMethod string, scope protoregistry.Scope, op protoregistry.OpType) {
+ t.Helper()
+
+ mi, err := registry.LookupMethod(fullMethod)
+ require.NoError(t, err)
+ require.Equal(t, scope, mi.Scope, "scope doesn't match requested")
+ require.Equal(t, op, mi.Operation, "operation type doesn't match requested")
+}
diff --git a/internal/praefect/protoregistry/find_oid.go b/internal/praefect/protoregistry/find_oid.go
index 5a110c93c..abc87ccb0 100644
--- a/internal/praefect/protoregistry/find_oid.go
+++ b/internal/praefect/protoregistry/find_oid.go
@@ -50,6 +50,16 @@ func reflectFindStorage(pbMsg proto.Message, targetOID []int) (string, error) {
return targetRepo, nil
}
+func reflectSetStorage(pbMsg proto.Message, targetOID []int, storage string) error {
+ msgV, err := reflectFindOID(pbMsg, targetOID)
+ if err != nil {
+ return err
+ }
+
+ msgV.Set(reflect.ValueOf(storage))
+ return nil
+}
+
// ErrProtoFieldEmpty indicates the protobuf field is empty
var ErrProtoFieldEmpty = errors.New("proto field is empty")
diff --git a/internal/praefect/protoregistry/find_oid_test.go b/internal/praefect/protoregistry/find_oid_test.go
index 4a87db9c7..a040bb17b 100644
--- a/internal/praefect/protoregistry/find_oid_test.go
+++ b/internal/praefect/protoregistry/find_oid_test.go
@@ -159,3 +159,48 @@ func TestProtoRegistryStorage(t *testing.T) {
})
}
}
+
+func TestMethodInfo_SetStorage(t *testing.T) {
+ testCases := []struct {
+ desc string
+ service string
+ method string
+ pbMsg proto.Message
+ storage string
+ expectErr error
+ }{
+ {
+ desc: "valid request type",
+ service: "NamespaceService",
+ method: "AddNamespace",
+ pbMsg: &gitalypb.AddNamespaceRequest{
+ StorageName: "old_storage",
+ },
+ storage: "new_storage",
+ },
+ {
+ desc: "incorrect request type",
+ service: "RepositoryService",
+ method: "RepackIncremental",
+ pbMsg: &gitalypb.RepackIncrementalResponse{},
+ expectErr: errors.New("proto message gitaly.RepackIncrementalResponse does not match expected RPC request message gitaly.RepackIncrementalRequest"),
+ },
+ }
+
+ for _, tc := range testCases {
+ t.Run(tc.desc, func(t *testing.T) {
+ info, err := protoregistry.GitalyProtoPreregistered.LookupMethod("/gitaly." + tc.service + "/" + tc.method)
+ require.NoError(t, err)
+
+ err = info.SetStorage(tc.pbMsg, tc.storage)
+ if tc.expectErr == nil {
+ require.NoError(t, err)
+ changed, err := info.Storage(tc.pbMsg)
+ require.NoError(t, err)
+ require.Equal(t, tc.storage, changed)
+ } else {
+ require.Equal(t, tc.expectErr, err)
+ }
+ })
+ }
+}
diff --git a/internal/praefect/protoregistry/protoregistry.go b/internal/praefect/protoregistry/protoregistry.go
index cabc38b19..7acc94d03 100644
--- a/internal/praefect/protoregistry/protoregistry.go
+++ b/internal/praefect/protoregistry/protoregistry.go
@@ -67,6 +67,19 @@ const (
ScopeServer
)
+func (s Scope) String() string {
+ switch s {
+ case ScopeServer:
+ return "server"
+ case ScopeStorage:
+ return "storage"
+ case ScopeRepository:
+ return "repository"
+ default:
+ return fmt.Sprintf("N/A: %d", s)
+ }
+}
+
var protoScope = map[gitalypb.OperationMsg_Scope]Scope{
gitalypb.OperationMsg_SERVER: ScopeServer,
gitalypb.OperationMsg_REPOSITORY: ScopeRepository,
@@ -137,6 +150,18 @@ func (mi MethodInfo) Storage(msg proto.Message) (string, error) {
return reflectFindStorage(msg, mi.storage)
}
+// SetStorage sets the storage name for a protobuf message
+func (mi MethodInfo) SetStorage(msg proto.Message, storage string) error {
+ if mi.requestName != proto.MessageName(msg) {
+ return fmt.Errorf(
+ "proto message %s does not match expected RPC request message %s",
+ proto.MessageName(msg), mi.requestName,
+ )
+ }
+
+ return reflectSetStorage(msg, mi.storage, storage)
+}
+
// UnmarshalRequestProto will unmarshal the bytes into the method's request
// message type
func (mi MethodInfo) UnmarshalRequestProto(b []byte) (proto.Message, error) {
diff --git a/internal/service/remote/remotes_test.go b/internal/service/remote/remotes_test.go
index 1e5a3a5c4..4d13eec2e 100644
--- a/internal/service/remote/remotes_test.go
+++ b/internal/service/remote/remotes_test.go
@@ -232,7 +232,7 @@ func TestFindRemoteRepository(t *testing.T) {
}))
defer ts.Close()
- resp, err := client.FindRemoteRepository(ctx, &gitalypb.FindRemoteRepositoryRequest{Remote: ts.URL})
+ resp, err := client.FindRemoteRepository(ctx, &gitalypb.FindRemoteRepositoryRequest{Remote: ts.URL, StorageName: "default"})
require.NoError(t, err)
require.True(t, resp.Exists)
@@ -259,7 +259,7 @@ func TestFailedFindRemoteRepository(t *testing.T) {
}
for _, tc := range testCases {
- resp, err := client.FindRemoteRepository(ctx, &gitalypb.FindRemoteRepositoryRequest{Remote: tc.remote})
+ resp, err := client.FindRemoteRepository(ctx, &gitalypb.FindRemoteRepositoryRequest{Remote: tc.remote, StorageName: "default"})
if tc.code == codes.OK {
require.NoError(t, err)
} else {
diff --git a/proto/go/gitalypb/remote.pb.go b/proto/go/gitalypb/remote.pb.go
index a72ca4484..4401ddc73 100644
--- a/proto/go/gitalypb/remote.pb.go
+++ b/proto/go/gitalypb/remote.pb.go
@@ -740,57 +740,57 @@ func init() {
func init() { proto.RegisterFile("remote.proto", fileDescriptor_eefc82927d57d89b) }
var fileDescriptor_eefc82927d57d89b = []byte{
- // 791 bytes of a gzipped FileDescriptorProto
- 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x56, 0xcd, 0x6e, 0xda, 0x4a,
- 0x14, 0x96, 0x81, 0xf0, 0x73, 0x20, 0x11, 0x0c, 0x51, 0xae, 0x31, 0x91, 0x2e, 0xf1, 0xbd, 0x91,
- 0x58, 0xdc, 0x40, 0x94, 0xdb, 0x56, 0x5d, 0x54, 0xaa, 0x9a, 0x56, 0x51, 0xfa, 0x93, 0xa8, 0x75,
- 0x1b, 0xa9, 0xea, 0xc6, 0x35, 0x30, 0x80, 0x85, 0xf1, 0xb8, 0x33, 0x43, 0x5a, 0xde, 0xa2, 0xbb,
- 0x74, 0xd1, 0x17, 0xe8, 0xa6, 0x8f, 0xd1, 0x87, 0xca, 0xaa, 0x1a, 0xcf, 0x18, 0x0c, 0x31, 0x74,
- 0x91, 0x76, 0xe7, 0xf3, 0x9d, 0x33, 0xdf, 0xf9, 0x3f, 0x32, 0x94, 0x28, 0x1e, 0x13, 0x8e, 0x5b,
- 0x01, 0x25, 0x9c, 0xa0, 0xec, 0xc0, 0xe5, 0x8e, 0x37, 0x35, 0xc0, 0x73, 0x7d, 0x2e, 0x31, 0xa3,
- 0xc4, 0x86, 0x0e, 0xc5, 0x3d, 0x29, 0x99, 0xdf, 0x35, 0x28, 0x3f, 0xea, 0xf5, 0xac, 0xf0, 0x95,
- 0x85, 0x3f, 0x4c, 0x30, 0xe3, 0xe8, 0x3e, 0x00, 0xc5, 0x01, 0x61, 0x2e, 0x27, 0x74, 0xaa, 0x6b,
- 0x0d, 0xad, 0x59, 0x3c, 0x42, 0x2d, 0xc9, 0xd5, 0xb2, 0x66, 0x9a, 0xe3, 0xcc, 0x97, 0x1f, 0xff,
- 0x69, 0x56, 0xcc, 0x16, 0x21, 0xc8, 0xf8, 0xce, 0x18, 0xeb, 0xa9, 0x86, 0xd6, 0x2c, 0x58, 0xe1,
- 0x37, 0x2a, 0x43, 0x7a, 0x42, 0x3d, 0x3d, 0x1d, 0x42, 0xe2, 0x13, 0xed, 0xc3, 0xd6, 0xd8, 0xa5,
- 0x94, 0x50, 0x9b, 0xe2, 0xfe, 0xd8, 0x09, 0x98, 0xbe, 0xd1, 0x48, 0x37, 0x0b, 0xd6, 0xa6, 0x44,
- 0x2d, 0x09, 0x3e, 0xcb, 0xe4, 0x33, 0xe5, 0x8d, 0x08, 0x54, 0xa6, 0x66, 0x15, 0x2a, 0xb1, 0x78,
- 0x59, 0x40, 0x7c, 0x86, 0xcd, 0x2e, 0x54, 0x05, 0x72, 0x89, 0xff, 0x60, 0x1e, 0x66, 0x0b, 0xb6,
- 0x17, 0x9d, 0x48, 0xe7, 0x68, 0x07, 0xb2, 0x14, 0xb3, 0x89, 0xc7, 0x43, 0x0f, 0x79, 0x4b, 0x49,
- 0xe6, 0x95, 0x06, 0xc6, 0x09, 0xe6, 0xdd, 0xe1, 0x53, 0x9f, 0x63, 0xea, 0x3b, 0xde, 0xef, 0x0a,
- 0xee, 0x21, 0x54, 0x64, 0x97, 0xed, 0x18, 0x41, 0x6a, 0x15, 0x81, 0x55, 0xa6, 0xca, 0x6f, 0x84,
- 0x98, 0x77, 0xa1, 0x9e, 0x18, 0xd8, 0x2f, 0x12, 0xfa, 0x9c, 0x82, 0xda, 0x45, 0xd0, 0x73, 0xb8,
- 0xaa, 0xc0, 0x99, 0xea, 0xd6, 0x6d, 0xf3, 0xa9, 0x41, 0x9e, 0xe2, 0xbe, 0x1d, 0x2b, 0x78, 0x8e,
- 0xe2, 0xfe, 0xb9, 0x98, 0x9d, 0x3b, 0xb0, 0x43, 0x7c, 0x6f, 0x6a, 0x77, 0xa8, 0xe3, 0x77, 0x87,
- 0x98, 0xd9, 0x63, 0x87, 0x77, 0x87, 0xae, 0x3f, 0xd0, 0xd3, 0x8d, 0x74, 0xb3, 0x64, 0x6d, 0x0b,
- 0xed, 0xb1, 0x52, 0x9e, 0x29, 0x1d, 0xfa, 0x0b, 0x72, 0x8c, 0x0d, 0xed, 0x11, 0x9e, 0xea, 0x99,
- 0x90, 0x2f, 0xcb, 0xd8, 0xf0, 0x39, 0x9e, 0xa2, 0xbf, 0xa1, 0x38, 0xf2, 0xc9, 0x47, 0xdf, 0x1e,
- 0x12, 0xc6, 0xc5, 0xd4, 0x09, 0x25, 0x84, 0xd0, 0xa9, 0x40, 0x50, 0x0b, 0xaa, 0x23, 0x8c, 0x03,
- 0xbb, 0xe7, 0x5e, 0x62, 0x3a, 0xc0, 0x3e, 0x17, 0x63, 0xc7, 0xf4, 0x6c, 0x58, 0x87, 0x8a, 0x50,
- 0x3d, 0x89, 0x34, 0x16, 0xee, 0x33, 0xf3, 0x31, 0x18, 0x49, 0x15, 0x51, 0x85, 0xdc, 0x87, 0xad,
- 0x25, 0x22, 0x2d, 0x8c, 0x7a, 0xb3, 0xb7, 0x40, 0xf2, 0x16, 0xea, 0x27, 0xae, 0x3f, 0x9b, 0xe9,
- 0x59, 0xe3, 0x54, 0x61, 0xc3, 0x76, 0x08, 0x55, 0x58, 0xd4, 0x82, 0xa5, 0x24, 0xb4, 0x07, 0x25,
- 0xc6, 0x09, 0x75, 0x06, 0x38, 0x5e, 0xba, 0xa2, 0xc2, 0x44, 0xf9, 0xcc, 0x7b, 0xb0, 0x9b, 0xcc,
- 0x3c, 0xef, 0x34, 0xfe, 0xe4, 0x8a, 0x52, 0xa8, 0x4e, 0x4b, 0xc9, 0xf4, 0x40, 0x8f, 0xbd, 0x23,
- 0x44, 0xc4, 0x79, 0xfb, 0x3e, 0xcf, 0x13, 0x49, 0xc5, 0x13, 0x31, 0x0f, 0xa0, 0x96, 0xe0, 0x4d,
- 0x85, 0x58, 0x86, 0x34, 0xc5, 0x7d, 0x95, 0xba, 0xf8, 0x34, 0xcf, 0x01, 0xbd, 0x70, 0x19, 0x97,
- 0xe6, 0xec, 0xd6, 0x61, 0x99, 0xdf, 0x34, 0xa8, 0x2e, 0x10, 0x2a, 0xcf, 0x0f, 0x20, 0x27, 0x03,
- 0x94, 0x6d, 0x2b, 0x1e, 0x99, 0x11, 0x5d, 0x82, 0x75, 0x4b, 0x45, 0x1f, 0x3d, 0x31, 0xde, 0x40,
- 0x56, 0x42, 0xb3, 0x5b, 0xa2, 0xc5, 0x6e, 0x62, 0x1d, 0x0a, 0x7d, 0xb1, 0x81, 0xb6, 0xb8, 0x8c,
- 0xb2, 0x1a, 0xf9, 0x10, 0xb8, 0xa0, 0x9e, 0xd8, 0x87, 0x60, 0xc2, 0xa4, 0x4e, 0x5e, 0xcd, 0x9c,
- 0x90, 0x2f, 0xa8, 0x77, 0xf4, 0x75, 0x03, 0x36, 0x25, 0xed, 0x6b, 0x4c, 0x2f, 0xdd, 0x2e, 0x46,
- 0xa7, 0x50, 0x98, 0xdd, 0x43, 0xa4, 0x47, 0x11, 0x2e, 0x9f, 0x74, 0xa3, 0x96, 0xa0, 0x51, 0xc7,
- 0x33, 0x7b, 0x7d, 0xd5, 0x4c, 0xe5, 0x35, 0xd4, 0x87, 0x6a, 0xc2, 0x55, 0x40, 0xb3, 0xac, 0x57,
- 0xdf, 0x32, 0xe3, 0x9f, 0xb5, 0x36, 0x4b, 0x7e, 0x5e, 0x41, 0x29, 0x7e, 0x47, 0x51, 0x7d, 0xde,
- 0xa5, 0x1b, 0x27, 0xdc, 0xd8, 0x4d, 0x56, 0x2e, 0x51, 0x62, 0x40, 0x37, 0xd7, 0x10, 0xed, 0x45,
- 0x6f, 0x57, 0x1e, 0x2d, 0xc3, 0x5c, 0x67, 0xb2, 0xe8, 0xa4, 0xa9, 0xa1, 0x11, 0x6c, 0x27, 0xad,
- 0x13, 0x9a, 0xa7, 0xbf, 0x7a, 0x8d, 0x8d, 0x7f, 0xd7, 0x1b, 0x29, 0x67, 0xf9, 0xeb, 0xab, 0x66,
- 0x26, 0x9f, 0x2a, 0x6b, 0xe8, 0x3d, 0x54, 0x6e, 0x6c, 0x05, 0x6a, 0x24, 0x90, 0x2c, 0xac, 0xa7,
- 0xb1, 0xb7, 0xc6, 0x62, 0x21, 0xa1, 0x14, 0x7a, 0x09, 0xc5, 0xd8, 0x24, 0x23, 0x23, 0x71, 0xbc,
- 0x25, 0x6b, 0x7d, 0xcd, 0xe8, 0x47, 0x7c, 0x87, 0xda, 0xf1, 0xe1, 0x3b, 0x61, 0xe7, 0x39, 0x9d,
- 0x56, 0x97, 0x8c, 0xdb, 0xf2, 0xf3, 0x80, 0xd0, 0x41, 0x5b, 0xbe, 0x6e, 0x87, 0xff, 0x1c, 0xed,
- 0x01, 0x51, 0x72, 0xd0, 0xe9, 0x64, 0x43, 0xe8, 0xff, 0x9f, 0x01, 0x00, 0x00, 0xff, 0xff, 0x53,
- 0x32, 0x43, 0xff, 0xb8, 0x08, 0x00, 0x00,
+ // 798 bytes of a gzipped FileDescriptorProto
+ 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x56, 0x4b, 0x6f, 0xda, 0x4a,
+ 0x14, 0x96, 0x0d, 0xe1, 0x71, 0x20, 0x11, 0x0c, 0x51, 0xae, 0x31, 0x91, 0x2e, 0xf1, 0xbd, 0xd1,
+ 0x65, 0x71, 0x03, 0x51, 0xfa, 0x50, 0x17, 0x95, 0xaa, 0xd2, 0x2a, 0x4a, 0x1f, 0x89, 0x5a, 0xb7,
+ 0xd9, 0x74, 0x51, 0xd7, 0xc0, 0x00, 0x16, 0xc6, 0xe3, 0xce, 0x98, 0xb4, 0xfc, 0x83, 0x2e, 0xbb,
+ 0x4b, 0x17, 0xfd, 0x03, 0xdd, 0xf4, 0x67, 0xf4, 0x47, 0x65, 0x55, 0x8d, 0x67, 0x0c, 0x86, 0x18,
+ 0xba, 0x48, 0xbb, 0xf3, 0x79, 0xcc, 0x77, 0xce, 0xf9, 0xce, 0x43, 0x86, 0x22, 0xc5, 0x63, 0x12,
+ 0xe0, 0xa6, 0x4f, 0x49, 0x40, 0x50, 0x66, 0xe0, 0x04, 0xb6, 0x3b, 0xd5, 0xc1, 0x75, 0xbc, 0x40,
+ 0xe8, 0xf4, 0x22, 0x1b, 0xda, 0x14, 0xf7, 0x84, 0x64, 0x7c, 0x57, 0xa0, 0xf4, 0xb0, 0xd7, 0x33,
+ 0xc3, 0x57, 0x26, 0x7e, 0x3f, 0xc1, 0x2c, 0x40, 0xf7, 0x00, 0x28, 0xf6, 0x09, 0x73, 0x02, 0x42,
+ 0xa7, 0x9a, 0x52, 0x57, 0x1a, 0x85, 0x23, 0xd4, 0x14, 0x58, 0x4d, 0x73, 0x66, 0x69, 0xa7, 0xbf,
+ 0xfc, 0xf8, 0x5f, 0x31, 0x63, 0xbe, 0x08, 0x41, 0xda, 0xb3, 0xc7, 0x58, 0x53, 0xeb, 0x4a, 0x23,
+ 0x6f, 0x86, 0xdf, 0xa8, 0x04, 0xa9, 0x09, 0x75, 0xb5, 0x54, 0xa8, 0xe2, 0x9f, 0x68, 0x1f, 0xb6,
+ 0xc6, 0x0e, 0xa5, 0x84, 0x5a, 0x14, 0xf7, 0xc7, 0xb6, 0xcf, 0xb4, 0x8d, 0x7a, 0xaa, 0x91, 0x37,
+ 0x37, 0x85, 0xd6, 0x14, 0xca, 0xa7, 0xe9, 0x5c, 0xba, 0xb4, 0x11, 0x29, 0xa5, 0xab, 0x51, 0x81,
+ 0x72, 0x2c, 0x5f, 0xe6, 0x13, 0x8f, 0x61, 0xa3, 0x0b, 0x15, 0xae, 0xb9, 0xc0, 0x7f, 0xb0, 0x0e,
+ 0xa3, 0x09, 0xdb, 0x8b, 0x41, 0x44, 0x70, 0xb4, 0x03, 0x19, 0x8a, 0xd9, 0xc4, 0x0d, 0xc2, 0x08,
+ 0x39, 0x53, 0x4a, 0xc6, 0xa5, 0x02, 0xfa, 0x31, 0x0e, 0xba, 0xc3, 0x27, 0x5e, 0x80, 0xa9, 0x67,
+ 0xbb, 0xbf, 0x2b, 0xb9, 0x07, 0x50, 0x16, 0x5d, 0xb6, 0x62, 0x00, 0xea, 0x2a, 0x00, 0xb3, 0x44,
+ 0x65, 0xdc, 0x48, 0x63, 0xdc, 0x81, 0x5a, 0x62, 0x62, 0xbf, 0x28, 0xe8, 0xb3, 0x0a, 0xd5, 0x73,
+ 0xbf, 0x67, 0x07, 0x92, 0x81, 0x53, 0xd9, 0xad, 0x9b, 0xd6, 0x53, 0x85, 0x1c, 0xc5, 0x7d, 0x2b,
+ 0x46, 0x78, 0x96, 0xe2, 0xfe, 0x19, 0x9f, 0x9d, 0xdb, 0xb0, 0x43, 0x3c, 0x77, 0x6a, 0x75, 0xa8,
+ 0xed, 0x75, 0x87, 0x98, 0x59, 0x63, 0x3b, 0xe8, 0x0e, 0x1d, 0x6f, 0xa0, 0xa5, 0xea, 0xa9, 0x46,
+ 0xd1, 0xdc, 0xe6, 0xd6, 0xb6, 0x34, 0x9e, 0x4a, 0x1b, 0xfa, 0x0b, 0xb2, 0x8c, 0x0d, 0xad, 0x11,
+ 0x9e, 0x6a, 0xe9, 0x10, 0x2f, 0xc3, 0xd8, 0xf0, 0x19, 0x9e, 0xa2, 0xbf, 0xa1, 0x30, 0xf2, 0xc8,
+ 0x07, 0xcf, 0x1a, 0x12, 0x16, 0xf0, 0xa9, 0xe3, 0x46, 0x08, 0x55, 0x27, 0x5c, 0x83, 0x9a, 0x50,
+ 0x19, 0x61, 0xec, 0x5b, 0x3d, 0xe7, 0x02, 0xd3, 0x01, 0xf6, 0x02, 0x3e, 0x76, 0x4c, 0xcb, 0x84,
+ 0x3c, 0x94, 0xb9, 0xe9, 0x71, 0x64, 0x31, 0x71, 0x9f, 0x19, 0x8f, 0x40, 0x4f, 0x62, 0x44, 0x12,
+ 0xb9, 0x0f, 0x5b, 0x4b, 0x40, 0x4a, 0x98, 0xf5, 0x66, 0x6f, 0x01, 0xe4, 0x2d, 0xd4, 0x8e, 0x1d,
+ 0x6f, 0x36, 0xd3, 0xb3, 0xc6, 0x49, 0x62, 0xc3, 0x76, 0x70, 0x53, 0x48, 0x6a, 0xde, 0x94, 0x12,
+ 0xfa, 0x0f, 0x8a, 0x2c, 0x20, 0xd4, 0x1e, 0xe0, 0x18, 0x75, 0xed, 0xf4, 0x27, 0x4e, 0x6f, 0x41,
+ 0x5a, 0x38, 0x89, 0xc6, 0x5d, 0xd8, 0x4d, 0xc6, 0x9f, 0xf7, 0x1b, 0x7f, 0x74, 0x38, 0x21, 0xb2,
+ 0xdf, 0x42, 0x32, 0x5c, 0xd0, 0x62, 0xef, 0x08, 0xe1, 0xd9, 0xde, 0xbc, 0xdb, 0xf3, 0x72, 0xd4,
+ 0x78, 0x39, 0xc6, 0x01, 0x54, 0x13, 0xa2, 0xc9, 0x14, 0x4b, 0x90, 0xa2, 0xb8, 0x2f, 0x09, 0xe0,
+ 0x9f, 0xc6, 0x19, 0xa0, 0xe7, 0x0e, 0x0b, 0x84, 0x3b, 0xbb, 0x71, 0x5a, 0xc6, 0x37, 0x05, 0x2a,
+ 0x0b, 0x80, 0x32, 0xf2, 0x7d, 0xc8, 0x8a, 0x04, 0x45, 0xf3, 0x0a, 0x47, 0x46, 0x04, 0x97, 0xe0,
+ 0xdd, 0x94, 0xd9, 0x47, 0x4f, 0xf4, 0xd7, 0x90, 0x11, 0xaa, 0xd9, 0x45, 0x51, 0x62, 0x97, 0xb1,
+ 0x06, 0xf9, 0x3e, 0xdf, 0x43, 0x8b, 0xdf, 0x47, 0xc1, 0x46, 0x2e, 0x54, 0x9c, 0x53, 0x97, 0x6f,
+ 0x85, 0x3f, 0x61, 0xc2, 0x26, 0x6e, 0x67, 0x96, 0xcb, 0xe7, 0xd4, 0x3d, 0xfa, 0xba, 0x01, 0x9b,
+ 0x02, 0xf6, 0x15, 0xa6, 0x17, 0x4e, 0x17, 0xa3, 0x13, 0xc8, 0xcf, 0xae, 0x22, 0xd2, 0xa2, 0x0c,
+ 0x97, 0x0f, 0xbb, 0x5e, 0x4d, 0xb0, 0xc8, 0x13, 0x9a, 0xb9, 0xba, 0x6c, 0xa8, 0x39, 0x05, 0xf5,
+ 0xa1, 0x92, 0x70, 0x1b, 0xd0, 0xac, 0xea, 0xd5, 0x17, 0x4d, 0xff, 0x67, 0xad, 0xcf, 0x52, 0x9c,
+ 0x97, 0x50, 0x8c, 0x5f, 0x53, 0x54, 0x9b, 0x77, 0xe9, 0xda, 0x21, 0xd7, 0x77, 0x93, 0x8d, 0x4b,
+ 0x90, 0x18, 0xd0, 0xf5, 0x65, 0x44, 0x7b, 0xd1, 0xdb, 0x95, 0xa7, 0x4b, 0x37, 0xd6, 0xb9, 0x2c,
+ 0x06, 0x69, 0x28, 0x68, 0x04, 0xdb, 0x49, 0xeb, 0x84, 0xe6, 0xe5, 0xaf, 0x5e, 0x66, 0xfd, 0xdf,
+ 0xf5, 0x4e, 0x32, 0x58, 0xee, 0xea, 0xb2, 0x91, 0xce, 0xa9, 0x25, 0x15, 0xbd, 0x83, 0xf2, 0xb5,
+ 0xad, 0x40, 0xf5, 0x04, 0x90, 0x85, 0xf5, 0xd4, 0xf7, 0xd6, 0x78, 0x2c, 0x14, 0xa4, 0xa2, 0x17,
+ 0x50, 0x88, 0x4d, 0x32, 0xd2, 0x13, 0xc7, 0x5b, 0xa0, 0xd6, 0xd6, 0x8c, 0x7e, 0x84, 0x77, 0xa8,
+ 0xb4, 0x0f, 0xdf, 0x70, 0x3f, 0xd7, 0xee, 0x34, 0xbb, 0x64, 0xdc, 0x12, 0x9f, 0x07, 0x84, 0x0e,
+ 0x5a, 0xe2, 0x75, 0x2b, 0xfc, 0xf3, 0x68, 0x0d, 0x88, 0x94, 0xfd, 0x4e, 0x27, 0x13, 0xaa, 0x6e,
+ 0xfd, 0x0c, 0x00, 0x00, 0xff, 0xff, 0xd8, 0xf5, 0x59, 0x0f, 0xbe, 0x08, 0x00, 0x00,
}
// Reference imports to suppress errors if they are not otherwise used.
diff --git a/proto/remote.proto b/proto/remote.proto
index 2a5bad3e4..d8c23447a 100644
--- a/proto/remote.proto
+++ b/proto/remote.proto
@@ -32,7 +32,7 @@ service RemoteService {
rpc FindRemoteRepository(FindRemoteRepositoryRequest) returns (FindRemoteRepositoryResponse) {
option (op_type) = {
op: ACCESSOR
- scope_level: SERVER
+ scope_level: STORAGE
};
}
rpc FindRemoteRootRef(FindRemoteRootRefRequest) returns (FindRemoteRootRefResponse) {
@@ -96,7 +96,7 @@ message FindRemoteRepositoryRequest {
// This field is used to redirect request to proper storage where it can be handled.
// As of now it doesn't matter what storage will be used, but it still must be a valid.
// For more details: https://gitlab.com/gitlab-org/gitaly/-/issues/2442
- string storage_name = 2;
+ string storage_name = 2 [(storage)=true];
}
// This migth throw a GRPC Unavailable code, to signal the request failure