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>2023-11-15 13:28:00 +0300
committerSami Hiltunen <shiltunen@gitlab.com>2023-11-15 13:28:00 +0300
commit94ff31fb91d07a62f2605f5af3a1d11024565a9d (patch)
treed52133f33fcc73652233f6afe227eb1a5bdbf308
parentbf3c6b19adecba57e3a6ed36e37df685a58778cc (diff)
parent652a1a2c2da724ef5199e3271db43a92dee5914d (diff)
Merge branch '4277-deprecate-the-renamerepository-rpc' into 'master'
repository: Remove RenameRepository RPC Closes #4277 See merge request https://gitlab.com/gitlab-org/gitaly/-/merge_requests/6506 Merged-by: Sami Hiltunen <shiltunen@gitlab.com> Approved-by: Sami Hiltunen <shiltunen@gitlab.com> Reviewed-by: Sami Hiltunen <shiltunen@gitlab.com> Reviewed-by: karthik nayak <knayak@gitlab.com> Co-authored-by: Karthik Nayak <knayak@gitlab.com>
-rw-r--r--internal/gitaly/service/repository/rename.go123
-rw-r--r--internal/gitaly/service/repository/rename_test.go138
-rw-r--r--internal/gitaly/storage/storagemgr/middleware.go3
-rw-r--r--internal/praefect/coordinator.go18
-rw-r--r--internal/praefect/coordinator_test.go9
-rw-r--r--internal/praefect/datastore/datastore.go4
-rw-r--r--internal/praefect/datastore/queue_test.go12
-rw-r--r--internal/praefect/datastore/repository_store.go72
-rw-r--r--internal/praefect/datastore/repository_store_mock.go16
-rw-r--r--internal/praefect/datastore/repository_store_test.go164
-rw-r--r--internal/praefect/reconciler/reconciler_test.go72
-rw-r--r--internal/praefect/rename_repository.go78
-rw-r--r--internal/praefect/replicator.go53
-rw-r--r--internal/praefect/replicator_test.go47
-rw-r--r--internal/praefect/server.go1
-rw-r--r--internal/praefect/server_test.go113
-rw-r--r--proto/go/gitalypb/repository.pb.go1421
-rw-r--r--proto/go/gitalypb/repository_grpc.pb.go38
-rw-r--r--proto/repository.proto19
19 files changed, 654 insertions, 1747 deletions
diff --git a/internal/gitaly/service/repository/rename.go b/internal/gitaly/service/repository/rename.go
deleted file mode 100644
index 08cc1c8f1..000000000
--- a/internal/gitaly/service/repository/rename.go
+++ /dev/null
@@ -1,123 +0,0 @@
-package repository
-
-import (
- "context"
- "errors"
- "fmt"
- "os"
- "path/filepath"
-
- "gitlab.com/gitlab-org/gitaly/v16/internal/gitaly/storage"
- "gitlab.com/gitlab-org/gitaly/v16/internal/helper/perm"
- "gitlab.com/gitlab-org/gitaly/v16/internal/safe"
- "gitlab.com/gitlab-org/gitaly/v16/internal/structerr"
- "gitlab.com/gitlab-org/gitaly/v16/proto/go/gitalypb"
-)
-
-func (s *server) RenameRepository(ctx context.Context, in *gitalypb.RenameRepositoryRequest) (*gitalypb.RenameRepositoryResponse, error) {
- if err := validateRenameRepositoryRequest(s.locator, in); err != nil {
- return nil, structerr.NewInvalidArgument("%w", err)
- }
-
- targetRepo := &gitalypb.Repository{
- StorageName: in.GetRepository().GetStorageName(),
- RelativePath: in.GetRelativePath(),
- }
-
- if err := s.renameRepository(ctx, in.GetRepository(), targetRepo); err != nil {
- return nil, structerr.NewInternal("%w", err)
- }
-
- return &gitalypb.RenameRepositoryResponse{}, nil
-}
-
-func (s *server) renameRepository(ctx context.Context, sourceRepo, targetRepo *gitalypb.Repository) error {
- sourcePath, err := s.locator.GetRepoPath(sourceRepo)
- if err != nil {
- return structerr.NewInvalidArgument("%w", err)
- }
-
- targetPath, err := s.locator.GetRepoPath(targetRepo, storage.WithRepositoryVerificationSkipped())
- if err != nil {
- return structerr.NewInvalidArgument("%w", err)
- }
-
- // Check up front whether the target path exists already. If it does, we can avoid going
- // into the critical section altogether.
- if _, err := os.Stat(targetPath); !os.IsNotExist(err) {
- return structerr.NewAlreadyExists("target repo exists already")
- }
-
- if err := os.MkdirAll(filepath.Dir(targetPath), perm.GroupPrivateDir); err != nil {
- return fmt.Errorf("create target parent dir: %w", err)
- }
-
- // We're locking both the source repository path and the target repository path for
- // concurrent modification. This is so that the source repo doesn't get moved somewhere else
- // meanwhile, and so that the target repo doesn't get created concurrently either.
- sourceLocker, err := safe.NewLockingFileWriter(sourcePath)
- if err != nil {
- return fmt.Errorf("creating source repo locker: %w", err)
- }
- defer func() {
- if err := sourceLocker.Close(); err != nil {
- s.logger.WithError(err).ErrorContext(ctx, "closing source repo locker failed")
- }
- }()
-
- targetLocker, err := safe.NewLockingFileWriter(targetPath)
- if err != nil {
- return fmt.Errorf("creating target repo locker: %w", err)
- }
- defer func() {
- if err := targetLocker.Close(); err != nil {
- s.logger.WithError(err).ErrorContext(ctx, "closing target repo locker failed")
- }
- }()
-
- // We're now entering the critical section where both the source and target path are locked.
- if err := sourceLocker.Lock(); err != nil {
- return fmt.Errorf("locking source repo: %w", err)
- }
- if err := targetLocker.Lock(); err != nil {
- return fmt.Errorf("locking target repo: %w", err)
- }
-
- // We need to re-check whether the target path exists in case somebody has removed it before
- // we have taken the lock.
- if _, err := os.Stat(targetPath); !os.IsNotExist(err) {
- return structerr.NewAlreadyExists("target repo exists already")
- }
-
- if err := os.Rename(sourcePath, targetPath); err != nil {
- return fmt.Errorf("moving repository into place: %w", err)
- }
-
- storagePath, err := s.locator.GetStorageByName(targetRepo.GetStorageName())
- if err != nil {
- return fmt.Errorf("get storage by name: %w", err)
- }
-
- syncer := safe.NewSyncer()
- if err := syncer.SyncHierarchy(storagePath, targetRepo.GetRelativePath()); err != nil {
- return fmt.Errorf("sync hierarchy: %w", err)
- }
-
- if err := syncer.SyncParent(sourcePath); err != nil {
- return fmt.Errorf("sync parent: %w", err)
- }
-
- return nil
-}
-
-func validateRenameRepositoryRequest(locator storage.Locator, in *gitalypb.RenameRepositoryRequest) error {
- if err := locator.ValidateRepository(in.GetRepository()); err != nil {
- return structerr.NewInvalidArgument("%w", err)
- }
-
- if in.GetRelativePath() == "" {
- return errors.New("destination relative path is empty")
- }
-
- return nil
-}
diff --git a/internal/gitaly/service/repository/rename_test.go b/internal/gitaly/service/repository/rename_test.go
deleted file mode 100644
index 120ece26f..000000000
--- a/internal/gitaly/service/repository/rename_test.go
+++ /dev/null
@@ -1,138 +0,0 @@
-package repository
-
-import (
- "path/filepath"
- "testing"
-
- "github.com/stretchr/testify/require"
- "gitlab.com/gitlab-org/gitaly/v16/internal/git/gittest"
- "gitlab.com/gitlab-org/gitaly/v16/internal/gitaly/config"
- "gitlab.com/gitlab-org/gitaly/v16/internal/gitaly/storage"
- "gitlab.com/gitlab-org/gitaly/v16/internal/structerr"
- "gitlab.com/gitlab-org/gitaly/v16/internal/testhelper"
- "gitlab.com/gitlab-org/gitaly/v16/proto/go/gitalypb"
- "google.golang.org/grpc/codes"
- "google.golang.org/grpc/status"
-)
-
-func TestRenameRepositorySuccess(t *testing.T) {
- t.Parallel()
-
- ctx := testhelper.Context(t)
-
- cfg, client := setupRepositoryService(t)
- locator := config.NewLocator(cfg)
- originalRepo, originalPath := gittest.CreateRepository(t, ctx, cfg)
- commitID := gittest.WriteCommit(t, cfg, originalPath)
-
- targetPath := "a-new-location"
- _, err := client.RenameRepository(ctx, &gitalypb.RenameRepositoryRequest{
- Repository: originalRepo,
- RelativePath: targetPath,
- })
- require.NoError(t, err)
-
- // A repository should not exist with the previous relative path
- exists, err := client.RepositoryExists(ctx, &gitalypb.RepositoryExistsRequest{
- Repository: originalRepo,
- })
- require.NoError(t, err)
- testhelper.ProtoEqual(t, &gitalypb.RepositoryExistsResponse{Exists: false}, exists)
-
- // A repository should exist with the new relative path
- renamedRepo := &gitalypb.Repository{StorageName: originalRepo.StorageName, RelativePath: targetPath}
- exists, err = client.RepositoryExists(ctx, &gitalypb.RepositoryExistsRequest{
- Repository: renamedRepo,
- })
- require.NoError(t, err)
- testhelper.ProtoEqual(t, &gitalypb.RepositoryExistsResponse{Exists: true}, exists)
-
- if testhelper.IsPraefectEnabled() {
- // Praefect does not move repositories on the disk.
- targetPath = gittest.GetReplicaPath(t, ctx, cfg, renamedRepo)
- }
- newDirectory := filepath.Join(cfg.Storages[0].Path, targetPath)
-
- require.NoError(t, locator.ValidateRepository(&gitalypb.Repository{
- StorageName: cfg.Storages[0].Name,
- RelativePath: targetPath,
- }), "moved Git repository has been corrupted")
- // ensure the git directory that got renamed contains the original commit.
- gittest.RequireObjectExists(t, cfg, newDirectory, commitID)
-}
-
-func TestRenameRepositoryDestinationExists(t *testing.T) {
- t.Parallel()
-
- ctx := testhelper.Context(t)
-
- cfg, client := setupRepositoryService(t)
-
- existingDestinationRepo, destinationRepoPath := gittest.CreateRepository(t, ctx, cfg)
- commitID := gittest.WriteCommit(t, cfg, destinationRepoPath)
- renamedRepo, _ := gittest.CreateRepository(t, ctx, cfg)
-
- _, err := client.RenameRepository(ctx, &gitalypb.RenameRepositoryRequest{
- Repository: renamedRepo,
- RelativePath: existingDestinationRepo.RelativePath,
- })
- testhelper.RequireGrpcCode(t, err, codes.AlreadyExists)
-
- // ensure the git directory that already existed didn't get overwritten
- gittest.RequireObjectExists(t, cfg, destinationRepoPath, commitID)
-}
-
-func TestRenameRepositoryInvalidRequest(t *testing.T) {
- t.Parallel()
-
- ctx := testhelper.Context(t)
-
- cfg, client := setupRepositoryService(t)
- repo, _ := gittest.CreateRepository(t, ctx, cfg)
-
- testCases := []struct {
- desc string
- req *gitalypb.RenameRepositoryRequest
- exp error
- }{
- {
- desc: "empty repository",
- req: &gitalypb.RenameRepositoryRequest{Repository: nil, RelativePath: "/tmp/abc"},
- exp: structerr.NewInvalidArgument("%w", storage.ErrRepositoryNotSet),
- },
- {
- desc: "empty destination relative path",
- req: &gitalypb.RenameRepositoryRequest{Repository: repo, RelativePath: ""},
- exp: status.Error(codes.InvalidArgument, "destination relative path is empty"),
- },
- {
- desc: "destination relative path contains path traversal",
- req: &gitalypb.RenameRepositoryRequest{Repository: repo, RelativePath: "../usr/bin"},
- exp: testhelper.WithInterceptedMetadata(
- structerr.NewInvalidArgument("%w", storage.ErrRelativePathEscapesRoot),
- "relative_path", "../usr/bin",
- ),
- },
- {
- desc: "repository storage doesn't exist",
- req: &gitalypb.RenameRepositoryRequest{Repository: &gitalypb.Repository{StorageName: "stub", RelativePath: repo.RelativePath}, RelativePath: "usr/bin"},
- exp: testhelper.ToInterceptedMetadata(structerr.NewInvalidArgument(
- "%w", storage.NewStorageNotFoundError("stub"),
- )),
- },
- {
- desc: "repository relative path doesn't exist",
- req: &gitalypb.RenameRepositoryRequest{Repository: &gitalypb.Repository{StorageName: repo.StorageName, RelativePath: "stub"}, RelativePath: "non-existent/directory"},
- exp: testhelper.ToInterceptedMetadata(
- structerr.New("%w", storage.NewRepositoryNotFoundError(cfg.Storages[0].Name, "stub")),
- ),
- },
- }
-
- for _, tc := range testCases {
- t.Run(tc.desc, func(t *testing.T) {
- _, err := client.RenameRepository(ctx, tc.req)
- testhelper.RequireGrpcError(t, tc.exp, err)
- })
- }
-}
diff --git a/internal/gitaly/storage/storagemgr/middleware.go b/internal/gitaly/storage/storagemgr/middleware.go
index 62c6c2d42..ac36e972c 100644
--- a/internal/gitaly/storage/storagemgr/middleware.go
+++ b/internal/gitaly/storage/storagemgr/middleware.go
@@ -56,9 +56,6 @@ var nonTransactionalRPCs = map[string]struct{}{
// SetFullPath writes the full path into git config and is the last RPC that writes into the
// git config. Writing into the config won't be supported.
"/gitaly.RepositoryService/SetFullPath": {},
- // RenameRepository is pending removal as we need stable IDs for repositories. Renaming won't
- // be supported with WAL.
- "/gitaly.RepositoryService/RenameRepository": {},
}
// NewUnaryInterceptor returns an unary interceptor that manages a unary RPC's transaction. It starts a transaction
diff --git a/internal/praefect/coordinator.go b/internal/praefect/coordinator.go
index ac1d554f3..0719e68cb 100644
--- a/internal/praefect/coordinator.go
+++ b/internal/praefect/coordinator.go
@@ -95,7 +95,6 @@ var transactionRPCs = map[string]transactionsCondition{
// transactional in the future if the need arises.
"/gitaly.ObjectPoolService/CreateObjectPool": transactionsDisabled,
"/gitaly.ObjectPoolService/DeleteObjectPool": transactionsDisabled,
- "/gitaly.RepositoryService/RenameRepository": transactionsDisabled,
}
// forcePrimaryRoutingRPCs tracks RPCs which need to always get routed to the primary. This should
@@ -162,12 +161,6 @@ func getReplicationDetails(methodName string, m proto.Message) (datastore.Change
"/gitaly.RepositoryService/ReplicateRepository",
"/gitaly.RepositoryService/RestoreRepository":
return datastore.CreateRepo, nil, nil
- case "/gitaly.RepositoryService/RenameRepository":
- req, ok := m.(*gitalypb.RenameRepositoryRequest)
- if !ok {
- return "", nil, fmt.Errorf("protocol changed: for method %q expected message type '%T', got '%T'", methodName, req, m)
- }
- return datastore.RenameRepo, datastore.Params{"RelativePath": req.RelativePath}, nil
default:
return datastore.UpdateRepo, nil, nil
}
@@ -1093,17 +1086,6 @@ func (c *Coordinator) newRequestFinalizer(
if err := c.rs.IncrementGeneration(ctx, repositoryID, primary, updatedSecondaries); err != nil {
return fmt.Errorf("increment generation: %w", err)
}
- case datastore.RenameRepo:
- // Renaming a repository is not idempotent on Gitaly's side. This combined with a failure here results in a problematic state,
- // where the client receives an error but can't retry the call as the repository has already been moved on the primary.
- // Ideally the rename RPC should copy the repository instead of moving so the client can retry if this failed.
- if err := c.rs.RenameRepository(ctx, virtualStorage, targetRepo.GetRelativePath(), primary, params["RelativePath"].(string)); err != nil {
- if !errors.Is(err, datastore.ErrRepositoryNotFound) {
- return fmt.Errorf("rename repository: %w", err)
- }
-
- c.logger.WithError(err).InfoContext(ctx, "renamed repository does not have a store entry")
- }
case datastore.CreateRepo:
repositorySpecificPrimariesEnabled := c.conf.Failover.ElectionStrategy == config.ElectionStrategyPerRepository
variableReplicationFactorEnabled := repositorySpecificPrimariesEnabled &&
diff --git a/internal/praefect/coordinator_test.go b/internal/praefect/coordinator_test.go
index 8f781ed01..7c91eea8d 100644
--- a/internal/praefect/coordinator_test.go
+++ b/internal/praefect/coordinator_test.go
@@ -2900,11 +2900,6 @@ func TestNewRequestFinalizer_contextIsDisjointedFromTheRPC(t *testing.T) {
errMsg: "increment generation: error",
},
{
- desc: "rename repository receives suppressed cancellation",
- change: datastore.RenameRepo,
- errMsg: "rename repository: error",
- },
- {
desc: "enqueue receives suppressed cancellation",
errMsg: "enqueue replication event: error",
enqueueError: err,
@@ -2929,10 +2924,6 @@ func TestNewRequestFinalizer_contextIsDisjointedFromTheRPC(t *testing.T) {
requireSuppressedCancellation(t, ctx)
return err
},
- RenameRepositoryFunc: func(ctx context.Context, _, _, _, _ string) error {
- requireSuppressedCancellation(t, ctx)
- return err
- },
CreateRepositoryFunc: func(ctx context.Context, _ int64, _, _, _, _ string, _, _ []string, _, _ bool) error {
requireSuppressedCancellation(t, ctx)
return err
diff --git a/internal/praefect/datastore/datastore.go b/internal/praefect/datastore/datastore.go
index dfa9d586e..52633a9ad 100644
--- a/internal/praefect/datastore/datastore.go
+++ b/internal/praefect/datastore/datastore.go
@@ -46,14 +46,12 @@ const (
DeleteRepo = ChangeType("delete")
// DeleteReplica change type indicates that the targeted replica is due for deletion.
DeleteReplica = ChangeType("delete_replica")
- // RenameRepo is when a replication renames repo
- RenameRepo = ChangeType("rename")
)
// GetAllChangeTypes is used to define and provide all the various ChangeType
// constants. This is useful to iterate over and set labels in metrics.
func GetAllChangeTypes() []ChangeType {
- return []ChangeType{UpdateRepo, CreateRepo, DeleteRepo, DeleteReplica, RenameRepo}
+ return []ChangeType{UpdateRepo, CreateRepo, DeleteRepo, DeleteReplica}
}
func (ct ChangeType) String() string {
diff --git a/internal/praefect/datastore/queue_test.go b/internal/praefect/datastore/queue_test.go
index c7ea3d8bc..0c07a28b7 100644
--- a/internal/praefect/datastore/queue_test.go
+++ b/internal/praefect/datastore/queue_test.go
@@ -252,12 +252,12 @@ func TestPostgresReplicationEventQueue_EnqueueMultiple(t *testing.T) {
eventType2 := ReplicationEvent{
Job: ReplicationJob{
- Change: RenameRepo,
+ Change: UpdateRepo,
RelativePath: "/project/path-1",
TargetNodeStorage: "gitaly-2",
SourceNodeStorage: "",
VirtualStorage: "praefect-0",
- Params: Params{"RelativePath": "/project/path-1-renamed"},
+ Params: nil,
},
}
@@ -315,12 +315,12 @@ func TestPostgresReplicationEventQueue_EnqueueMultiple(t *testing.T) {
Attempt: 3,
LockID: "praefect-0|gitaly-2|/project/path-1",
Job: ReplicationJob{
- Change: RenameRepo,
+ Change: UpdateRepo,
RelativePath: "/project/path-1",
TargetNodeStorage: "gitaly-2",
SourceNodeStorage: "",
VirtualStorage: "praefect-0",
- Params: Params{"RelativePath": "/project/path-1-renamed"},
+ Params: nil,
},
}
@@ -429,12 +429,12 @@ func TestPostgresReplicationEventQueue_DequeueMultiple(t *testing.T) {
eventType3 := ReplicationEvent{
Job: ReplicationJob{
- Change: RenameRepo,
+ Change: UpdateRepo,
RelativePath: "/project/path-2",
TargetNodeStorage: "gitaly-1",
SourceNodeStorage: "gitaly-0",
VirtualStorage: "praefect",
- Params: Params{"RelativePath": "/project/path-2-renamed"},
+ Params: nil,
},
}
diff --git a/internal/praefect/datastore/repository_store.go b/internal/praefect/datastore/repository_store.go
index 500b1281c..77dce6ba0 100644
--- a/internal/praefect/datastore/repository_store.go
+++ b/internal/praefect/datastore/repository_store.go
@@ -83,13 +83,6 @@ type RepositoryStore interface {
DeleteAllRepositories(ctx context.Context, virtualStorage string) error
// DeleteReplica deletes a replica of a repository from a storage without affecting other state in the virtual storage.
DeleteReplica(ctx context.Context, repositoryID int64, storage string) error
- // RenameRepository updates a repository's relative path. It renames the virtual storage wide record as well
- // as the storage's which is calling it. Returns ErrRepositoryNotFound when trying to rename a repository
- // which has no record in the virtual storage or the storage.
- RenameRepository(ctx context.Context, virtualStorage, relativePath, storage, newRelativePath string) error
- // RenameRepositoryInPlace renames the repository in the database without changing the replica path. This will replace
- // RenameRepository which can be removed in a later release.
- RenameRepositoryInPlace(ctx context.Context, virtualStorage, relativePath, newRelativePath string) error
// GetConsistentStoragesByRepositoryID returns the replica path and the set of up to date storages for the given repository keyed by repository ID.
GetConsistentStoragesByRepositoryID(ctx context.Context, repositoryID int64) (string, *datastructure.Set[string], error)
ConsistentStoragesGetter
@@ -534,71 +527,6 @@ AND storage = $2
return nil
}
-// RenameRepositoryInPlace renames the repository in the database without changing the replica path. This will replace
-// RenameRepository which can be removed in a later release.
-func (rs *PostgresRepositoryStore) RenameRepositoryInPlace(ctx context.Context, virtualStorage, relativePath, newRelativePath string) error {
- result, err := rs.db.ExecContext(ctx, `
-WITH repository AS (
- UPDATE repositories
- SET relative_path = $3
- WHERE virtual_storage = $1
- AND relative_path = $2
- RETURNING repository_id
-)
-
-UPDATE storage_repositories
-SET relative_path = $3
-WHERE repository_id = (SELECT repository_id FROM repository)
- `, virtualStorage, relativePath, newRelativePath)
- if err != nil {
- if glsql.IsUniqueViolation(err, "repository_lookup_index") {
- return ErrRepositoryAlreadyExists
- }
-
- return fmt.Errorf("query: %w", err)
- }
-
- if rowsAffected, err := result.RowsAffected(); err != nil {
- return fmt.Errorf("rows affected: %w", err)
- } else if rowsAffected == 0 {
- return ErrRepositoryNotFound
- }
-
- return nil
-}
-
-//nolint:revive // This is unintentionally missing documentation.
-func (rs *PostgresRepositoryStore) RenameRepository(ctx context.Context, virtualStorage, relativePath, storage, newRelativePath string) error {
- const q = `
-WITH repo AS (
- UPDATE repositories
- SET relative_path = $4,
- replica_path = $4
- WHERE virtual_storage = $1
- AND relative_path = $2
-)
-
-UPDATE storage_repositories
-SET relative_path = $4
-WHERE virtual_storage = $1
-AND relative_path = $2
-AND storage = $3
-`
-
- result, err := rs.db.ExecContext(ctx, q, virtualStorage, relativePath, storage, newRelativePath)
- if err != nil {
- return err
- }
-
- if n, err := result.RowsAffected(); err != nil {
- return err
- } else if n == 0 {
- return ErrRepositoryNotFound
- }
-
- return err
-}
-
// GetConsistentStoragesByRepositoryID returns the replica path and the set of up to date storages for the given repository keyed by repository ID.
func (rs *PostgresRepositoryStore) GetConsistentStoragesByRepositoryID(ctx context.Context, repositoryID int64) (string, *datastructure.Set[string], error) {
return rs.getConsistentStorages(ctx, `
diff --git a/internal/praefect/datastore/repository_store_mock.go b/internal/praefect/datastore/repository_store_mock.go
index cd0877040..4a4350598 100644
--- a/internal/praefect/datastore/repository_store_mock.go
+++ b/internal/praefect/datastore/repository_store_mock.go
@@ -18,8 +18,6 @@ type MockRepositoryStore struct {
SetAuthoritativeReplicaFunc func(ctx context.Context, virtualStorage, relativePath, storage string) error
DeleteRepositoryFunc func(ctx context.Context, virtualStorage, relativePath string) (string, []string, error)
DeleteReplicaFunc func(ctx context.Context, repositoryID int64, storage string) error
- RenameRepositoryInPlaceFunc func(ctx context.Context, virtualStorage, relativePath, newRelativePath string) error
- RenameRepositoryFunc func(ctx context.Context, virtualStorage, relativePath, storage, newRelativePath string) error
GetConsistentStoragesByRepositoryIDFunc func(ctx context.Context, repositoryID int64) (string, *datastructure.Set[string], error)
GetConsistentStoragesFunc func(ctx context.Context, virtualStorage, relativePath string) (string, *datastructure.Set[string], error)
GetPartiallyAvailableRepositoriesFunc func(ctx context.Context, virtualStorage string) ([]RepositoryMetadata, error)
@@ -104,20 +102,6 @@ func (m MockRepositoryStore) DeleteReplica(ctx context.Context, repositoryID int
return m.DeleteReplicaFunc(ctx, repositoryID, storage)
}
-// RenameRepositoryInPlace runs the mock's RenameRepositoryInPlaceFunc.
-func (m MockRepositoryStore) RenameRepositoryInPlace(ctx context.Context, virtualStorage, relativePath, newRelativePath string) error {
- return m.RenameRepositoryInPlaceFunc(ctx, virtualStorage, relativePath, newRelativePath)
-}
-
-//nolint:revive // This is unintentionally missing documentation.
-func (m MockRepositoryStore) RenameRepository(ctx context.Context, virtualStorage, relativePath, storage, newRelativePath string) error {
- if m.RenameRepositoryFunc == nil {
- return nil
- }
-
- return m.RenameRepositoryFunc(ctx, virtualStorage, relativePath, storage, newRelativePath)
-}
-
// GetConsistentStoragesByRepositoryID returns result of execution of the GetConsistentStoragesByRepositoryIDFunc field if it is set or an empty map.
func (m MockRepositoryStore) GetConsistentStoragesByRepositoryID(ctx context.Context, repositoryID int64) (string, *datastructure.Set[string], error) {
if m.GetConsistentStoragesFunc == nil {
diff --git a/internal/praefect/datastore/repository_store_test.go b/internal/praefect/datastore/repository_store_test.go
index 136e97ecb..577ffc0e7 100644
--- a/internal/praefect/datastore/repository_store_test.go
+++ b/internal/praefect/datastore/repository_store_test.go
@@ -292,35 +292,16 @@ func TestRepositoryStore_Postgres(t *testing.T) {
},
)
- require.NoError(t, rs.RenameRepository(ctx, vs, "original-path", "storage-1", "new-path"))
+ require.NoError(t, rs.SetGeneration(ctx, 1, "storage-2", "original-path", 1))
requireState(t, ctx, db,
virtualStorageState{
"virtual-storage-1": {
- "new-path": {repositoryID: 1, primary: "storage-1", replicaPath: "new-path"},
+ "original-path": {repositoryID: 1, primary: "storage-1", replicaPath: "replica-path"},
},
},
storageState{
"virtual-storage-1": {
"original-path": {
- "storage-2": {repositoryID: 1, generation: 0},
- },
- "new-path": {
- "storage-1": {repositoryID: 1, generation: 0},
- },
- },
- },
- )
-
- require.NoError(t, rs.SetGeneration(ctx, 1, "storage-2", "new-path", 1))
- requireState(t, ctx, db,
- virtualStorageState{
- "virtual-storage-1": {
- "new-path": {repositoryID: 1, primary: "storage-1", replicaPath: "new-path"},
- },
- },
- storageState{
- "virtual-storage-1": {
- "new-path": {
"storage-1": {repositoryID: 1, generation: 0},
"storage-2": {repositoryID: 1, generation: 1},
},
@@ -892,113 +873,6 @@ func TestRepositoryStore_Postgres(t *testing.T) {
})
})
- t.Run("RenameRepositoryInPlace", func(t *testing.T) {
- t.Run("rename non-existing", func(t *testing.T) {
- rs := newRepositoryStore(t, nil)
-
- require.Equal(t,
- ErrRepositoryNotFound,
- rs.RenameRepositoryInPlace(ctx, vs, repo, "new-relative-path"),
- )
- })
-
- t.Run("destination exists", func(t *testing.T) {
- rs := newRepositoryStore(t, nil)
-
- require.NoError(t, rs.CreateRepository(ctx, 1, vs, "relative-path-1", "replica-path-1", "primary", nil, nil, true, false))
- require.NoError(t, rs.CreateRepository(ctx, 2, vs, "relative-path-2", "replica-path-2", "primary", nil, nil, true, false))
-
- require.Equal(t,
- ErrRepositoryAlreadyExists,
- rs.RenameRepositoryInPlace(ctx, vs, "relative-path-1", "relative-path-2"),
- )
- })
-
- t.Run("successfully renamed", func(t *testing.T) {
- rs := newRepositoryStore(t, nil)
-
- require.NoError(t, rs.CreateRepository(ctx, 1, vs, "original-relative-path", "original-replica-path", "primary", nil, nil, false, false))
- require.NoError(t, rs.RenameRepositoryInPlace(ctx, vs, "original-relative-path", "renamed-relative-path"))
- requireState(t, ctx, db,
- virtualStorageState{
- vs: {
- "renamed-relative-path": {repositoryID: 1, replicaPath: "original-replica-path"},
- },
- },
- storageState{
- vs: {
- "renamed-relative-path": {
- "primary": {repositoryID: 1},
- },
- },
- },
- )
- })
- })
-
- t.Run("RenameRepository", func(t *testing.T) {
- t.Run("rename non-existing", func(t *testing.T) {
- rs := newRepositoryStore(t, nil)
-
- require.Equal(t,
- ErrRepositoryNotFound,
- rs.RenameRepository(ctx, vs, repo, stor, "repository-2"),
- )
- })
-
- t.Run("rename existing", func(t *testing.T) {
- rs := newRepositoryStore(t, nil)
-
- require.NoError(t, rs.CreateRepository(ctx, 1, vs, "renamed-all", "replica-path-1", "storage-1", nil, nil, false, false))
- require.NoError(t, rs.CreateRepository(ctx, 2, vs, "renamed-some", "replica-path-2", "storage-1", []string{"storage-2"}, nil, false, false))
-
- requireState(t, ctx, db,
- virtualStorageState{
- "virtual-storage-1": {
- "renamed-all": {repositoryID: 1, replicaPath: "replica-path-1"},
- "renamed-some": {repositoryID: 2, replicaPath: "replica-path-2"},
- },
- },
- storageState{
- "virtual-storage-1": {
- "renamed-all": {
- "storage-1": {repositoryID: 1, generation: 0},
- },
- "renamed-some": {
- "storage-1": {repositoryID: 2, generation: 0},
- "storage-2": {repositoryID: 2, generation: 0},
- },
- },
- },
- )
-
- require.NoError(t, rs.RenameRepository(ctx, vs, "renamed-all", "storage-1", "renamed-all-new"))
- require.NoError(t, rs.RenameRepository(ctx, vs, "renamed-some", "storage-1", "renamed-some-new"))
-
- requireState(t, ctx, db,
- virtualStorageState{
- "virtual-storage-1": {
- "renamed-all-new": {repositoryID: 1, replicaPath: "renamed-all-new"},
- "renamed-some-new": {repositoryID: 2, replicaPath: "renamed-some-new"},
- },
- },
- storageState{
- "virtual-storage-1": {
- "renamed-all-new": {
- "storage-1": {repositoryID: 1, generation: 0},
- },
- "renamed-some-new": {
- "storage-1": {repositoryID: 2, generation: 0},
- },
- "renamed-some": {
- "storage-2": {repositoryID: 2, generation: 0},
- },
- },
- },
- )
- })
- })
-
t.Run("GetConsistentStorages", func(t *testing.T) {
rs := newRepositoryStore(t, map[string][]string{
vs: {"primary", "consistent-secondary", "inconsistent-secondary", "no-record"},
@@ -1109,40 +983,6 @@ func TestRepositoryStore_Postgres(t *testing.T) {
require.Empty(t, secondaries)
require.Empty(t, replicaPath)
})
-
- t.Run("replicas pending rename are considered outdated", func(t *testing.T) {
- rs := newRepositoryStore(t, nil)
-
- require.NoError(t, rs.CreateRepository(ctx, 1, vs, "original-path", "replica-path", "storage-1", []string{"storage-2"}, nil, true, false))
- replicaPath, storages, err := rs.GetConsistentStorages(ctx, vs, "original-path")
- require.NoError(t, err)
- require.Equal(t, "replica-path", replicaPath)
- require.ElementsMatch(t, []string{"storage-1", "storage-2"}, storages.Values())
- replicaPath, storages, err = rs.GetConsistentStoragesByRepositoryID(ctx, 1)
- require.NoError(t, err)
- require.Equal(t, "replica-path", replicaPath)
- require.ElementsMatch(t, []string{"storage-1", "storage-2"}, storages.Values())
-
- require.NoError(t, rs.RenameRepository(ctx, vs, "original-path", "storage-1", "new-path"))
- replicaPath, storages, err = rs.GetConsistentStorages(ctx, vs, "new-path")
- require.NoError(t, err)
- require.Equal(t, "new-path", replicaPath)
- require.Equal(t, []string{"storage-1"}, storages.Values())
- replicaPath, storages, err = rs.GetConsistentStoragesByRepositoryID(ctx, 1)
- require.NoError(t, err)
- require.Equal(t, "new-path", replicaPath)
- require.Equal(t, []string{"storage-1"}, storages.Values())
-
- require.NoError(t, rs.RenameRepository(ctx, vs, "original-path", "storage-2", "new-path"))
- replicaPath, storages, err = rs.GetConsistentStorages(ctx, vs, "new-path")
- require.NoError(t, err)
- require.Equal(t, "new-path", replicaPath)
- require.ElementsMatch(t, []string{"storage-1", "storage-2"}, storages.Values())
- replicaPath, storages, err = rs.GetConsistentStoragesByRepositoryID(ctx, 1)
- require.NoError(t, err)
- require.Equal(t, "new-path", replicaPath)
- require.ElementsMatch(t, []string{"storage-1", "storage-2"}, storages.Values())
- })
})
t.Run("DeleteInvalidRepository", func(t *testing.T) {
diff --git a/internal/praefect/reconciler/reconciler_test.go b/internal/praefect/reconciler/reconciler_test.go
index 0a30858cb..230447d88 100644
--- a/internal/praefect/reconciler/reconciler_test.go
+++ b/internal/praefect/reconciler/reconciler_test.go
@@ -384,7 +384,6 @@ func TestReconciler(t *testing.T) {
},
[]datastore.ChangeType{
datastore.DeleteRepo,
- datastore.RenameRepo,
},
datastore.ReplicationJob{
VirtualStorage: "virtual-storage-1",
@@ -1163,74 +1162,3 @@ func TestReconciler(t *testing.T) {
})
}
}
-
-func TestReconciler_renames(t *testing.T) {
- ctx := testhelper.Context(t)
-
- db := testdb.New(t)
-
- for _, tc := range []struct {
- desc string
- latestStorage string
- expectedJob datastore.ReplicationJob
- }{
- {
- desc: "replicas pending rename are targeted by updates",
- latestStorage: "storage-1",
- expectedJob: datastore.ReplicationJob{
- RepositoryID: 1,
- VirtualStorage: "virtual-storage",
- RelativePath: "new-path",
- SourceNodeStorage: "storage-1",
- TargetNodeStorage: "storage-2",
- Change: "update",
- },
- },
- {
- desc: "replicas pending rename are not used as a source",
- latestStorage: "storage-2",
- },
- } {
- t.Run(tc.desc, func(t *testing.T) {
- db.TruncateAll(t)
-
- configuredStorages := map[string][]string{"virtual-storage": {"storage-1", "storage-2"}}
-
- reconciler := NewReconciler(
- testhelper.SharedLogger(t),
- db,
- praefect.StaticHealthChecker(configuredStorages),
- configuredStorages,
- prometheus.DefBuckets,
- )
-
- rs := datastore.NewPostgresRepositoryStore(db, configuredStorages)
- require.NoError(t, rs.CreateRepository(ctx, 1, "virtual-storage", "original-path", "replica-path", "storage-1", []string{"storage-2"}, nil, true, false))
- require.NoError(t, rs.SetGeneration(ctx, 1, tc.latestStorage, "original-path", 1))
-
- require.NoError(t, rs.RenameRepository(ctx, "virtual-storage", "original-path", "storage-1", "new-path"))
-
- runCtx, cancelRun := context.WithCancel(ctx)
- var resetted bool
- ticker := helper.NewManualTicker()
- ticker.ResetFunc = func() {
- if resetted {
- cancelRun()
- return
- }
-
- resetted = true
- ticker.Tick()
- }
-
- require.Equal(t, context.Canceled, reconciler.Run(runCtx, ticker))
-
- var job datastore.ReplicationJob
- if err := db.QueryRowContext(ctx, `SELECT job FROM replication_queue`).Scan(&job); err != nil {
- require.Equal(t, sql.ErrNoRows, err)
- }
-
- require.Equal(t, tc.expectedJob, job)
- })
- }
-}
diff --git a/internal/praefect/rename_repository.go b/internal/praefect/rename_repository.go
deleted file mode 100644
index 089ab557c..000000000
--- a/internal/praefect/rename_repository.go
+++ /dev/null
@@ -1,78 +0,0 @@
-package praefect
-
-import (
- "errors"
- "fmt"
-
- "gitlab.com/gitlab-org/gitaly/v16/internal/gitaly/storage"
- "gitlab.com/gitlab-org/gitaly/v16/internal/praefect/datastore"
- "gitlab.com/gitlab-org/gitaly/v16/internal/structerr"
- "gitlab.com/gitlab-org/gitaly/v16/proto/go/gitalypb"
- "google.golang.org/grpc"
-)
-
-func validateRenameRepositoryRequest(req *gitalypb.RenameRepositoryRequest, virtualStorages map[string]struct{}) error {
- // These checks are not strictly necessary but they exist to keep retain compatibility with
- // Gitaly's tested behavior.
- repository := req.GetRepository()
- if repository == nil || repository.GetStorageName() == "" && repository.GetRelativePath() == "" {
- return structerr.NewInvalidArgument("%w", storage.ErrRepositoryNotSet)
- } else if repository.GetStorageName() == "" {
- return structerr.NewInvalidArgument("%w", storage.ErrStorageNotSet)
- } else if repository.GetRelativePath() == "" {
- return structerr.NewInvalidArgument("%w", storage.ErrRepositoryPathNotSet)
- } else if _, ok := virtualStorages[repository.GetStorageName()]; !ok {
- return storage.NewStorageNotFoundError(repository.GetStorageName())
- } else if req.GetRelativePath() == "" {
- return structerr.NewInvalidArgument("destination relative path is empty")
- } else if _, err := storage.ValidateRelativePath("/fake-root", req.GetRelativePath()); err != nil {
- // Gitaly uses ValidateRelativePath to verify there are no traversals, so we use the same function
- // here. Praefect is not susceptible to path traversals as it generates its own disk paths but we
- // do this to retain API compatibility with Gitaly. ValidateRelativePath checks for traversals by
- // seeing whether the relative path escapes the root directory. It's not possible to traverse up
- // from the /, so the traversals in the path wouldn't be caught. To allow for the check to work,
- // we use the /fake-root directory simply to notice if there were traversals in the path.
- return structerr.NewInvalidArgument("%w", err).WithMetadata("relative_path", req.GetRelativePath())
- }
-
- return nil
-}
-
-// RenameRepositoryHandler handles /gitaly.RepositoryService/RenameRepository calls by renaming
-// the repository in the lookup table stored in the database.
-func RenameRepositoryHandler(virtualStoragesNames []string, rs datastore.RepositoryStore) grpc.StreamHandler {
- virtualStorages := make(map[string]struct{}, len(virtualStoragesNames))
- for _, virtualStorage := range virtualStoragesNames {
- virtualStorages[virtualStorage] = struct{}{}
- }
-
- return func(srv interface{}, stream grpc.ServerStream) error {
- var req gitalypb.RenameRepositoryRequest
- if err := stream.RecvMsg(&req); err != nil {
- return fmt.Errorf("receive request: %w", err)
- }
-
- if err := validateRenameRepositoryRequest(&req, virtualStorages); err != nil {
- return err
- }
-
- if err := rs.RenameRepositoryInPlace(stream.Context(),
- req.GetRepository().GetStorageName(),
- req.GetRepository().GetRelativePath(),
- req.GetRelativePath(),
- ); err != nil {
- if errors.Is(err, datastore.ErrRepositoryNotFound) {
- return storage.NewRepositoryNotFoundError(
- req.GetRepository().GetStorageName(),
- req.GetRepository().GetRelativePath(),
- )
- } else if errors.Is(err, datastore.ErrRepositoryAlreadyExists) {
- return structerr.NewAlreadyExists("target repo exists already")
- }
-
- return structerr.NewInternal("%w", err)
- }
-
- return stream.SendMsg(&gitalypb.RenameRepositoryResponse{})
- }
-}
diff --git a/internal/praefect/replicator.go b/internal/praefect/replicator.go
index b19d17481..26e47b26b 100644
--- a/internal/praefect/replicator.go
+++ b/internal/praefect/replicator.go
@@ -27,8 +27,6 @@ type Replicator interface {
Replicate(ctx context.Context, event datastore.ReplicationEvent, source, target *grpc.ClientConn) error
// Destroy will remove the target repo on the specified target connection
Destroy(ctx context.Context, event datastore.ReplicationEvent, target *grpc.ClientConn) error
- // Rename will rename(move) the target repo on the specified target connection
- Rename(ctx context.Context, event datastore.ReplicationEvent, target *grpc.ClientConn) error
}
type defaultReplicator struct {
@@ -166,48 +164,6 @@ func (dr defaultReplicator) Destroy(ctx context.Context, event datastore.Replica
return nil
}
-func (dr defaultReplicator) Rename(ctx context.Context, event datastore.ReplicationEvent, targetCC *grpc.ClientConn) error {
- targetRepo := &gitalypb.Repository{
- StorageName: event.Job.TargetNodeStorage,
- RelativePath: event.Job.RelativePath,
- }
-
- repoSvcClient := gitalypb.NewRepositoryServiceClient(targetCC)
-
- val, found := event.Job.Params["RelativePath"]
- if !found {
- return errors.New("no 'RelativePath' parameter for rename")
- }
-
- relativePath, ok := val.(string)
- if !ok {
- return fmt.Errorf("parameter 'RelativePath' has unexpected type: %T", relativePath)
- }
-
- if _, err := repoSvcClient.RenameRepository(ctx, &gitalypb.RenameRepositoryRequest{
- Repository: targetRepo,
- RelativePath: relativePath,
- }); err != nil {
- return err
- }
-
- // If the repository was moved but this fails, we'll have a stale record on the storage but it is missing from the
- // virtual storage. We can later schedule a deletion to fix the situation. The newly named repository's record
- // will be present once a replication job arrives for it.
- if err := dr.rs.RenameRepository(ctx,
- event.Job.VirtualStorage, event.Job.RelativePath, event.Job.TargetNodeStorage, relativePath); err != nil {
- if !errors.Is(err, datastore.ErrRepositoryNotFound) {
- return err
- }
-
- dr.log.WithField(correlation.FieldName, correlation.ExtractFromContext(ctx)).
- WithError(err).
- Info("replicated repository rename does not have a store entry")
- }
-
- return nil
-}
-
// ReplMgr is a replication manager for handling replication jobs
type ReplMgr struct {
log log.Logger
@@ -585,13 +541,6 @@ func (r ReplMgr) backfillReplicaPath(ctx context.Context, event datastore.Replic
// for backwards compatibility.
case event.Job.Change == datastore.DeleteRepo:
fallthrough
- // RenameRepo doesn't need to use repository ID as the RenameRepository RPC
- // call will be intercepted in 14.6 by Praefect to perform an atomic rename in
- // the database. Any jobs still in flight are from 14.5 and older, and should be
- // handled in the old manner. We'll use the relative path from the replication job
- // for the backwards compatible handling.
- case event.Job.Change == datastore.RenameRepo:
- return event.Job.RelativePath, nil
default:
replicaPath, err := r.repositoryStore.GetReplicaPath(ctx, event.Job.RepositoryID)
if err != nil {
@@ -646,8 +595,6 @@ func (r ReplMgr) processReplicationEvent(ctx context.Context, event datastore.Re
err = r.replicator.Replicate(ctx, event, source.Connection, targetCC)
case datastore.DeleteRepo, datastore.DeleteReplica:
err = r.replicator.Destroy(ctx, event, targetCC)
- case datastore.RenameRepo:
- err = r.replicator.Rename(ctx, event, targetCC)
default:
err = fmt.Errorf("unknown replication change type encountered: %q", event.Job.Change)
}
diff --git a/internal/praefect/replicator_test.go b/internal/praefect/replicator_test.go
index 369064ee6..c3f3c5797 100644
--- a/internal/praefect/replicator_test.go
+++ b/internal/praefect/replicator_test.go
@@ -659,14 +659,14 @@ func testProcessBacklogSuccess(t *testing.T, ctx context.Context) {
ackIDs, err := queue.Acknowledge(ctx, state, ids)
if len(ids) > 0 {
assert.Equal(t, datastore.JobStateCompleted, state, "no fails expected")
- assert.Equal(t, []uint64{1, 2, 3}, ids, "all jobs must be processed at once")
+ assert.Equal(t, []uint64{1}, ids, "all jobs must be processed at once")
}
return ackIDs, err
})
var healthUpdated int32
queueInterceptor.OnStartHealthUpdate(func(ctx context.Context, trigger <-chan time.Time, events []datastore.ReplicationEvent) error {
- assert.Len(t, events, 3)
+ assert.Len(t, events, 1)
atomic.AddInt32(&healthUpdated, 1)
return nil
})
@@ -686,42 +686,6 @@ func testProcessBacklogSuccess(t *testing.T, ctx context.Context) {
_, err := queueInterceptor.Enqueue(ctx, eventType1)
require.NoError(t, err)
- renameTo1 := filepath.Join(testRepo.GetRelativePath(), "..", filepath.Base(testRepo.GetRelativePath())+"-mv1")
- fullNewPath1 := filepath.Join(backupCfg.Storages[0].Path, renameTo1)
-
- renameTo2 := filepath.Join(renameTo1, "..", filepath.Base(testRepo.GetRelativePath())+"-mv2")
-
- // Rename replication job
- eventType2 := datastore.ReplicationEvent{
- Job: datastore.ReplicationJob{
- Change: datastore.RenameRepo,
- RelativePath: testRepo.GetRelativePath(),
- TargetNodeStorage: secondary.Storage,
- SourceNodeStorage: primary.Storage,
- VirtualStorage: conf.VirtualStorages[0].Name,
- Params: datastore.Params{"RelativePath": renameTo1},
- },
- }
-
- _, err = queueInterceptor.Enqueue(ctx, eventType2)
- require.NoError(t, err)
-
- // Rename replication job
- eventType3 := datastore.ReplicationEvent{
- Job: datastore.ReplicationJob{
- Change: datastore.RenameRepo,
- RelativePath: renameTo1,
- TargetNodeStorage: secondary.Storage,
- SourceNodeStorage: primary.Storage,
- VirtualStorage: conf.VirtualStorages[0].Name,
- Params: datastore.Params{"RelativePath": renameTo2},
- },
- }
- require.NoError(t, err)
-
- _, err = queueInterceptor.Enqueue(ctx, eventType3)
- require.NoError(t, err)
-
logEntry := testhelper.SharedLogger(t)
nodeMgr, err := nodes.NewManager(logEntry, conf, nil, nil, promtest.NewMockHistogramVec(), protoregistry.GitalyProtoPreregistered, nil, nil, nil)
@@ -748,16 +712,15 @@ func testProcessBacklogSuccess(t *testing.T, ctx context.Context) {
for _, params := range i.GetAcknowledge() {
ids = append(ids, params.IDs...)
}
- return len(ids) == 3
+ return len(ids) == 1
}))
cancel()
<-replMgrDone
- require.NoDirExists(t, fullNewPath1, "repository must be moved from %q to the new location", fullNewPath1)
require.NoError(t, backupLocator.ValidateRepository(&gitalypb.Repository{
StorageName: backupCfg.Storages[0].Name,
- RelativePath: renameTo2,
- }), "repository must exist at new last RenameRepository location")
+ RelativePath: testRepo.GetRelativePath(),
+ }), "repository must exist at the relative path")
}
func TestReplMgrProcessBacklog_OnlyHealthyNodes(t *testing.T) {
diff --git a/internal/praefect/server.go b/internal/praefect/server.go
index c54c2b502..4792e4655 100644
--- a/internal/praefect/server.go
+++ b/internal/praefect/server.go
@@ -187,7 +187,6 @@ func NewGRPCServer(
proxy.RegisterStreamHandlers(srv, "gitaly.RepositoryService", map[string]grpc.StreamHandler{
"RemoveAll": RemoveAllHandler(deps.RepositoryStore, deps.Conns),
"RemoveRepository": RemoveRepositoryHandler(deps.RepositoryStore, deps.Logger, deps.Conns),
- "RenameRepository": RenameRepositoryHandler(deps.Config.VirtualStorageNames(), deps.RepositoryStore),
"ReplicateRepository": ReplicateRepositoryHandler(deps.Coordinator),
"RepositoryExists": RepositoryExistsHandler(deps.RepositoryStore),
})
diff --git a/internal/praefect/server_test.go b/internal/praefect/server_test.go
index 16b51b084..c0bc070eb 100644
--- a/internal/praefect/server_test.go
+++ b/internal/praefect/server_test.go
@@ -5,7 +5,6 @@ import (
"context"
"errors"
"io"
- "math/rand"
"net"
"sort"
"strings"
@@ -578,118 +577,6 @@ func TestRemoveRepository(t *testing.T) {
verifyReposExistence(t, codes.NotFound)
}
-func TestRenameRepository(t *testing.T) {
- gitalyStorages := []string{"gitaly-1", "gitaly-2", "gitaly-3"}
- praefectCfg := config.Config{
- VirtualStorages: []*config.VirtualStorage{{Name: "praefect"}},
- Failover: config.Failover{Enabled: true, ElectionStrategy: config.ElectionStrategyPerRepository},
- }
-
- for _, storageName := range gitalyStorages {
- cfgBuilder := testcfg.NewGitalyCfgBuilder(testcfg.WithStorages(storageName))
- gitalyCfg := cfgBuilder.Build(t)
- gitalyAddr := testserver.RunGitalyServer(t, gitalyCfg, setup.RegisterAll, testserver.WithDisablePraefect())
-
- praefectCfg.VirtualStorages[0].Nodes = append(praefectCfg.VirtualStorages[0].Nodes, &config.Node{
- Storage: storageName,
- Address: gitalyAddr,
- Token: gitalyCfg.Auth.Token,
- })
- }
-
- evq := datastore.NewReplicationEventQueueInterceptor(datastore.NewPostgresReplicationEventQueue(testdb.New(t)))
-
- db := testdb.New(t)
-
- rs := datastore.NewPostgresRepositoryStore(db, nil)
-
- logger := testhelper.SharedLogger(t)
- txManager := transactions.NewManager(praefectCfg, logger)
- clientHandshaker := backchannel.NewClientHandshaker(
- logger,
- NewBackchannelServerFactory(
- logger,
- transaction.NewServer(txManager),
- nil,
- ),
- backchannel.DefaultConfiguration(),
- )
-
- ctx := testhelper.Context(t)
- nodeSet, err := DialNodes(ctx, praefectCfg.VirtualStorages, nil, nil, clientHandshaker, nil, logger)
- require.NoError(t, err)
- defer nodeSet.Close()
-
- cc, _, cleanup := RunPraefectServer(t, ctx, praefectCfg, BuildOptions{
- WithQueue: evq,
- WithRepoStore: rs,
- WithRouter: NewPerRepositoryRouter(
- nodeSet.Connections(),
- nodes.NewPerRepositoryElector(logger, db),
- StaticHealthChecker(praefectCfg.StorageNames()),
- NewLockedRandom(rand.New(rand.NewSource(0))),
- rs,
- datastore.NewAssignmentStore(db, praefectCfg.StorageNames()),
- rs,
- nil,
- ),
- WithTxMgr: txManager,
- })
- t.Cleanup(cleanup)
-
- virtualRepo1, _ := gittest.CreateRepository(t, ctx, gconfig.Cfg{
- Storages: []gconfig.Storage{{Name: "praefect"}},
- }, gittest.CreateRepositoryConfig{ClientConn: cc})
-
- virtualRepo2, _ := gittest.CreateRepository(t, ctx, gconfig.Cfg{
- Storages: []gconfig.Storage{{Name: "praefect"}},
- }, gittest.CreateRepositoryConfig{ClientConn: cc})
-
- const newRelativePath = "unused-relative-path"
-
- repoServiceClient := gitalypb.NewRepositoryServiceClient(cc)
-
- _, err = repoServiceClient.RenameRepository(ctx, &gitalypb.RenameRepositoryRequest{
- Repository: &gitalypb.Repository{
- StorageName: virtualRepo1.StorageName,
- RelativePath: "not-found",
- },
- RelativePath: virtualRepo2.RelativePath,
- })
- testhelper.RequireGrpcError(t, testhelper.ToInterceptedMetadata(
- structerr.NewNotFound("%w", storage.NewRepositoryNotFoundError("praefect", "not-found")),
- ), err)
-
- _, err = repoServiceClient.RenameRepository(ctx, &gitalypb.RenameRepositoryRequest{
- Repository: virtualRepo1,
- RelativePath: virtualRepo2.RelativePath,
- })
-
- expectedErr := structerr.NewAlreadyExists("target repo exists already")
- testhelper.RequireGrpcError(t, expectedErr, err)
-
- _, err = repoServiceClient.RenameRepository(ctx, &gitalypb.RenameRepositoryRequest{
- Repository: virtualRepo1,
- RelativePath: newRelativePath,
- })
- require.NoError(t, err)
-
- resp, err := repoServiceClient.RepositoryExists(ctx, &gitalypb.RepositoryExistsRequest{
- Repository: virtualRepo1,
- })
- require.NoError(t, err)
- require.False(t, resp.GetExists(), "repo with old name must gone")
-
- resp, err = repoServiceClient.RepositoryExists(ctx, &gitalypb.RepositoryExistsRequest{
- Repository: &gitalypb.Repository{
- StorageName: virtualRepo1.StorageName,
- RelativePath: newRelativePath,
- },
- })
- require.NoError(t, err)
- require.True(t, resp.GetExists(), "repo with new name must exist")
-}
-
type mockSmartHTTP struct {
gitalypb.UnimplementedSmartHTTPServiceServer
txMgr *transactions.Manager
diff --git a/proto/go/gitalypb/repository.pb.go b/proto/go/gitalypb/repository.pb.go
index c97380b65..01e2dfe9e 100644
--- a/proto/go/gitalypb/repository.pb.go
+++ b/proto/go/gitalypb/repository.pb.go
@@ -200,7 +200,7 @@ func (x OptimizeRepositoryRequest_Strategy) Number() protoreflect.EnumNumber {
// Deprecated: Use OptimizeRepositoryRequest_Strategy.Descriptor instead.
func (OptimizeRepositoryRequest_Strategy) EnumDescriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{75, 0}
+ return file_repository_proto_rawDescGZIP(), []int{73, 0}
}
// RepositoryExistsRequest ...
@@ -4225,103 +4225,6 @@ func (*RemoveRepositoryResponse) Descriptor() ([]byte, []int) {
return file_repository_proto_rawDescGZIP(), []int{70}
}
-// RenameRepositoryRequest ...
-type RenameRepositoryRequest struct {
- state protoimpl.MessageState
- sizeCache protoimpl.SizeCache
- unknownFields protoimpl.UnknownFields
-
- // repository ...
- Repository *Repository `protobuf:"bytes,1,opt,name=repository,proto3" json:"repository,omitempty"`
- // relative_path ...
- RelativePath string `protobuf:"bytes,2,opt,name=relative_path,json=relativePath,proto3" json:"relative_path,omitempty"`
-}
-
-func (x *RenameRepositoryRequest) Reset() {
- *x = RenameRepositoryRequest{}
- if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[71]
- ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
- ms.StoreMessageInfo(mi)
- }
-}
-
-func (x *RenameRepositoryRequest) String() string {
- return protoimpl.X.MessageStringOf(x)
-}
-
-func (*RenameRepositoryRequest) ProtoMessage() {}
-
-func (x *RenameRepositoryRequest) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[71]
- if protoimpl.UnsafeEnabled && x != nil {
- ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
- if ms.LoadMessageInfo() == nil {
- ms.StoreMessageInfo(mi)
- }
- return ms
- }
- return mi.MessageOf(x)
-}
-
-// Deprecated: Use RenameRepositoryRequest.ProtoReflect.Descriptor instead.
-func (*RenameRepositoryRequest) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{71}
-}
-
-func (x *RenameRepositoryRequest) GetRepository() *Repository {
- if x != nil {
- return x.Repository
- }
- return nil
-}
-
-func (x *RenameRepositoryRequest) GetRelativePath() string {
- if x != nil {
- return x.RelativePath
- }
- return ""
-}
-
-// RenameRepositoryResponse ...
-type RenameRepositoryResponse struct {
- state protoimpl.MessageState
- sizeCache protoimpl.SizeCache
- unknownFields protoimpl.UnknownFields
-}
-
-func (x *RenameRepositoryResponse) Reset() {
- *x = RenameRepositoryResponse{}
- if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[72]
- ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
- ms.StoreMessageInfo(mi)
- }
-}
-
-func (x *RenameRepositoryResponse) String() string {
- return protoimpl.X.MessageStringOf(x)
-}
-
-func (*RenameRepositoryResponse) ProtoMessage() {}
-
-func (x *RenameRepositoryResponse) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[72]
- if protoimpl.UnsafeEnabled && x != nil {
- ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
- if ms.LoadMessageInfo() == nil {
- ms.StoreMessageInfo(mi)
- }
- return ms
- }
- return mi.MessageOf(x)
-}
-
-// Deprecated: Use RenameRepositoryResponse.ProtoReflect.Descriptor instead.
-func (*RenameRepositoryResponse) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{72}
-}
-
// ReplicateRepositoryRequest is a request for the ReplicateRepository RPC.
type ReplicateRepositoryRequest struct {
state protoimpl.MessageState
@@ -4346,7 +4249,7 @@ type ReplicateRepositoryRequest struct {
func (x *ReplicateRepositoryRequest) Reset() {
*x = ReplicateRepositoryRequest{}
if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[73]
+ mi := &file_repository_proto_msgTypes[71]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -4359,7 +4262,7 @@ func (x *ReplicateRepositoryRequest) String() string {
func (*ReplicateRepositoryRequest) ProtoMessage() {}
func (x *ReplicateRepositoryRequest) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[73]
+ mi := &file_repository_proto_msgTypes[71]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -4372,7 +4275,7 @@ func (x *ReplicateRepositoryRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use ReplicateRepositoryRequest.ProtoReflect.Descriptor instead.
func (*ReplicateRepositoryRequest) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{73}
+ return file_repository_proto_rawDescGZIP(), []int{71}
}
func (x *ReplicateRepositoryRequest) GetRepository() *Repository {
@@ -4406,7 +4309,7 @@ type ReplicateRepositoryResponse struct {
func (x *ReplicateRepositoryResponse) Reset() {
*x = ReplicateRepositoryResponse{}
if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[74]
+ mi := &file_repository_proto_msgTypes[72]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -4419,7 +4322,7 @@ func (x *ReplicateRepositoryResponse) String() string {
func (*ReplicateRepositoryResponse) ProtoMessage() {}
func (x *ReplicateRepositoryResponse) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[74]
+ mi := &file_repository_proto_msgTypes[72]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -4432,7 +4335,7 @@ func (x *ReplicateRepositoryResponse) ProtoReflect() protoreflect.Message {
// Deprecated: Use ReplicateRepositoryResponse.ProtoReflect.Descriptor instead.
func (*ReplicateRepositoryResponse) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{74}
+ return file_repository_proto_rawDescGZIP(), []int{72}
}
// OptimizeRepositoryRequest is a request for the OptimizeRepository RPC.
@@ -4450,7 +4353,7 @@ type OptimizeRepositoryRequest struct {
func (x *OptimizeRepositoryRequest) Reset() {
*x = OptimizeRepositoryRequest{}
if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[75]
+ mi := &file_repository_proto_msgTypes[73]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -4463,7 +4366,7 @@ func (x *OptimizeRepositoryRequest) String() string {
func (*OptimizeRepositoryRequest) ProtoMessage() {}
func (x *OptimizeRepositoryRequest) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[75]
+ mi := &file_repository_proto_msgTypes[73]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -4476,7 +4379,7 @@ func (x *OptimizeRepositoryRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use OptimizeRepositoryRequest.ProtoReflect.Descriptor instead.
func (*OptimizeRepositoryRequest) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{75}
+ return file_repository_proto_rawDescGZIP(), []int{73}
}
func (x *OptimizeRepositoryRequest) GetRepository() *Repository {
@@ -4503,7 +4406,7 @@ type OptimizeRepositoryResponse struct {
func (x *OptimizeRepositoryResponse) Reset() {
*x = OptimizeRepositoryResponse{}
if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[76]
+ mi := &file_repository_proto_msgTypes[74]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -4516,7 +4419,7 @@ func (x *OptimizeRepositoryResponse) String() string {
func (*OptimizeRepositoryResponse) ProtoMessage() {}
func (x *OptimizeRepositoryResponse) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[76]
+ mi := &file_repository_proto_msgTypes[74]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -4529,7 +4432,7 @@ func (x *OptimizeRepositoryResponse) ProtoReflect() protoreflect.Message {
// Deprecated: Use OptimizeRepositoryResponse.ProtoReflect.Descriptor instead.
func (*OptimizeRepositoryResponse) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{76}
+ return file_repository_proto_rawDescGZIP(), []int{74}
}
// PruneUnreachableObjectsRequest is a request for the PruneUnreachableObjects
@@ -4546,7 +4449,7 @@ type PruneUnreachableObjectsRequest struct {
func (x *PruneUnreachableObjectsRequest) Reset() {
*x = PruneUnreachableObjectsRequest{}
if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[77]
+ mi := &file_repository_proto_msgTypes[75]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -4559,7 +4462,7 @@ func (x *PruneUnreachableObjectsRequest) String() string {
func (*PruneUnreachableObjectsRequest) ProtoMessage() {}
func (x *PruneUnreachableObjectsRequest) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[77]
+ mi := &file_repository_proto_msgTypes[75]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -4572,7 +4475,7 @@ func (x *PruneUnreachableObjectsRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use PruneUnreachableObjectsRequest.ProtoReflect.Descriptor instead.
func (*PruneUnreachableObjectsRequest) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{77}
+ return file_repository_proto_rawDescGZIP(), []int{75}
}
func (x *PruneUnreachableObjectsRequest) GetRepository() *Repository {
@@ -4593,7 +4496,7 @@ type PruneUnreachableObjectsResponse struct {
func (x *PruneUnreachableObjectsResponse) Reset() {
*x = PruneUnreachableObjectsResponse{}
if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[78]
+ mi := &file_repository_proto_msgTypes[76]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -4606,7 +4509,7 @@ func (x *PruneUnreachableObjectsResponse) String() string {
func (*PruneUnreachableObjectsResponse) ProtoMessage() {}
func (x *PruneUnreachableObjectsResponse) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[78]
+ mi := &file_repository_proto_msgTypes[76]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -4619,7 +4522,7 @@ func (x *PruneUnreachableObjectsResponse) ProtoReflect() protoreflect.Message {
// Deprecated: Use PruneUnreachableObjectsResponse.ProtoReflect.Descriptor instead.
func (*PruneUnreachableObjectsResponse) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{78}
+ return file_repository_proto_rawDescGZIP(), []int{76}
}
// SetFullPathRequest is a request for the SetFullPath RPC.
@@ -4637,7 +4540,7 @@ type SetFullPathRequest struct {
func (x *SetFullPathRequest) Reset() {
*x = SetFullPathRequest{}
if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[79]
+ mi := &file_repository_proto_msgTypes[77]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -4650,7 +4553,7 @@ func (x *SetFullPathRequest) String() string {
func (*SetFullPathRequest) ProtoMessage() {}
func (x *SetFullPathRequest) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[79]
+ mi := &file_repository_proto_msgTypes[77]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -4663,7 +4566,7 @@ func (x *SetFullPathRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use SetFullPathRequest.ProtoReflect.Descriptor instead.
func (*SetFullPathRequest) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{79}
+ return file_repository_proto_rawDescGZIP(), []int{77}
}
func (x *SetFullPathRequest) GetRepository() *Repository {
@@ -4690,7 +4593,7 @@ type SetFullPathResponse struct {
func (x *SetFullPathResponse) Reset() {
*x = SetFullPathResponse{}
if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[80]
+ mi := &file_repository_proto_msgTypes[78]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -4703,7 +4606,7 @@ func (x *SetFullPathResponse) String() string {
func (*SetFullPathResponse) ProtoMessage() {}
func (x *SetFullPathResponse) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[80]
+ mi := &file_repository_proto_msgTypes[78]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -4716,7 +4619,7 @@ func (x *SetFullPathResponse) ProtoReflect() protoreflect.Message {
// Deprecated: Use SetFullPathResponse.ProtoReflect.Descriptor instead.
func (*SetFullPathResponse) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{80}
+ return file_repository_proto_rawDescGZIP(), []int{78}
}
// FullPathRequest is a request for the FullPath RPC.
@@ -4732,7 +4635,7 @@ type FullPathRequest struct {
func (x *FullPathRequest) Reset() {
*x = FullPathRequest{}
if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[81]
+ mi := &file_repository_proto_msgTypes[79]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -4745,7 +4648,7 @@ func (x *FullPathRequest) String() string {
func (*FullPathRequest) ProtoMessage() {}
func (x *FullPathRequest) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[81]
+ mi := &file_repository_proto_msgTypes[79]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -4758,7 +4661,7 @@ func (x *FullPathRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use FullPathRequest.ProtoReflect.Descriptor instead.
func (*FullPathRequest) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{81}
+ return file_repository_proto_rawDescGZIP(), []int{79}
}
func (x *FullPathRequest) GetRepository() *Repository {
@@ -4781,7 +4684,7 @@ type FullPathResponse struct {
func (x *FullPathResponse) Reset() {
*x = FullPathResponse{}
if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[82]
+ mi := &file_repository_proto_msgTypes[80]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -4794,7 +4697,7 @@ func (x *FullPathResponse) String() string {
func (*FullPathResponse) ProtoMessage() {}
func (x *FullPathResponse) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[82]
+ mi := &file_repository_proto_msgTypes[80]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -4807,7 +4710,7 @@ func (x *FullPathResponse) ProtoReflect() protoreflect.Message {
// Deprecated: Use FullPathResponse.ProtoReflect.Descriptor instead.
func (*FullPathResponse) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{82}
+ return file_repository_proto_rawDescGZIP(), []int{80}
}
func (x *FullPathResponse) GetPath() string {
@@ -4830,7 +4733,7 @@ type RemoveAllRequest struct {
func (x *RemoveAllRequest) Reset() {
*x = RemoveAllRequest{}
if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[83]
+ mi := &file_repository_proto_msgTypes[81]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -4843,7 +4746,7 @@ func (x *RemoveAllRequest) String() string {
func (*RemoveAllRequest) ProtoMessage() {}
func (x *RemoveAllRequest) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[83]
+ mi := &file_repository_proto_msgTypes[81]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -4856,7 +4759,7 @@ func (x *RemoveAllRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use RemoveAllRequest.ProtoReflect.Descriptor instead.
func (*RemoveAllRequest) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{83}
+ return file_repository_proto_rawDescGZIP(), []int{81}
}
func (x *RemoveAllRequest) GetStorageName() string {
@@ -4876,7 +4779,7 @@ type RemoveAllResponse struct {
func (x *RemoveAllResponse) Reset() {
*x = RemoveAllResponse{}
if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[84]
+ mi := &file_repository_proto_msgTypes[82]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -4889,7 +4792,7 @@ func (x *RemoveAllResponse) String() string {
func (*RemoveAllResponse) ProtoMessage() {}
func (x *RemoveAllResponse) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[84]
+ mi := &file_repository_proto_msgTypes[82]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -4902,7 +4805,7 @@ func (x *RemoveAllResponse) ProtoReflect() protoreflect.Message {
// Deprecated: Use RemoveAllResponse.ProtoReflect.Descriptor instead.
func (*RemoveAllResponse) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{84}
+ return file_repository_proto_rawDescGZIP(), []int{82}
}
// BackupRepositoryRequest is a request for the BackupRepository RPC.
@@ -4925,7 +4828,7 @@ type BackupRepositoryRequest struct {
func (x *BackupRepositoryRequest) Reset() {
*x = BackupRepositoryRequest{}
if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[85]
+ mi := &file_repository_proto_msgTypes[83]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -4938,7 +4841,7 @@ func (x *BackupRepositoryRequest) String() string {
func (*BackupRepositoryRequest) ProtoMessage() {}
func (x *BackupRepositoryRequest) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[85]
+ mi := &file_repository_proto_msgTypes[83]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -4951,7 +4854,7 @@ func (x *BackupRepositoryRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use BackupRepositoryRequest.ProtoReflect.Descriptor instead.
func (*BackupRepositoryRequest) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{85}
+ return file_repository_proto_rawDescGZIP(), []int{83}
}
func (x *BackupRepositoryRequest) GetRepository() *Repository {
@@ -4992,7 +4895,7 @@ type BackupRepositoryResponse struct {
func (x *BackupRepositoryResponse) Reset() {
*x = BackupRepositoryResponse{}
if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[86]
+ mi := &file_repository_proto_msgTypes[84]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -5005,7 +4908,7 @@ func (x *BackupRepositoryResponse) String() string {
func (*BackupRepositoryResponse) ProtoMessage() {}
func (x *BackupRepositoryResponse) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[86]
+ mi := &file_repository_proto_msgTypes[84]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -5018,7 +4921,7 @@ func (x *BackupRepositoryResponse) ProtoReflect() protoreflect.Message {
// Deprecated: Use BackupRepositoryResponse.ProtoReflect.Descriptor instead.
func (*BackupRepositoryResponse) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{86}
+ return file_repository_proto_rawDescGZIP(), []int{84}
}
// RestoreRepositoryRequest is a request for the RestoreRepository RPC.
@@ -5042,7 +4945,7 @@ type RestoreRepositoryRequest struct {
func (x *RestoreRepositoryRequest) Reset() {
*x = RestoreRepositoryRequest{}
if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[87]
+ mi := &file_repository_proto_msgTypes[85]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -5055,7 +4958,7 @@ func (x *RestoreRepositoryRequest) String() string {
func (*RestoreRepositoryRequest) ProtoMessage() {}
func (x *RestoreRepositoryRequest) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[87]
+ mi := &file_repository_proto_msgTypes[85]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -5068,7 +4971,7 @@ func (x *RestoreRepositoryRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use RestoreRepositoryRequest.ProtoReflect.Descriptor instead.
func (*RestoreRepositoryRequest) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{87}
+ return file_repository_proto_rawDescGZIP(), []int{85}
}
func (x *RestoreRepositoryRequest) GetRepository() *Repository {
@@ -5109,7 +5012,7 @@ type RestoreRepositoryResponse struct {
func (x *RestoreRepositoryResponse) Reset() {
*x = RestoreRepositoryResponse{}
if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[88]
+ mi := &file_repository_proto_msgTypes[86]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -5122,7 +5025,7 @@ func (x *RestoreRepositoryResponse) String() string {
func (*RestoreRepositoryResponse) ProtoMessage() {}
func (x *RestoreRepositoryResponse) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[88]
+ mi := &file_repository_proto_msgTypes[86]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -5135,7 +5038,7 @@ func (x *RestoreRepositoryResponse) ProtoReflect() protoreflect.Message {
// Deprecated: Use RestoreRepositoryResponse.ProtoReflect.Descriptor instead.
func (*RestoreRepositoryResponse) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{88}
+ return file_repository_proto_rawDescGZIP(), []int{86}
}
// GetFileAttributesRequest is a request for the GetFileAttributes RPC.
@@ -5157,7 +5060,7 @@ type GetFileAttributesRequest struct {
func (x *GetFileAttributesRequest) Reset() {
*x = GetFileAttributesRequest{}
if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[89]
+ mi := &file_repository_proto_msgTypes[87]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -5170,7 +5073,7 @@ func (x *GetFileAttributesRequest) String() string {
func (*GetFileAttributesRequest) ProtoMessage() {}
func (x *GetFileAttributesRequest) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[89]
+ mi := &file_repository_proto_msgTypes[87]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -5183,7 +5086,7 @@ func (x *GetFileAttributesRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use GetFileAttributesRequest.ProtoReflect.Descriptor instead.
func (*GetFileAttributesRequest) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{89}
+ return file_repository_proto_rawDescGZIP(), []int{87}
}
func (x *GetFileAttributesRequest) GetRepository() *Repository {
@@ -5227,7 +5130,7 @@ type GetFileAttributesResponse struct {
func (x *GetFileAttributesResponse) Reset() {
*x = GetFileAttributesResponse{}
if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[90]
+ mi := &file_repository_proto_msgTypes[88]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -5240,7 +5143,7 @@ func (x *GetFileAttributesResponse) String() string {
func (*GetFileAttributesResponse) ProtoMessage() {}
func (x *GetFileAttributesResponse) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[90]
+ mi := &file_repository_proto_msgTypes[88]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -5253,7 +5156,7 @@ func (x *GetFileAttributesResponse) ProtoReflect() protoreflect.Message {
// Deprecated: Use GetFileAttributesResponse.ProtoReflect.Descriptor instead.
func (*GetFileAttributesResponse) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{90}
+ return file_repository_proto_rawDescGZIP(), []int{88}
}
func (x *GetFileAttributesResponse) GetAttributeInfos() []*GetFileAttributesResponse_AttributeInfo {
@@ -5287,7 +5190,7 @@ type RepositoryInfoResponse_ReferencesInfo struct {
func (x *RepositoryInfoResponse_ReferencesInfo) Reset() {
*x = RepositoryInfoResponse_ReferencesInfo{}
if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[91]
+ mi := &file_repository_proto_msgTypes[89]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -5300,7 +5203,7 @@ func (x *RepositoryInfoResponse_ReferencesInfo) String() string {
func (*RepositoryInfoResponse_ReferencesInfo) ProtoMessage() {}
func (x *RepositoryInfoResponse_ReferencesInfo) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[91]
+ mi := &file_repository_proto_msgTypes[89]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -5366,7 +5269,7 @@ type RepositoryInfoResponse_ObjectsInfo struct {
func (x *RepositoryInfoResponse_ObjectsInfo) Reset() {
*x = RepositoryInfoResponse_ObjectsInfo{}
if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[92]
+ mi := &file_repository_proto_msgTypes[90]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -5379,7 +5282,7 @@ func (x *RepositoryInfoResponse_ObjectsInfo) String() string {
func (*RepositoryInfoResponse_ObjectsInfo) ProtoMessage() {}
func (x *RepositoryInfoResponse_ObjectsInfo) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[92]
+ mi := &file_repository_proto_msgTypes[90]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -5451,7 +5354,7 @@ type GetRawChangesResponse_RawChange struct {
func (x *GetRawChangesResponse_RawChange) Reset() {
*x = GetRawChangesResponse_RawChange{}
if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[93]
+ mi := &file_repository_proto_msgTypes[91]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -5464,7 +5367,7 @@ func (x *GetRawChangesResponse_RawChange) String() string {
func (*GetRawChangesResponse_RawChange) ProtoMessage() {}
func (x *GetRawChangesResponse_RawChange) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[93]
+ mi := &file_repository_proto_msgTypes[91]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -5546,7 +5449,7 @@ type BackupRepositoryResponse_SkippedError struct {
func (x *BackupRepositoryResponse_SkippedError) Reset() {
*x = BackupRepositoryResponse_SkippedError{}
if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[94]
+ mi := &file_repository_proto_msgTypes[92]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -5559,7 +5462,7 @@ func (x *BackupRepositoryResponse_SkippedError) String() string {
func (*BackupRepositoryResponse_SkippedError) ProtoMessage() {}
func (x *BackupRepositoryResponse_SkippedError) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[94]
+ mi := &file_repository_proto_msgTypes[92]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -5572,7 +5475,7 @@ func (x *BackupRepositoryResponse_SkippedError) ProtoReflect() protoreflect.Mess
// Deprecated: Use BackupRepositoryResponse_SkippedError.ProtoReflect.Descriptor instead.
func (*BackupRepositoryResponse_SkippedError) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{86, 0}
+ return file_repository_proto_rawDescGZIP(), []int{84, 0}
}
// SkippedError is returned when the repository restore has been skipped.
@@ -5585,7 +5488,7 @@ type RestoreRepositoryResponse_SkippedError struct {
func (x *RestoreRepositoryResponse_SkippedError) Reset() {
*x = RestoreRepositoryResponse_SkippedError{}
if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[95]
+ mi := &file_repository_proto_msgTypes[93]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -5598,7 +5501,7 @@ func (x *RestoreRepositoryResponse_SkippedError) String() string {
func (*RestoreRepositoryResponse_SkippedError) ProtoMessage() {}
func (x *RestoreRepositoryResponse_SkippedError) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[95]
+ mi := &file_repository_proto_msgTypes[93]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -5611,7 +5514,7 @@ func (x *RestoreRepositoryResponse_SkippedError) ProtoReflect() protoreflect.Mes
// Deprecated: Use RestoreRepositoryResponse_SkippedError.ProtoReflect.Descriptor instead.
func (*RestoreRepositoryResponse_SkippedError) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{88, 0}
+ return file_repository_proto_rawDescGZIP(), []int{86, 0}
}
// AttributeInfo is a specified git attribute for a given path.
@@ -5633,7 +5536,7 @@ type GetFileAttributesResponse_AttributeInfo struct {
func (x *GetFileAttributesResponse_AttributeInfo) Reset() {
*x = GetFileAttributesResponse_AttributeInfo{}
if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[96]
+ mi := &file_repository_proto_msgTypes[94]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -5646,7 +5549,7 @@ func (x *GetFileAttributesResponse_AttributeInfo) String() string {
func (*GetFileAttributesResponse_AttributeInfo) ProtoMessage() {}
func (x *GetFileAttributesResponse_AttributeInfo) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[96]
+ mi := &file_repository_proto_msgTypes[94]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -5659,7 +5562,7 @@ func (x *GetFileAttributesResponse_AttributeInfo) ProtoReflect() protoreflect.Me
// Deprecated: Use GetFileAttributesResponse_AttributeInfo.ProtoReflect.Descriptor instead.
func (*GetFileAttributesResponse_AttributeInfo) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{90, 0}
+ return file_repository_proto_rawDescGZIP(), []int{88, 0}
}
func (x *GetFileAttributesResponse_AttributeInfo) GetPath() string {
@@ -6160,403 +6063,388 @@ var file_repository_proto_rawDesc = []byte{
0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x42, 0x04, 0x98, 0xc6, 0x2c, 0x01, 0x52, 0x0a,
0x72, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x22, 0x1a, 0x0a, 0x18, 0x52, 0x65,
0x6d, 0x6f, 0x76, 0x65, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x52, 0x65,
- 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x78, 0x0a, 0x17, 0x52, 0x65, 0x6e, 0x61, 0x6d, 0x65,
- 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73,
- 0x74, 0x12, 0x38, 0x0a, 0x0a, 0x72, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x18,
- 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x52,
- 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x42, 0x04, 0x98, 0xc6, 0x2c, 0x01, 0x52,
- 0x0a, 0x72, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x12, 0x23, 0x0a, 0x0d, 0x72,
- 0x65, 0x6c, 0x61, 0x74, 0x69, 0x76, 0x65, 0x5f, 0x70, 0x61, 0x74, 0x68, 0x18, 0x02, 0x20, 0x01,
- 0x28, 0x09, 0x52, 0x0c, 0x72, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x76, 0x65, 0x50, 0x61, 0x74, 0x68,
- 0x22, 0x1a, 0x0a, 0x18, 0x52, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69,
- 0x74, 0x6f, 0x72, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xec, 0x01, 0x0a,
- 0x1a, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x65, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69,
- 0x74, 0x6f, 0x72, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x38, 0x0a, 0x0a, 0x72,
- 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32,
+ 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xec, 0x01, 0x0a, 0x1a, 0x52, 0x65, 0x70, 0x6c, 0x69,
+ 0x63, 0x61, 0x74, 0x65, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x52, 0x65,
+ 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x38, 0x0a, 0x0a, 0x72, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74,
+ 0x6f, 0x72, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x67, 0x69, 0x74, 0x61,
+ 0x6c, 0x79, 0x2e, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x42, 0x04, 0x98,
+ 0xc6, 0x2c, 0x01, 0x52, 0x0a, 0x72, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x12,
+ 0x2a, 0x0a, 0x06, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32,
0x12, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74,
- 0x6f, 0x72, 0x79, 0x42, 0x04, 0x98, 0xc6, 0x2c, 0x01, 0x52, 0x0a, 0x72, 0x65, 0x70, 0x6f, 0x73,
- 0x69, 0x74, 0x6f, 0x72, 0x79, 0x12, 0x2a, 0x0a, 0x06, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x18,
- 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x52,
- 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x52, 0x06, 0x73, 0x6f, 0x75, 0x72, 0x63,
- 0x65, 0x12, 0x68, 0x0a, 0x31, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x65, 0x5f, 0x6f,
- 0x62, 0x6a, 0x65, 0x63, 0x74, 0x5f, 0x64, 0x65, 0x64, 0x75, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74,
- 0x69, 0x6f, 0x6e, 0x5f, 0x6e, 0x65, 0x74, 0x77, 0x6f, 0x72, 0x6b, 0x5f, 0x6d, 0x65, 0x6d, 0x62,
- 0x65, 0x72, 0x73, 0x68, 0x69, 0x70, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x2d, 0x72, 0x65,
- 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x65, 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x44, 0x65, 0x64,
- 0x75, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x65, 0x74, 0x77, 0x6f, 0x72,
- 0x6b, 0x4d, 0x65, 0x6d, 0x62, 0x65, 0x72, 0x73, 0x68, 0x69, 0x70, 0x22, 0x1d, 0x0a, 0x1b, 0x52,
- 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x65, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f,
- 0x72, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xf1, 0x01, 0x0a, 0x19, 0x4f,
- 0x70, 0x74, 0x69, 0x6d, 0x69, 0x7a, 0x65, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72,
- 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x38, 0x0a, 0x0a, 0x72, 0x65, 0x70, 0x6f,
+ 0x6f, 0x72, 0x79, 0x52, 0x06, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x12, 0x68, 0x0a, 0x31, 0x72,
+ 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x65, 0x5f, 0x6f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x5f,
+ 0x64, 0x65, 0x64, 0x75, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6e, 0x65,
+ 0x74, 0x77, 0x6f, 0x72, 0x6b, 0x5f, 0x6d, 0x65, 0x6d, 0x62, 0x65, 0x72, 0x73, 0x68, 0x69, 0x70,
+ 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x2d, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74,
+ 0x65, 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x44, 0x65, 0x64, 0x75, 0x70, 0x6c, 0x69, 0x63, 0x61,
+ 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x65, 0x74, 0x77, 0x6f, 0x72, 0x6b, 0x4d, 0x65, 0x6d, 0x62, 0x65,
+ 0x72, 0x73, 0x68, 0x69, 0x70, 0x22, 0x1d, 0x0a, 0x1b, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61,
+ 0x74, 0x65, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x52, 0x65, 0x73, 0x70,
+ 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xf1, 0x01, 0x0a, 0x19, 0x4f, 0x70, 0x74, 0x69, 0x6d, 0x69, 0x7a,
+ 0x65, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65,
+ 0x73, 0x74, 0x12, 0x38, 0x0a, 0x0a, 0x72, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79,
+ 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e,
+ 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x42, 0x04, 0x98, 0xc6, 0x2c, 0x01,
+ 0x52, 0x0a, 0x72, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x12, 0x46, 0x0a, 0x08,
+ 0x73, 0x74, 0x72, 0x61, 0x74, 0x65, 0x67, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x2a,
+ 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x4f, 0x70, 0x74, 0x69, 0x6d, 0x69, 0x7a, 0x65,
+ 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73,
+ 0x74, 0x2e, 0x53, 0x74, 0x72, 0x61, 0x74, 0x65, 0x67, 0x79, 0x52, 0x08, 0x73, 0x74, 0x72, 0x61,
+ 0x74, 0x65, 0x67, 0x79, 0x22, 0x52, 0x0a, 0x08, 0x53, 0x74, 0x72, 0x61, 0x74, 0x65, 0x67, 0x79,
+ 0x12, 0x18, 0x0a, 0x14, 0x53, 0x54, 0x52, 0x41, 0x54, 0x45, 0x47, 0x59, 0x5f, 0x55, 0x4e, 0x53,
+ 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x18, 0x0a, 0x14, 0x53, 0x54,
+ 0x52, 0x41, 0x54, 0x45, 0x47, 0x59, 0x5f, 0x48, 0x45, 0x55, 0x52, 0x49, 0x53, 0x54, 0x49, 0x43,
+ 0x41, 0x4c, 0x10, 0x01, 0x12, 0x12, 0x0a, 0x0e, 0x53, 0x54, 0x52, 0x41, 0x54, 0x45, 0x47, 0x59,
+ 0x5f, 0x45, 0x41, 0x47, 0x45, 0x52, 0x10, 0x02, 0x22, 0x1c, 0x0a, 0x1a, 0x4f, 0x70, 0x74, 0x69,
+ 0x6d, 0x69, 0x7a, 0x65, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x52, 0x65,
+ 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x5a, 0x0a, 0x1e, 0x50, 0x72, 0x75, 0x6e, 0x65, 0x55,
+ 0x6e, 0x72, 0x65, 0x61, 0x63, 0x68, 0x61, 0x62, 0x6c, 0x65, 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74,
+ 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x38, 0x0a, 0x0a, 0x72, 0x65, 0x70, 0x6f,
0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x67,
0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79,
0x42, 0x04, 0x98, 0xc6, 0x2c, 0x01, 0x52, 0x0a, 0x72, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f,
- 0x72, 0x79, 0x12, 0x46, 0x0a, 0x08, 0x73, 0x74, 0x72, 0x61, 0x74, 0x65, 0x67, 0x79, 0x18, 0x02,
- 0x20, 0x01, 0x28, 0x0e, 0x32, 0x2a, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x4f, 0x70,
- 0x74, 0x69, 0x6d, 0x69, 0x7a, 0x65, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79,
- 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x53, 0x74, 0x72, 0x61, 0x74, 0x65, 0x67, 0x79,
- 0x52, 0x08, 0x73, 0x74, 0x72, 0x61, 0x74, 0x65, 0x67, 0x79, 0x22, 0x52, 0x0a, 0x08, 0x53, 0x74,
- 0x72, 0x61, 0x74, 0x65, 0x67, 0x79, 0x12, 0x18, 0x0a, 0x14, 0x53, 0x54, 0x52, 0x41, 0x54, 0x45,
- 0x47, 0x59, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00,
- 0x12, 0x18, 0x0a, 0x14, 0x53, 0x54, 0x52, 0x41, 0x54, 0x45, 0x47, 0x59, 0x5f, 0x48, 0x45, 0x55,
- 0x52, 0x49, 0x53, 0x54, 0x49, 0x43, 0x41, 0x4c, 0x10, 0x01, 0x12, 0x12, 0x0a, 0x0e, 0x53, 0x54,
- 0x52, 0x41, 0x54, 0x45, 0x47, 0x59, 0x5f, 0x45, 0x41, 0x47, 0x45, 0x52, 0x10, 0x02, 0x22, 0x1c,
- 0x0a, 0x1a, 0x4f, 0x70, 0x74, 0x69, 0x6d, 0x69, 0x7a, 0x65, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69,
- 0x74, 0x6f, 0x72, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x5a, 0x0a, 0x1e,
- 0x50, 0x72, 0x75, 0x6e, 0x65, 0x55, 0x6e, 0x72, 0x65, 0x61, 0x63, 0x68, 0x61, 0x62, 0x6c, 0x65,
- 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x38,
- 0x0a, 0x0a, 0x72, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x18, 0x01, 0x20, 0x01,
- 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x52, 0x65, 0x70, 0x6f,
- 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x42, 0x04, 0x98, 0xc6, 0x2c, 0x01, 0x52, 0x0a, 0x72, 0x65,
- 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x22, 0x21, 0x0a, 0x1f, 0x50, 0x72, 0x75, 0x6e,
- 0x65, 0x55, 0x6e, 0x72, 0x65, 0x61, 0x63, 0x68, 0x61, 0x62, 0x6c, 0x65, 0x4f, 0x62, 0x6a, 0x65,
- 0x63, 0x74, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x62, 0x0a, 0x12, 0x53,
- 0x65, 0x74, 0x46, 0x75, 0x6c, 0x6c, 0x50, 0x61, 0x74, 0x68, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73,
- 0x74, 0x12, 0x38, 0x0a, 0x0a, 0x72, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x18,
- 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x52,
- 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x42, 0x04, 0x98, 0xc6, 0x2c, 0x01, 0x52,
- 0x0a, 0x72, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x12, 0x12, 0x0a, 0x04, 0x70,
- 0x61, 0x74, 0x68, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x70, 0x61, 0x74, 0x68, 0x22,
- 0x15, 0x0a, 0x13, 0x53, 0x65, 0x74, 0x46, 0x75, 0x6c, 0x6c, 0x50, 0x61, 0x74, 0x68, 0x52, 0x65,
- 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x4b, 0x0a, 0x0f, 0x46, 0x75, 0x6c, 0x6c, 0x50, 0x61,
- 0x74, 0x68, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x38, 0x0a, 0x0a, 0x72, 0x65, 0x70,
- 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e,
- 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72,
- 0x79, 0x42, 0x04, 0x98, 0xc6, 0x2c, 0x01, 0x52, 0x0a, 0x72, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74,
- 0x6f, 0x72, 0x79, 0x22, 0x26, 0x0a, 0x10, 0x46, 0x75, 0x6c, 0x6c, 0x50, 0x61, 0x74, 0x68, 0x52,
- 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x70, 0x61, 0x74, 0x68, 0x18,
- 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x70, 0x61, 0x74, 0x68, 0x22, 0x3b, 0x0a, 0x10, 0x52,
- 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x41, 0x6c, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12,
- 0x27, 0x0a, 0x0c, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18,
- 0x01, 0x20, 0x01, 0x28, 0x09, 0x42, 0x04, 0x88, 0xc6, 0x2c, 0x01, 0x52, 0x0b, 0x73, 0x74, 0x6f,
- 0x72, 0x61, 0x67, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0x13, 0x0a, 0x11, 0x52, 0x65, 0x6d, 0x6f,
- 0x76, 0x65, 0x41, 0x6c, 0x6c, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xd3, 0x01,
- 0x0a, 0x17, 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f,
- 0x72, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x38, 0x0a, 0x0a, 0x72, 0x65, 0x70,
- 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e,
- 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72,
- 0x79, 0x42, 0x04, 0x98, 0xc6, 0x2c, 0x01, 0x52, 0x0a, 0x72, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74,
- 0x6f, 0x72, 0x79, 0x12, 0x3f, 0x0a, 0x11, 0x76, 0x61, 0x6e, 0x69, 0x74, 0x79, 0x5f, 0x72, 0x65,
- 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12,
- 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f,
- 0x72, 0x79, 0x52, 0x10, 0x76, 0x61, 0x6e, 0x69, 0x74, 0x79, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69,
- 0x74, 0x6f, 0x72, 0x79, 0x12, 0x1b, 0x0a, 0x09, 0x62, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x5f, 0x69,
- 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x62, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x49,
- 0x64, 0x12, 0x20, 0x0a, 0x0b, 0x69, 0x6e, 0x63, 0x72, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x61, 0x6c,
- 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0b, 0x69, 0x6e, 0x63, 0x72, 0x65, 0x6d, 0x65, 0x6e,
- 0x74, 0x61, 0x6c, 0x22, 0x2a, 0x0a, 0x18, 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x52, 0x65, 0x70,
- 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x1a,
- 0x0e, 0x0a, 0x0c, 0x53, 0x6b, 0x69, 0x70, 0x70, 0x65, 0x64, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x22,
- 0xd7, 0x01, 0x0a, 0x18, 0x52, 0x65, 0x73, 0x74, 0x6f, 0x72, 0x65, 0x52, 0x65, 0x70, 0x6f, 0x73,
- 0x69, 0x74, 0x6f, 0x72, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x38, 0x0a, 0x0a,
- 0x72, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b,
- 0x32, 0x12, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69,
- 0x74, 0x6f, 0x72, 0x79, 0x42, 0x04, 0x98, 0xc6, 0x2c, 0x01, 0x52, 0x0a, 0x72, 0x65, 0x70, 0x6f,
- 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x12, 0x3f, 0x0a, 0x11, 0x76, 0x61, 0x6e, 0x69, 0x74, 0x79,
- 0x5f, 0x72, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28,
- 0x0b, 0x32, 0x12, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x52, 0x65, 0x70, 0x6f, 0x73,
- 0x69, 0x74, 0x6f, 0x72, 0x79, 0x52, 0x10, 0x76, 0x61, 0x6e, 0x69, 0x74, 0x79, 0x52, 0x65, 0x70,
- 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x12, 0x1b, 0x0a, 0x09, 0x62, 0x61, 0x63, 0x6b, 0x75,
- 0x70, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x62, 0x61, 0x63, 0x6b,
- 0x75, 0x70, 0x49, 0x64, 0x12, 0x23, 0x0a, 0x0d, 0x61, 0x6c, 0x77, 0x61, 0x79, 0x73, 0x5f, 0x63,
- 0x72, 0x65, 0x61, 0x74, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0c, 0x61, 0x6c, 0x77,
- 0x61, 0x79, 0x73, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x22, 0x2b, 0x0a, 0x19, 0x52, 0x65, 0x73,
- 0x74, 0x6f, 0x72, 0x65, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x52, 0x65,
- 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x1a, 0x0e, 0x0a, 0x0c, 0x53, 0x6b, 0x69, 0x70, 0x70, 0x65,
- 0x64, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x22, 0xa6, 0x01, 0x0a, 0x18, 0x47, 0x65, 0x74, 0x46, 0x69,
- 0x6c, 0x65, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75,
+ 0x72, 0x79, 0x22, 0x21, 0x0a, 0x1f, 0x50, 0x72, 0x75, 0x6e, 0x65, 0x55, 0x6e, 0x72, 0x65, 0x61,
+ 0x63, 0x68, 0x61, 0x62, 0x6c, 0x65, 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x73, 0x52, 0x65, 0x73,
+ 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x62, 0x0a, 0x12, 0x53, 0x65, 0x74, 0x46, 0x75, 0x6c, 0x6c,
+ 0x50, 0x61, 0x74, 0x68, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x38, 0x0a, 0x0a, 0x72,
+ 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32,
+ 0x12, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74,
+ 0x6f, 0x72, 0x79, 0x42, 0x04, 0x98, 0xc6, 0x2c, 0x01, 0x52, 0x0a, 0x72, 0x65, 0x70, 0x6f, 0x73,
+ 0x69, 0x74, 0x6f, 0x72, 0x79, 0x12, 0x12, 0x0a, 0x04, 0x70, 0x61, 0x74, 0x68, 0x18, 0x02, 0x20,
+ 0x01, 0x28, 0x09, 0x52, 0x04, 0x70, 0x61, 0x74, 0x68, 0x22, 0x15, 0x0a, 0x13, 0x53, 0x65, 0x74,
+ 0x46, 0x75, 0x6c, 0x6c, 0x50, 0x61, 0x74, 0x68, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65,
+ 0x22, 0x4b, 0x0a, 0x0f, 0x46, 0x75, 0x6c, 0x6c, 0x50, 0x61, 0x74, 0x68, 0x52, 0x65, 0x71, 0x75,
0x65, 0x73, 0x74, 0x12, 0x38, 0x0a, 0x0a, 0x72, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72,
0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79,
0x2e, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x42, 0x04, 0x98, 0xc6, 0x2c,
- 0x01, 0x52, 0x0a, 0x72, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x12, 0x1a, 0x0a,
- 0x08, 0x72, 0x65, 0x76, 0x69, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52,
- 0x08, 0x72, 0x65, 0x76, 0x69, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x1e, 0x0a, 0x0a, 0x61, 0x74, 0x74,
- 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0a, 0x61,
- 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x12, 0x14, 0x0a, 0x05, 0x70, 0x61, 0x74,
- 0x68, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x09, 0x52, 0x05, 0x70, 0x61, 0x74, 0x68, 0x73, 0x22,
- 0xce, 0x01, 0x0a, 0x19, 0x47, 0x65, 0x74, 0x46, 0x69, 0x6c, 0x65, 0x41, 0x74, 0x74, 0x72, 0x69,
- 0x62, 0x75, 0x74, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x58, 0x0a,
- 0x0f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x73,
- 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e,
- 0x47, 0x65, 0x74, 0x46, 0x69, 0x6c, 0x65, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65,
- 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62,
- 0x75, 0x74, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x0e, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75,
- 0x74, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x73, 0x1a, 0x57, 0x0a, 0x0d, 0x41, 0x74, 0x74, 0x72, 0x69,
- 0x62, 0x75, 0x74, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x12, 0x0a, 0x04, 0x70, 0x61, 0x74, 0x68,
- 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x70, 0x61, 0x74, 0x68, 0x12, 0x1c, 0x0a, 0x09,
- 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52,
- 0x09, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61,
- 0x6c, 0x75, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65,
- 0x32, 0x8e, 0x21, 0x0a, 0x11, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x53,
- 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x5d, 0x0a, 0x10, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69,
- 0x74, 0x6f, 0x72, 0x79, 0x45, 0x78, 0x69, 0x73, 0x74, 0x73, 0x12, 0x1f, 0x2e, 0x67, 0x69, 0x74,
- 0x61, 0x6c, 0x79, 0x2e, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x45, 0x78,
- 0x69, 0x73, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x20, 0x2e, 0x67, 0x69,
- 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x45,
- 0x78, 0x69, 0x73, 0x74, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x06, 0xfa,
- 0x97, 0x28, 0x02, 0x08, 0x02, 0x12, 0x57, 0x0a, 0x0e, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74,
- 0x6f, 0x72, 0x79, 0x53, 0x69, 0x7a, 0x65, 0x12, 0x1d, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79,
- 0x2e, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x53, 0x69, 0x7a, 0x65, 0x52,
- 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1e, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e,
- 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x53, 0x69, 0x7a, 0x65, 0x52, 0x65,
- 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x06, 0xfa, 0x97, 0x28, 0x02, 0x08, 0x02, 0x12, 0x57,
- 0x0a, 0x0e, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x49, 0x6e, 0x66, 0x6f,
- 0x12, 0x1d, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69,
- 0x74, 0x6f, 0x72, 0x79, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a,
- 0x1e, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74,
- 0x6f, 0x72, 0x79, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22,
- 0x06, 0xfa, 0x97, 0x28, 0x02, 0x08, 0x02, 0x12, 0x50, 0x0a, 0x0b, 0x4f, 0x62, 0x6a, 0x65, 0x63,
- 0x74, 0x73, 0x53, 0x69, 0x7a, 0x65, 0x12, 0x1a, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e,
- 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x73, 0x53, 0x69, 0x7a, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65,
- 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x4f, 0x62, 0x6a, 0x65,
- 0x63, 0x74, 0x73, 0x53, 0x69, 0x7a, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22,
- 0x06, 0xfa, 0x97, 0x28, 0x02, 0x08, 0x02, 0x28, 0x01, 0x12, 0x51, 0x0a, 0x0c, 0x4f, 0x62, 0x6a,
- 0x65, 0x63, 0x74, 0x46, 0x6f, 0x72, 0x6d, 0x61, 0x74, 0x12, 0x1b, 0x2e, 0x67, 0x69, 0x74, 0x61,
- 0x6c, 0x79, 0x2e, 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x46, 0x6f, 0x72, 0x6d, 0x61, 0x74, 0x52,
- 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1c, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e,
- 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x46, 0x6f, 0x72, 0x6d, 0x61, 0x74, 0x52, 0x65, 0x73, 0x70,
- 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x06, 0xfa, 0x97, 0x28, 0x02, 0x08, 0x02, 0x12, 0x66, 0x0a, 0x12,
- 0x41, 0x70, 0x70, 0x6c, 0x79, 0x47, 0x69, 0x74, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74,
- 0x65, 0x73, 0x12, 0x21, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x41, 0x70, 0x70, 0x6c,
- 0x79, 0x47, 0x69, 0x74, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x52, 0x65,
- 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x22, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x41,
- 0x70, 0x70, 0x6c, 0x79, 0x47, 0x69, 0x74, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65,
- 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x09, 0xfa, 0x97, 0x28, 0x02, 0x08,
- 0x01, 0x88, 0x02, 0x01, 0x12, 0x4e, 0x0a, 0x0b, 0x46, 0x65, 0x74, 0x63, 0x68, 0x52, 0x65, 0x6d,
- 0x6f, 0x74, 0x65, 0x12, 0x1a, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x46, 0x65, 0x74,
- 0x63, 0x68, 0x52, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a,
- 0x1b, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x46, 0x65, 0x74, 0x63, 0x68, 0x52, 0x65,
- 0x6d, 0x6f, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x06, 0xfa, 0x97,
- 0x28, 0x02, 0x08, 0x01, 0x12, 0x5d, 0x0a, 0x10, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, 0x65,
- 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x12, 0x1f, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c,
- 0x79, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f,
- 0x72, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x20, 0x2e, 0x67, 0x69, 0x74, 0x61,
- 0x6c, 0x79, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74,
- 0x6f, 0x72, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x06, 0xfa, 0x97, 0x28,
- 0x02, 0x08, 0x01, 0x12, 0x4d, 0x0a, 0x0a, 0x47, 0x65, 0x74, 0x41, 0x72, 0x63, 0x68, 0x69, 0x76,
- 0x65, 0x12, 0x19, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x47, 0x65, 0x74, 0x41, 0x72,
- 0x63, 0x68, 0x69, 0x76, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1a, 0x2e, 0x67,
- 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x47, 0x65, 0x74, 0x41, 0x72, 0x63, 0x68, 0x69, 0x76, 0x65,
- 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x06, 0xfa, 0x97, 0x28, 0x02, 0x08, 0x02,
- 0x30, 0x01, 0x12, 0x5d, 0x0a, 0x10, 0x48, 0x61, 0x73, 0x4c, 0x6f, 0x63, 0x61, 0x6c, 0x42, 0x72,
- 0x61, 0x6e, 0x63, 0x68, 0x65, 0x73, 0x12, 0x1f, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e,
+ 0x01, 0x52, 0x0a, 0x72, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x22, 0x26, 0x0a,
+ 0x10, 0x46, 0x75, 0x6c, 0x6c, 0x50, 0x61, 0x74, 0x68, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73,
+ 0x65, 0x12, 0x12, 0x0a, 0x04, 0x70, 0x61, 0x74, 0x68, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52,
+ 0x04, 0x70, 0x61, 0x74, 0x68, 0x22, 0x3b, 0x0a, 0x10, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x41,
+ 0x6c, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x27, 0x0a, 0x0c, 0x73, 0x74, 0x6f,
+ 0x72, 0x61, 0x67, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x42,
+ 0x04, 0x88, 0xc6, 0x2c, 0x01, 0x52, 0x0b, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x4e, 0x61,
+ 0x6d, 0x65, 0x22, 0x13, 0x0a, 0x11, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x41, 0x6c, 0x6c, 0x52,
+ 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xd3, 0x01, 0x0a, 0x17, 0x42, 0x61, 0x63, 0x6b,
+ 0x75, 0x70, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x52, 0x65, 0x71, 0x75,
+ 0x65, 0x73, 0x74, 0x12, 0x38, 0x0a, 0x0a, 0x72, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72,
+ 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79,
+ 0x2e, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x42, 0x04, 0x98, 0xc6, 0x2c,
+ 0x01, 0x52, 0x0a, 0x72, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x12, 0x3f, 0x0a,
+ 0x11, 0x76, 0x61, 0x6e, 0x69, 0x74, 0x79, 0x5f, 0x72, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f,
+ 0x72, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c,
+ 0x79, 0x2e, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x52, 0x10, 0x76, 0x61,
+ 0x6e, 0x69, 0x74, 0x79, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x12, 0x1b,
+ 0x0a, 0x09, 0x62, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28,
+ 0x09, 0x52, 0x08, 0x62, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x49, 0x64, 0x12, 0x20, 0x0a, 0x0b, 0x69,
+ 0x6e, 0x63, 0x72, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x61, 0x6c, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08,
+ 0x52, 0x0b, 0x69, 0x6e, 0x63, 0x72, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x61, 0x6c, 0x22, 0x2a, 0x0a,
+ 0x18, 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72,
+ 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x1a, 0x0e, 0x0a, 0x0c, 0x53, 0x6b, 0x69,
+ 0x70, 0x70, 0x65, 0x64, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x22, 0xd7, 0x01, 0x0a, 0x18, 0x52, 0x65,
+ 0x73, 0x74, 0x6f, 0x72, 0x65, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x52,
+ 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x38, 0x0a, 0x0a, 0x72, 0x65, 0x70, 0x6f, 0x73, 0x69,
+ 0x74, 0x6f, 0x72, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x67, 0x69, 0x74,
+ 0x61, 0x6c, 0x79, 0x2e, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x42, 0x04,
+ 0x98, 0xc6, 0x2c, 0x01, 0x52, 0x0a, 0x72, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79,
+ 0x12, 0x3f, 0x0a, 0x11, 0x76, 0x61, 0x6e, 0x69, 0x74, 0x79, 0x5f, 0x72, 0x65, 0x70, 0x6f, 0x73,
+ 0x69, 0x74, 0x6f, 0x72, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x67, 0x69,
+ 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x52,
+ 0x10, 0x76, 0x61, 0x6e, 0x69, 0x74, 0x79, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72,
+ 0x79, 0x12, 0x1b, 0x0a, 0x09, 0x62, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x5f, 0x69, 0x64, 0x18, 0x03,
+ 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x62, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x49, 0x64, 0x12, 0x23,
+ 0x0a, 0x0d, 0x61, 0x6c, 0x77, 0x61, 0x79, 0x73, 0x5f, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x18,
+ 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0c, 0x61, 0x6c, 0x77, 0x61, 0x79, 0x73, 0x43, 0x72, 0x65,
+ 0x61, 0x74, 0x65, 0x22, 0x2b, 0x0a, 0x19, 0x52, 0x65, 0x73, 0x74, 0x6f, 0x72, 0x65, 0x52, 0x65,
+ 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65,
+ 0x1a, 0x0e, 0x0a, 0x0c, 0x53, 0x6b, 0x69, 0x70, 0x70, 0x65, 0x64, 0x45, 0x72, 0x72, 0x6f, 0x72,
+ 0x22, 0xa6, 0x01, 0x0a, 0x18, 0x47, 0x65, 0x74, 0x46, 0x69, 0x6c, 0x65, 0x41, 0x74, 0x74, 0x72,
+ 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x38, 0x0a,
+ 0x0a, 0x72, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28,
+ 0x0b, 0x32, 0x12, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x52, 0x65, 0x70, 0x6f, 0x73,
+ 0x69, 0x74, 0x6f, 0x72, 0x79, 0x42, 0x04, 0x98, 0xc6, 0x2c, 0x01, 0x52, 0x0a, 0x72, 0x65, 0x70,
+ 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x12, 0x1a, 0x0a, 0x08, 0x72, 0x65, 0x76, 0x69, 0x73,
+ 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x08, 0x72, 0x65, 0x76, 0x69, 0x73,
+ 0x69, 0x6f, 0x6e, 0x12, 0x1e, 0x0a, 0x0a, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65,
+ 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0a, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75,
+ 0x74, 0x65, 0x73, 0x12, 0x14, 0x0a, 0x05, 0x70, 0x61, 0x74, 0x68, 0x73, 0x18, 0x04, 0x20, 0x03,
+ 0x28, 0x09, 0x52, 0x05, 0x70, 0x61, 0x74, 0x68, 0x73, 0x22, 0xce, 0x01, 0x0a, 0x19, 0x47, 0x65,
+ 0x74, 0x46, 0x69, 0x6c, 0x65, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x52,
+ 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x58, 0x0a, 0x0f, 0x61, 0x74, 0x74, 0x72, 0x69,
+ 0x62, 0x75, 0x74, 0x65, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b,
+ 0x32, 0x2f, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x47, 0x65, 0x74, 0x46, 0x69, 0x6c,
+ 0x65, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f,
+ 0x6e, 0x73, 0x65, 0x2e, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x49, 0x6e, 0x66,
+ 0x6f, 0x52, 0x0e, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x49, 0x6e, 0x66, 0x6f,
+ 0x73, 0x1a, 0x57, 0x0a, 0x0d, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x49, 0x6e,
+ 0x66, 0x6f, 0x12, 0x12, 0x0a, 0x04, 0x70, 0x61, 0x74, 0x68, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09,
+ 0x52, 0x04, 0x70, 0x61, 0x74, 0x68, 0x12, 0x1c, 0x0a, 0x09, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62,
+ 0x75, 0x74, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x61, 0x74, 0x74, 0x72, 0x69,
+ 0x62, 0x75, 0x74, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x03, 0x20,
+ 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x32, 0xaf, 0x20, 0x0a, 0x11, 0x52,
+ 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65,
+ 0x12, 0x5d, 0x0a, 0x10, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x45, 0x78,
+ 0x69, 0x73, 0x74, 0x73, 0x12, 0x1f, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x52, 0x65,
+ 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x45, 0x78, 0x69, 0x73, 0x74, 0x73, 0x52, 0x65,
+ 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x20, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x52,
+ 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x45, 0x78, 0x69, 0x73, 0x74, 0x73, 0x52,
+ 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x06, 0xfa, 0x97, 0x28, 0x02, 0x08, 0x02, 0x12,
+ 0x57, 0x0a, 0x0e, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x53, 0x69, 0x7a,
+ 0x65, 0x12, 0x1d, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x52, 0x65, 0x70, 0x6f, 0x73,
+ 0x69, 0x74, 0x6f, 0x72, 0x79, 0x53, 0x69, 0x7a, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74,
+ 0x1a, 0x1e, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69,
+ 0x74, 0x6f, 0x72, 0x79, 0x53, 0x69, 0x7a, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65,
+ 0x22, 0x06, 0xfa, 0x97, 0x28, 0x02, 0x08, 0x02, 0x12, 0x57, 0x0a, 0x0e, 0x52, 0x65, 0x70, 0x6f,
+ 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x1d, 0x2e, 0x67, 0x69, 0x74,
+ 0x61, 0x6c, 0x79, 0x2e, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x49, 0x6e,
+ 0x66, 0x6f, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1e, 0x2e, 0x67, 0x69, 0x74, 0x61,
+ 0x6c, 0x79, 0x2e, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x49, 0x6e, 0x66,
+ 0x6f, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x06, 0xfa, 0x97, 0x28, 0x02, 0x08,
+ 0x02, 0x12, 0x50, 0x0a, 0x0b, 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x73, 0x53, 0x69, 0x7a, 0x65,
+ 0x12, 0x1a, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74,
+ 0x73, 0x53, 0x69, 0x7a, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x67,
+ 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x73, 0x53, 0x69, 0x7a,
+ 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x06, 0xfa, 0x97, 0x28, 0x02, 0x08,
+ 0x02, 0x28, 0x01, 0x12, 0x51, 0x0a, 0x0c, 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x46, 0x6f, 0x72,
+ 0x6d, 0x61, 0x74, 0x12, 0x1b, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x4f, 0x62, 0x6a,
+ 0x65, 0x63, 0x74, 0x46, 0x6f, 0x72, 0x6d, 0x61, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74,
+ 0x1a, 0x1c, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74,
+ 0x46, 0x6f, 0x72, 0x6d, 0x61, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x06,
+ 0xfa, 0x97, 0x28, 0x02, 0x08, 0x02, 0x12, 0x66, 0x0a, 0x12, 0x41, 0x70, 0x70, 0x6c, 0x79, 0x47,
+ 0x69, 0x74, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x12, 0x21, 0x2e, 0x67,
+ 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x41, 0x70, 0x70, 0x6c, 0x79, 0x47, 0x69, 0x74, 0x61, 0x74,
+ 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a,
+ 0x22, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x41, 0x70, 0x70, 0x6c, 0x79, 0x47, 0x69,
+ 0x74, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f,
+ 0x6e, 0x73, 0x65, 0x22, 0x09, 0xfa, 0x97, 0x28, 0x02, 0x08, 0x01, 0x88, 0x02, 0x01, 0x12, 0x4e,
+ 0x0a, 0x0b, 0x46, 0x65, 0x74, 0x63, 0x68, 0x52, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x12, 0x1a, 0x2e,
+ 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x46, 0x65, 0x74, 0x63, 0x68, 0x52, 0x65, 0x6d, 0x6f,
+ 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x67, 0x69, 0x74, 0x61,
+ 0x6c, 0x79, 0x2e, 0x46, 0x65, 0x74, 0x63, 0x68, 0x52, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x52, 0x65,
+ 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x06, 0xfa, 0x97, 0x28, 0x02, 0x08, 0x01, 0x12, 0x5d,
+ 0x0a, 0x10, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f,
+ 0x72, 0x79, 0x12, 0x1f, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x43, 0x72, 0x65, 0x61,
+ 0x74, 0x65, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x52, 0x65, 0x71, 0x75,
+ 0x65, 0x73, 0x74, 0x1a, 0x20, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x43, 0x72, 0x65,
+ 0x61, 0x74, 0x65, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x52, 0x65, 0x73,
+ 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x06, 0xfa, 0x97, 0x28, 0x02, 0x08, 0x01, 0x12, 0x4d, 0x0a,
+ 0x0a, 0x47, 0x65, 0x74, 0x41, 0x72, 0x63, 0x68, 0x69, 0x76, 0x65, 0x12, 0x19, 0x2e, 0x67, 0x69,
+ 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x47, 0x65, 0x74, 0x41, 0x72, 0x63, 0x68, 0x69, 0x76, 0x65, 0x52,
+ 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1a, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e,
+ 0x47, 0x65, 0x74, 0x41, 0x72, 0x63, 0x68, 0x69, 0x76, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e,
+ 0x73, 0x65, 0x22, 0x06, 0xfa, 0x97, 0x28, 0x02, 0x08, 0x02, 0x30, 0x01, 0x12, 0x5d, 0x0a, 0x10,
0x48, 0x61, 0x73, 0x4c, 0x6f, 0x63, 0x61, 0x6c, 0x42, 0x72, 0x61, 0x6e, 0x63, 0x68, 0x65, 0x73,
- 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x20, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79,
- 0x2e, 0x48, 0x61, 0x73, 0x4c, 0x6f, 0x63, 0x61, 0x6c, 0x42, 0x72, 0x61, 0x6e, 0x63, 0x68, 0x65,
- 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x06, 0xfa, 0x97, 0x28, 0x02, 0x08,
- 0x02, 0x12, 0x60, 0x0a, 0x11, 0x46, 0x65, 0x74, 0x63, 0x68, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65,
- 0x42, 0x72, 0x61, 0x6e, 0x63, 0x68, 0x12, 0x20, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e,
- 0x46, 0x65, 0x74, 0x63, 0x68, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x42, 0x72, 0x61, 0x6e, 0x63,
- 0x68, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x21, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c,
- 0x79, 0x2e, 0x46, 0x65, 0x74, 0x63, 0x68, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x42, 0x72, 0x61,
- 0x6e, 0x63, 0x68, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x06, 0xfa, 0x97, 0x28,
- 0x02, 0x08, 0x01, 0x12, 0x39, 0x0a, 0x04, 0x46, 0x73, 0x63, 0x6b, 0x12, 0x13, 0x2e, 0x67, 0x69,
- 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x46, 0x73, 0x63, 0x6b, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74,
- 0x1a, 0x14, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x46, 0x73, 0x63, 0x6b, 0x52, 0x65,
- 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x06, 0xfa, 0x97, 0x28, 0x02, 0x08, 0x02, 0x12, 0x45,
- 0x0a, 0x08, 0x57, 0x72, 0x69, 0x74, 0x65, 0x52, 0x65, 0x66, 0x12, 0x17, 0x2e, 0x67, 0x69, 0x74,
- 0x61, 0x6c, 0x79, 0x2e, 0x57, 0x72, 0x69, 0x74, 0x65, 0x52, 0x65, 0x66, 0x52, 0x65, 0x71, 0x75,
- 0x65, 0x73, 0x74, 0x1a, 0x18, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x57, 0x72, 0x69,
- 0x74, 0x65, 0x52, 0x65, 0x66, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x06, 0xfa,
- 0x97, 0x28, 0x02, 0x08, 0x01, 0x12, 0x54, 0x0a, 0x0d, 0x46, 0x69, 0x6e, 0x64, 0x4d, 0x65, 0x72,
- 0x67, 0x65, 0x42, 0x61, 0x73, 0x65, 0x12, 0x1c, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e,
- 0x46, 0x69, 0x6e, 0x64, 0x4d, 0x65, 0x72, 0x67, 0x65, 0x42, 0x61, 0x73, 0x65, 0x52, 0x65, 0x71,
- 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1d, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x46, 0x69,
- 0x6e, 0x64, 0x4d, 0x65, 0x72, 0x67, 0x65, 0x42, 0x61, 0x73, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f,
- 0x6e, 0x73, 0x65, 0x22, 0x06, 0xfa, 0x97, 0x28, 0x02, 0x08, 0x02, 0x12, 0x4b, 0x0a, 0x0a, 0x43,
- 0x72, 0x65, 0x61, 0x74, 0x65, 0x46, 0x6f, 0x72, 0x6b, 0x12, 0x19, 0x2e, 0x67, 0x69, 0x74, 0x61,
- 0x6c, 0x79, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x46, 0x6f, 0x72, 0x6b, 0x52, 0x65, 0x71,
- 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1a, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x43, 0x72,
- 0x65, 0x61, 0x74, 0x65, 0x46, 0x6f, 0x72, 0x6b, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65,
- 0x22, 0x06, 0xfa, 0x97, 0x28, 0x02, 0x08, 0x01, 0x12, 0x72, 0x0a, 0x17, 0x43, 0x72, 0x65, 0x61,
- 0x74, 0x65, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x46, 0x72, 0x6f, 0x6d,
- 0x55, 0x52, 0x4c, 0x12, 0x26, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x43, 0x72, 0x65,
- 0x61, 0x74, 0x65, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x46, 0x72, 0x6f,
- 0x6d, 0x55, 0x52, 0x4c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x27, 0x2e, 0x67, 0x69,
- 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, 0x65, 0x70, 0x6f, 0x73,
- 0x69, 0x74, 0x6f, 0x72, 0x79, 0x46, 0x72, 0x6f, 0x6d, 0x55, 0x52, 0x4c, 0x52, 0x65, 0x73, 0x70,
- 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x06, 0xfa, 0x97, 0x28, 0x02, 0x08, 0x01, 0x12, 0x53, 0x0a, 0x0c,
- 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x12, 0x1b, 0x2e, 0x67,
- 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x42, 0x75, 0x6e, 0x64,
- 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1c, 0x2e, 0x67, 0x69, 0x74, 0x61,
- 0x6c, 0x79, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x52,
- 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x06, 0xfa, 0x97, 0x28, 0x02, 0x08, 0x02, 0x30,
- 0x01, 0x12, 0x76, 0x0a, 0x17, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x42, 0x75, 0x6e, 0x64, 0x6c,
- 0x65, 0x46, 0x72, 0x6f, 0x6d, 0x52, 0x65, 0x66, 0x4c, 0x69, 0x73, 0x74, 0x12, 0x26, 0x2e, 0x67,
- 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x42, 0x75, 0x6e, 0x64,
- 0x6c, 0x65, 0x46, 0x72, 0x6f, 0x6d, 0x52, 0x65, 0x66, 0x4c, 0x69, 0x73, 0x74, 0x52, 0x65, 0x71,
- 0x75, 0x65, 0x73, 0x74, 0x1a, 0x27, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x43, 0x72,
- 0x65, 0x61, 0x74, 0x65, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x46, 0x72, 0x6f, 0x6d, 0x52, 0x65,
- 0x66, 0x4c, 0x69, 0x73, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x06, 0xfa,
- 0x97, 0x28, 0x02, 0x08, 0x02, 0x28, 0x01, 0x30, 0x01, 0x12, 0x50, 0x0a, 0x0b, 0x46, 0x65, 0x74,
- 0x63, 0x68, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x12, 0x1a, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c,
- 0x79, 0x2e, 0x46, 0x65, 0x74, 0x63, 0x68, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x52, 0x65, 0x71,
- 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x46, 0x65,
- 0x74, 0x63, 0x68, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73,
- 0x65, 0x22, 0x06, 0xfa, 0x97, 0x28, 0x02, 0x08, 0x01, 0x28, 0x01, 0x12, 0x7d, 0x0a, 0x1a, 0x43,
+ 0x12, 0x1f, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x48, 0x61, 0x73, 0x4c, 0x6f, 0x63,
+ 0x61, 0x6c, 0x42, 0x72, 0x61, 0x6e, 0x63, 0x68, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73,
+ 0x74, 0x1a, 0x20, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x48, 0x61, 0x73, 0x4c, 0x6f,
+ 0x63, 0x61, 0x6c, 0x42, 0x72, 0x61, 0x6e, 0x63, 0x68, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f,
+ 0x6e, 0x73, 0x65, 0x22, 0x06, 0xfa, 0x97, 0x28, 0x02, 0x08, 0x02, 0x12, 0x60, 0x0a, 0x11, 0x46,
+ 0x65, 0x74, 0x63, 0x68, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x42, 0x72, 0x61, 0x6e, 0x63, 0x68,
+ 0x12, 0x20, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x46, 0x65, 0x74, 0x63, 0x68, 0x53,
+ 0x6f, 0x75, 0x72, 0x63, 0x65, 0x42, 0x72, 0x61, 0x6e, 0x63, 0x68, 0x52, 0x65, 0x71, 0x75, 0x65,
+ 0x73, 0x74, 0x1a, 0x21, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x46, 0x65, 0x74, 0x63,
+ 0x68, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x42, 0x72, 0x61, 0x6e, 0x63, 0x68, 0x52, 0x65, 0x73,
+ 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x06, 0xfa, 0x97, 0x28, 0x02, 0x08, 0x01, 0x12, 0x39, 0x0a,
+ 0x04, 0x46, 0x73, 0x63, 0x6b, 0x12, 0x13, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x46,
+ 0x73, 0x63, 0x6b, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x14, 0x2e, 0x67, 0x69, 0x74,
+ 0x61, 0x6c, 0x79, 0x2e, 0x46, 0x73, 0x63, 0x6b, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65,
+ 0x22, 0x06, 0xfa, 0x97, 0x28, 0x02, 0x08, 0x02, 0x12, 0x45, 0x0a, 0x08, 0x57, 0x72, 0x69, 0x74,
+ 0x65, 0x52, 0x65, 0x66, 0x12, 0x17, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x57, 0x72,
+ 0x69, 0x74, 0x65, 0x52, 0x65, 0x66, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x18, 0x2e,
+ 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x57, 0x72, 0x69, 0x74, 0x65, 0x52, 0x65, 0x66, 0x52,
+ 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x06, 0xfa, 0x97, 0x28, 0x02, 0x08, 0x01, 0x12,
+ 0x54, 0x0a, 0x0d, 0x46, 0x69, 0x6e, 0x64, 0x4d, 0x65, 0x72, 0x67, 0x65, 0x42, 0x61, 0x73, 0x65,
+ 0x12, 0x1c, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x46, 0x69, 0x6e, 0x64, 0x4d, 0x65,
+ 0x72, 0x67, 0x65, 0x42, 0x61, 0x73, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1d,
+ 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x46, 0x69, 0x6e, 0x64, 0x4d, 0x65, 0x72, 0x67,
+ 0x65, 0x42, 0x61, 0x73, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x06, 0xfa,
+ 0x97, 0x28, 0x02, 0x08, 0x02, 0x12, 0x4b, 0x0a, 0x0a, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x46,
+ 0x6f, 0x72, 0x6b, 0x12, 0x19, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x43, 0x72, 0x65,
+ 0x61, 0x74, 0x65, 0x46, 0x6f, 0x72, 0x6b, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1a,
+ 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x46, 0x6f,
+ 0x72, 0x6b, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x06, 0xfa, 0x97, 0x28, 0x02,
+ 0x08, 0x01, 0x12, 0x72, 0x0a, 0x17, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, 0x65, 0x70, 0x6f,
+ 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x46, 0x72, 0x6f, 0x6d, 0x55, 0x52, 0x4c, 0x12, 0x26, 0x2e,
+ 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, 0x65, 0x70,
+ 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x46, 0x72, 0x6f, 0x6d, 0x55, 0x52, 0x4c, 0x52, 0x65,
+ 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x27, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x43,
0x72, 0x65, 0x61, 0x74, 0x65, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x46,
- 0x72, 0x6f, 0x6d, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x12, 0x29, 0x2e, 0x67, 0x69, 0x74, 0x61,
- 0x6c, 0x79, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74,
- 0x6f, 0x72, 0x79, 0x46, 0x72, 0x6f, 0x6d, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x52, 0x65, 0x71,
- 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2a, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x43, 0x72,
- 0x65, 0x61, 0x74, 0x65, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x46, 0x72,
- 0x6f, 0x6d, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65,
- 0x22, 0x06, 0xfa, 0x97, 0x28, 0x02, 0x08, 0x01, 0x28, 0x01, 0x12, 0x4a, 0x0a, 0x09, 0x47, 0x65,
- 0x74, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x18, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79,
- 0x2e, 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73,
- 0x74, 0x1a, 0x19, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x47, 0x65, 0x74, 0x43, 0x6f,
- 0x6e, 0x66, 0x69, 0x67, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x06, 0xfa, 0x97,
- 0x28, 0x02, 0x08, 0x02, 0x30, 0x01, 0x12, 0x4e, 0x0a, 0x0b, 0x46, 0x69, 0x6e, 0x64, 0x4c, 0x69,
- 0x63, 0x65, 0x6e, 0x73, 0x65, 0x12, 0x1a, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x46,
- 0x69, 0x6e, 0x64, 0x4c, 0x69, 0x63, 0x65, 0x6e, 0x73, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73,
- 0x74, 0x1a, 0x1b, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x46, 0x69, 0x6e, 0x64, 0x4c,
- 0x69, 0x63, 0x65, 0x6e, 0x73, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x06,
- 0xfa, 0x97, 0x28, 0x02, 0x08, 0x02, 0x12, 0x65, 0x0a, 0x11, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x66,
- 0x6f, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x12, 0x20, 0x2e, 0x67, 0x69,
- 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x41, 0x74, 0x74, 0x72,
- 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x21, 0x2e,
- 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x41, 0x74,
- 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65,
- 0x22, 0x09, 0xfa, 0x97, 0x28, 0x02, 0x08, 0x02, 0x88, 0x02, 0x01, 0x30, 0x01, 0x12, 0x60, 0x0a,
- 0x11, 0x43, 0x61, 0x6c, 0x63, 0x75, 0x6c, 0x61, 0x74, 0x65, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x73,
- 0x75, 0x6d, 0x12, 0x20, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x43, 0x61, 0x6c, 0x63,
- 0x75, 0x6c, 0x61, 0x74, 0x65, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x73, 0x75, 0x6d, 0x52, 0x65, 0x71,
- 0x75, 0x65, 0x73, 0x74, 0x1a, 0x21, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x43, 0x61,
- 0x6c, 0x63, 0x75, 0x6c, 0x61, 0x74, 0x65, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x73, 0x75, 0x6d, 0x52,
- 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x06, 0xfa, 0x97, 0x28, 0x02, 0x08, 0x02, 0x12,
- 0x50, 0x0a, 0x0b, 0x47, 0x65, 0x74, 0x53, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x12, 0x1a,
- 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x6e, 0x61, 0x70, 0x73,
- 0x68, 0x6f, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x67, 0x69, 0x74,
- 0x61, 0x6c, 0x79, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x52,
- 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x06, 0xfa, 0x97, 0x28, 0x02, 0x08, 0x02, 0x30,
- 0x01, 0x12, 0x81, 0x01, 0x0a, 0x1c, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, 0x65, 0x70, 0x6f,
+ 0x72, 0x6f, 0x6d, 0x55, 0x52, 0x4c, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x06,
+ 0xfa, 0x97, 0x28, 0x02, 0x08, 0x01, 0x12, 0x53, 0x0a, 0x0c, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65,
+ 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x12, 0x1b, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e,
+ 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75,
+ 0x65, 0x73, 0x74, 0x1a, 0x1c, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x43, 0x72, 0x65,
+ 0x61, 0x74, 0x65, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73,
+ 0x65, 0x22, 0x06, 0xfa, 0x97, 0x28, 0x02, 0x08, 0x02, 0x30, 0x01, 0x12, 0x76, 0x0a, 0x17, 0x43,
+ 0x72, 0x65, 0x61, 0x74, 0x65, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x46, 0x72, 0x6f, 0x6d, 0x52,
+ 0x65, 0x66, 0x4c, 0x69, 0x73, 0x74, 0x12, 0x26, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e,
+ 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x46, 0x72, 0x6f, 0x6d,
+ 0x52, 0x65, 0x66, 0x4c, 0x69, 0x73, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x27,
+ 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x42, 0x75,
+ 0x6e, 0x64, 0x6c, 0x65, 0x46, 0x72, 0x6f, 0x6d, 0x52, 0x65, 0x66, 0x4c, 0x69, 0x73, 0x74, 0x52,
+ 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x06, 0xfa, 0x97, 0x28, 0x02, 0x08, 0x02, 0x28,
+ 0x01, 0x30, 0x01, 0x12, 0x50, 0x0a, 0x0b, 0x46, 0x65, 0x74, 0x63, 0x68, 0x42, 0x75, 0x6e, 0x64,
+ 0x6c, 0x65, 0x12, 0x1a, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x46, 0x65, 0x74, 0x63,
+ 0x68, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b,
+ 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x46, 0x65, 0x74, 0x63, 0x68, 0x42, 0x75, 0x6e,
+ 0x64, 0x6c, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x06, 0xfa, 0x97, 0x28,
+ 0x02, 0x08, 0x01, 0x28, 0x01, 0x12, 0x7d, 0x0a, 0x1a, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x52,
+ 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x46, 0x72, 0x6f, 0x6d, 0x42, 0x75, 0x6e,
+ 0x64, 0x6c, 0x65, 0x12, 0x29, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x43, 0x72, 0x65,
+ 0x61, 0x74, 0x65, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x46, 0x72, 0x6f,
+ 0x6d, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2a,
+ 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, 0x65,
+ 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x46, 0x72, 0x6f, 0x6d, 0x42, 0x75, 0x6e, 0x64,
+ 0x6c, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x06, 0xfa, 0x97, 0x28, 0x02,
+ 0x08, 0x01, 0x28, 0x01, 0x12, 0x4a, 0x0a, 0x09, 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6e, 0x66, 0x69,
+ 0x67, 0x12, 0x18, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x47, 0x65, 0x74, 0x43, 0x6f,
+ 0x6e, 0x66, 0x69, 0x67, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x19, 0x2e, 0x67, 0x69,
+ 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x65,
+ 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x06, 0xfa, 0x97, 0x28, 0x02, 0x08, 0x02, 0x30, 0x01,
+ 0x12, 0x4e, 0x0a, 0x0b, 0x46, 0x69, 0x6e, 0x64, 0x4c, 0x69, 0x63, 0x65, 0x6e, 0x73, 0x65, 0x12,
+ 0x1a, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x46, 0x69, 0x6e, 0x64, 0x4c, 0x69, 0x63,
+ 0x65, 0x6e, 0x73, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x67, 0x69,
+ 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x46, 0x69, 0x6e, 0x64, 0x4c, 0x69, 0x63, 0x65, 0x6e, 0x73, 0x65,
+ 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x06, 0xfa, 0x97, 0x28, 0x02, 0x08, 0x02,
+ 0x12, 0x65, 0x0a, 0x11, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x41, 0x74, 0x74, 0x72, 0x69,
+ 0x62, 0x75, 0x74, 0x65, 0x73, 0x12, 0x20, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x47,
+ 0x65, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73,
+ 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x21, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79,
+ 0x2e, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74,
+ 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x09, 0xfa, 0x97, 0x28, 0x02,
+ 0x08, 0x02, 0x88, 0x02, 0x01, 0x30, 0x01, 0x12, 0x60, 0x0a, 0x11, 0x43, 0x61, 0x6c, 0x63, 0x75,
+ 0x6c, 0x61, 0x74, 0x65, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x73, 0x75, 0x6d, 0x12, 0x20, 0x2e, 0x67,
+ 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x43, 0x61, 0x6c, 0x63, 0x75, 0x6c, 0x61, 0x74, 0x65, 0x43,
+ 0x68, 0x65, 0x63, 0x6b, 0x73, 0x75, 0x6d, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x21,
+ 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x43, 0x61, 0x6c, 0x63, 0x75, 0x6c, 0x61, 0x74,
+ 0x65, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x73, 0x75, 0x6d, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73,
+ 0x65, 0x22, 0x06, 0xfa, 0x97, 0x28, 0x02, 0x08, 0x02, 0x12, 0x50, 0x0a, 0x0b, 0x47, 0x65, 0x74,
+ 0x53, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x12, 0x1a, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c,
+ 0x79, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x52, 0x65, 0x71,
+ 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x47, 0x65,
+ 0x74, 0x53, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73,
+ 0x65, 0x22, 0x06, 0xfa, 0x97, 0x28, 0x02, 0x08, 0x02, 0x30, 0x01, 0x12, 0x81, 0x01, 0x0a, 0x1c,
+ 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79,
+ 0x46, 0x72, 0x6f, 0x6d, 0x53, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x12, 0x2b, 0x2e, 0x67,
+ 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, 0x65, 0x70, 0x6f,
0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x46, 0x72, 0x6f, 0x6d, 0x53, 0x6e, 0x61, 0x70, 0x73, 0x68,
- 0x6f, 0x74, 0x12, 0x2b, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x43, 0x72, 0x65, 0x61,
- 0x74, 0x65, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x46, 0x72, 0x6f, 0x6d,
- 0x53, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a,
- 0x2c, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x52,
- 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x46, 0x72, 0x6f, 0x6d, 0x53, 0x6e, 0x61,
- 0x70, 0x73, 0x68, 0x6f, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x06, 0xfa,
- 0x97, 0x28, 0x02, 0x08, 0x01, 0x12, 0x56, 0x0a, 0x0d, 0x47, 0x65, 0x74, 0x52, 0x61, 0x77, 0x43,
- 0x68, 0x61, 0x6e, 0x67, 0x65, 0x73, 0x12, 0x1c, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e,
- 0x47, 0x65, 0x74, 0x52, 0x61, 0x77, 0x43, 0x68, 0x61, 0x6e, 0x67, 0x65, 0x73, 0x52, 0x65, 0x71,
- 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1d, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x47, 0x65,
- 0x74, 0x52, 0x61, 0x77, 0x43, 0x68, 0x61, 0x6e, 0x67, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f,
- 0x6e, 0x73, 0x65, 0x22, 0x06, 0xfa, 0x97, 0x28, 0x02, 0x08, 0x02, 0x30, 0x01, 0x12, 0x6b, 0x0a,
- 0x14, 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, 0x46, 0x69, 0x6c, 0x65, 0x73, 0x42, 0x79, 0x43, 0x6f,
- 0x6e, 0x74, 0x65, 0x6e, 0x74, 0x12, 0x23, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x53,
- 0x65, 0x61, 0x72, 0x63, 0x68, 0x46, 0x69, 0x6c, 0x65, 0x73, 0x42, 0x79, 0x43, 0x6f, 0x6e, 0x74,
- 0x65, 0x6e, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x24, 0x2e, 0x67, 0x69, 0x74,
- 0x61, 0x6c, 0x79, 0x2e, 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, 0x46, 0x69, 0x6c, 0x65, 0x73, 0x42,
- 0x79, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x6e, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65,
- 0x22, 0x06, 0xfa, 0x97, 0x28, 0x02, 0x08, 0x02, 0x30, 0x01, 0x12, 0x62, 0x0a, 0x11, 0x53, 0x65,
- 0x61, 0x72, 0x63, 0x68, 0x46, 0x69, 0x6c, 0x65, 0x73, 0x42, 0x79, 0x4e, 0x61, 0x6d, 0x65, 0x12,
- 0x20, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, 0x46,
- 0x69, 0x6c, 0x65, 0x73, 0x42, 0x79, 0x4e, 0x61, 0x6d, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73,
- 0x74, 0x1a, 0x21, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x53, 0x65, 0x61, 0x72, 0x63,
- 0x68, 0x46, 0x69, 0x6c, 0x65, 0x73, 0x42, 0x79, 0x4e, 0x61, 0x6d, 0x65, 0x52, 0x65, 0x73, 0x70,
- 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x06, 0xfa, 0x97, 0x28, 0x02, 0x08, 0x02, 0x30, 0x01, 0x12, 0x68,
- 0x0a, 0x12, 0x52, 0x65, 0x73, 0x74, 0x6f, 0x72, 0x65, 0x43, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x48,
- 0x6f, 0x6f, 0x6b, 0x73, 0x12, 0x21, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x52, 0x65,
- 0x73, 0x74, 0x6f, 0x72, 0x65, 0x43, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x48, 0x6f, 0x6f, 0x6b, 0x73,
- 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x22, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79,
- 0x2e, 0x52, 0x65, 0x73, 0x74, 0x6f, 0x72, 0x65, 0x43, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x48, 0x6f,
- 0x6f, 0x6b, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x09, 0xfa, 0x97, 0x28,
- 0x02, 0x08, 0x01, 0x88, 0x02, 0x01, 0x28, 0x01, 0x12, 0x59, 0x0a, 0x0e, 0x53, 0x65, 0x74, 0x43,
- 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x48, 0x6f, 0x6f, 0x6b, 0x73, 0x12, 0x1d, 0x2e, 0x67, 0x69, 0x74,
- 0x61, 0x6c, 0x79, 0x2e, 0x53, 0x65, 0x74, 0x43, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x48, 0x6f, 0x6f,
- 0x6b, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1e, 0x2e, 0x67, 0x69, 0x74, 0x61,
- 0x6c, 0x79, 0x2e, 0x53, 0x65, 0x74, 0x43, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x48, 0x6f, 0x6f, 0x6b,
- 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x06, 0xfa, 0x97, 0x28, 0x02, 0x08,
- 0x01, 0x28, 0x01, 0x12, 0x65, 0x0a, 0x11, 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x43, 0x75, 0x73,
- 0x74, 0x6f, 0x6d, 0x48, 0x6f, 0x6f, 0x6b, 0x73, 0x12, 0x20, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c,
- 0x79, 0x2e, 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x43, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x48, 0x6f,
- 0x6f, 0x6b, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x21, 0x2e, 0x67, 0x69, 0x74,
- 0x61, 0x6c, 0x79, 0x2e, 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x43, 0x75, 0x73, 0x74, 0x6f, 0x6d,
- 0x48, 0x6f, 0x6f, 0x6b, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x09, 0xfa,
- 0x97, 0x28, 0x02, 0x08, 0x02, 0x88, 0x02, 0x01, 0x30, 0x01, 0x12, 0x59, 0x0a, 0x0e, 0x47, 0x65,
- 0x74, 0x43, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x48, 0x6f, 0x6f, 0x6b, 0x73, 0x12, 0x1d, 0x2e, 0x67,
- 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x47, 0x65, 0x74, 0x43, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x48,
- 0x6f, 0x6f, 0x6b, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1e, 0x2e, 0x67, 0x69,
- 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x47, 0x65, 0x74, 0x43, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x48, 0x6f,
- 0x6f, 0x6b, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x06, 0xfa, 0x97, 0x28,
- 0x02, 0x08, 0x02, 0x30, 0x01, 0x12, 0x6f, 0x0a, 0x16, 0x47, 0x65, 0x74, 0x4f, 0x62, 0x6a, 0x65,
- 0x63, 0x74, 0x44, 0x69, 0x72, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x79, 0x53, 0x69, 0x7a, 0x65, 0x12,
- 0x25, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x47, 0x65, 0x74, 0x4f, 0x62, 0x6a, 0x65,
- 0x63, 0x74, 0x44, 0x69, 0x72, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x79, 0x53, 0x69, 0x7a, 0x65, 0x52,
- 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x26, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e,
- 0x47, 0x65, 0x74, 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x44, 0x69, 0x72, 0x65, 0x63, 0x74, 0x6f,
- 0x72, 0x79, 0x53, 0x69, 0x7a, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x06,
- 0xfa, 0x97, 0x28, 0x02, 0x08, 0x02, 0x12, 0x5d, 0x0a, 0x10, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65,
- 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x12, 0x1f, 0x2e, 0x67, 0x69, 0x74,
- 0x61, 0x6c, 0x79, 0x2e, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69,
- 0x74, 0x6f, 0x72, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x20, 0x2e, 0x67, 0x69,
- 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x52, 0x65, 0x70, 0x6f, 0x73,
- 0x69, 0x74, 0x6f, 0x72, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x06, 0xfa,
- 0x97, 0x28, 0x02, 0x08, 0x01, 0x12, 0x5d, 0x0a, 0x10, 0x52, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x52,
- 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x12, 0x1f, 0x2e, 0x67, 0x69, 0x74, 0x61,
- 0x6c, 0x79, 0x2e, 0x52, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74,
- 0x6f, 0x72, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x20, 0x2e, 0x67, 0x69, 0x74,
- 0x61, 0x6c, 0x79, 0x2e, 0x52, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69,
- 0x74, 0x6f, 0x72, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x06, 0xfa, 0x97,
- 0x28, 0x02, 0x08, 0x01, 0x12, 0x66, 0x0a, 0x13, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74,
- 0x65, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x12, 0x22, 0x2e, 0x67, 0x69,
- 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x65, 0x52, 0x65,
- 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a,
- 0x23, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61,
- 0x74, 0x65, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x52, 0x65, 0x73, 0x70,
- 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x06, 0xfa, 0x97, 0x28, 0x02, 0x08, 0x01, 0x12, 0x63, 0x0a, 0x12,
- 0x4f, 0x70, 0x74, 0x69, 0x6d, 0x69, 0x7a, 0x65, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f,
- 0x72, 0x79, 0x12, 0x21, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x4f, 0x70, 0x74, 0x69,
- 0x6d, 0x69, 0x7a, 0x65, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x52, 0x65,
- 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x22, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x4f,
- 0x70, 0x74, 0x69, 0x6d, 0x69, 0x7a, 0x65, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72,
- 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x06, 0xfa, 0x97, 0x28, 0x02, 0x08,
- 0x03, 0x12, 0x72, 0x0a, 0x17, 0x50, 0x72, 0x75, 0x6e, 0x65, 0x55, 0x6e, 0x72, 0x65, 0x61, 0x63,
- 0x68, 0x61, 0x62, 0x6c, 0x65, 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x73, 0x12, 0x26, 0x2e, 0x67,
- 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x50, 0x72, 0x75, 0x6e, 0x65, 0x55, 0x6e, 0x72, 0x65, 0x61,
- 0x63, 0x68, 0x61, 0x62, 0x6c, 0x65, 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x73, 0x52, 0x65, 0x71,
- 0x75, 0x65, 0x73, 0x74, 0x1a, 0x27, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x50, 0x72,
- 0x75, 0x6e, 0x65, 0x55, 0x6e, 0x72, 0x65, 0x61, 0x63, 0x68, 0x61, 0x62, 0x6c, 0x65, 0x4f, 0x62,
- 0x6a, 0x65, 0x63, 0x74, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x06, 0xfa,
- 0x97, 0x28, 0x02, 0x08, 0x03, 0x12, 0x51, 0x0a, 0x0b, 0x53, 0x65, 0x74, 0x46, 0x75, 0x6c, 0x6c,
- 0x50, 0x61, 0x74, 0x68, 0x12, 0x1a, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x53, 0x65,
- 0x74, 0x46, 0x75, 0x6c, 0x6c, 0x50, 0x61, 0x74, 0x68, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74,
- 0x1a, 0x1b, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x53, 0x65, 0x74, 0x46, 0x75, 0x6c,
- 0x6c, 0x50, 0x61, 0x74, 0x68, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x09, 0xfa,
- 0x97, 0x28, 0x02, 0x08, 0x01, 0x88, 0x02, 0x01, 0x12, 0x48, 0x0a, 0x08, 0x46, 0x75, 0x6c, 0x6c,
- 0x50, 0x61, 0x74, 0x68, 0x12, 0x17, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x46, 0x75,
- 0x6c, 0x6c, 0x50, 0x61, 0x74, 0x68, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x18, 0x2e,
- 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x46, 0x75, 0x6c, 0x6c, 0x50, 0x61, 0x74, 0x68, 0x52,
+ 0x6f, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2c, 0x2e, 0x67, 0x69, 0x74, 0x61,
+ 0x6c, 0x79, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74,
+ 0x6f, 0x72, 0x79, 0x46, 0x72, 0x6f, 0x6d, 0x53, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x52,
+ 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x06, 0xfa, 0x97, 0x28, 0x02, 0x08, 0x01, 0x12,
+ 0x56, 0x0a, 0x0d, 0x47, 0x65, 0x74, 0x52, 0x61, 0x77, 0x43, 0x68, 0x61, 0x6e, 0x67, 0x65, 0x73,
+ 0x12, 0x1c, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x47, 0x65, 0x74, 0x52, 0x61, 0x77,
+ 0x43, 0x68, 0x61, 0x6e, 0x67, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1d,
+ 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x47, 0x65, 0x74, 0x52, 0x61, 0x77, 0x43, 0x68,
+ 0x61, 0x6e, 0x67, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x06, 0xfa,
+ 0x97, 0x28, 0x02, 0x08, 0x02, 0x30, 0x01, 0x12, 0x6b, 0x0a, 0x14, 0x53, 0x65, 0x61, 0x72, 0x63,
+ 0x68, 0x46, 0x69, 0x6c, 0x65, 0x73, 0x42, 0x79, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x6e, 0x74, 0x12,
+ 0x23, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, 0x46,
+ 0x69, 0x6c, 0x65, 0x73, 0x42, 0x79, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x6e, 0x74, 0x52, 0x65, 0x71,
+ 0x75, 0x65, 0x73, 0x74, 0x1a, 0x24, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x53, 0x65,
+ 0x61, 0x72, 0x63, 0x68, 0x46, 0x69, 0x6c, 0x65, 0x73, 0x42, 0x79, 0x43, 0x6f, 0x6e, 0x74, 0x65,
+ 0x6e, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x06, 0xfa, 0x97, 0x28, 0x02,
+ 0x08, 0x02, 0x30, 0x01, 0x12, 0x62, 0x0a, 0x11, 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, 0x46, 0x69,
+ 0x6c, 0x65, 0x73, 0x42, 0x79, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x20, 0x2e, 0x67, 0x69, 0x74, 0x61,
+ 0x6c, 0x79, 0x2e, 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, 0x46, 0x69, 0x6c, 0x65, 0x73, 0x42, 0x79,
+ 0x4e, 0x61, 0x6d, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x21, 0x2e, 0x67, 0x69,
+ 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, 0x46, 0x69, 0x6c, 0x65, 0x73,
+ 0x42, 0x79, 0x4e, 0x61, 0x6d, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x06,
+ 0xfa, 0x97, 0x28, 0x02, 0x08, 0x02, 0x30, 0x01, 0x12, 0x68, 0x0a, 0x12, 0x52, 0x65, 0x73, 0x74,
+ 0x6f, 0x72, 0x65, 0x43, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x48, 0x6f, 0x6f, 0x6b, 0x73, 0x12, 0x21,
+ 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x52, 0x65, 0x73, 0x74, 0x6f, 0x72, 0x65, 0x43,
+ 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x48, 0x6f, 0x6f, 0x6b, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73,
+ 0x74, 0x1a, 0x22, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x52, 0x65, 0x73, 0x74, 0x6f,
+ 0x72, 0x65, 0x43, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x48, 0x6f, 0x6f, 0x6b, 0x73, 0x52, 0x65, 0x73,
+ 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x09, 0xfa, 0x97, 0x28, 0x02, 0x08, 0x01, 0x88, 0x02, 0x01,
+ 0x28, 0x01, 0x12, 0x59, 0x0a, 0x0e, 0x53, 0x65, 0x74, 0x43, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x48,
+ 0x6f, 0x6f, 0x6b, 0x73, 0x12, 0x1d, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x53, 0x65,
+ 0x74, 0x43, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x48, 0x6f, 0x6f, 0x6b, 0x73, 0x52, 0x65, 0x71, 0x75,
+ 0x65, 0x73, 0x74, 0x1a, 0x1e, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x53, 0x65, 0x74,
+ 0x43, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x48, 0x6f, 0x6f, 0x6b, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f,
+ 0x6e, 0x73, 0x65, 0x22, 0x06, 0xfa, 0x97, 0x28, 0x02, 0x08, 0x01, 0x28, 0x01, 0x12, 0x65, 0x0a,
+ 0x11, 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x43, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x48, 0x6f, 0x6f,
+ 0x6b, 0x73, 0x12, 0x20, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x42, 0x61, 0x63, 0x6b,
+ 0x75, 0x70, 0x43, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x48, 0x6f, 0x6f, 0x6b, 0x73, 0x52, 0x65, 0x71,
+ 0x75, 0x65, 0x73, 0x74, 0x1a, 0x21, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x42, 0x61,
+ 0x63, 0x6b, 0x75, 0x70, 0x43, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x48, 0x6f, 0x6f, 0x6b, 0x73, 0x52,
0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x09, 0xfa, 0x97, 0x28, 0x02, 0x08, 0x02, 0x88,
- 0x02, 0x01, 0x12, 0x4a, 0x0a, 0x09, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x41, 0x6c, 0x6c, 0x12,
- 0x18, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x41,
- 0x6c, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x19, 0x2e, 0x67, 0x69, 0x74, 0x61,
- 0x6c, 0x79, 0x2e, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x41, 0x6c, 0x6c, 0x52, 0x65, 0x73, 0x70,
- 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x08, 0xfa, 0x97, 0x28, 0x04, 0x08, 0x01, 0x10, 0x02, 0x12, 0x5d,
- 0x0a, 0x10, 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f,
- 0x72, 0x79, 0x12, 0x1f, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x42, 0x61, 0x63, 0x6b,
- 0x75, 0x70, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x52, 0x65, 0x71, 0x75,
- 0x65, 0x73, 0x74, 0x1a, 0x20, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x42, 0x61, 0x63,
- 0x6b, 0x75, 0x70, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x52, 0x65, 0x73,
- 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x06, 0xfa, 0x97, 0x28, 0x02, 0x08, 0x02, 0x12, 0x60, 0x0a,
- 0x11, 0x52, 0x65, 0x73, 0x74, 0x6f, 0x72, 0x65, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f,
- 0x72, 0x79, 0x12, 0x20, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x52, 0x65, 0x73, 0x74,
- 0x6f, 0x72, 0x65, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x52, 0x65, 0x71,
- 0x75, 0x65, 0x73, 0x74, 0x1a, 0x21, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x52, 0x65,
- 0x73, 0x74, 0x6f, 0x72, 0x65, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x52,
+ 0x02, 0x01, 0x30, 0x01, 0x12, 0x59, 0x0a, 0x0e, 0x47, 0x65, 0x74, 0x43, 0x75, 0x73, 0x74, 0x6f,
+ 0x6d, 0x48, 0x6f, 0x6f, 0x6b, 0x73, 0x12, 0x1d, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e,
+ 0x47, 0x65, 0x74, 0x43, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x48, 0x6f, 0x6f, 0x6b, 0x73, 0x52, 0x65,
+ 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1e, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x47,
+ 0x65, 0x74, 0x43, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x48, 0x6f, 0x6f, 0x6b, 0x73, 0x52, 0x65, 0x73,
+ 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x06, 0xfa, 0x97, 0x28, 0x02, 0x08, 0x02, 0x30, 0x01, 0x12,
+ 0x6f, 0x0a, 0x16, 0x47, 0x65, 0x74, 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x44, 0x69, 0x72, 0x65,
+ 0x63, 0x74, 0x6f, 0x72, 0x79, 0x53, 0x69, 0x7a, 0x65, 0x12, 0x25, 0x2e, 0x67, 0x69, 0x74, 0x61,
+ 0x6c, 0x79, 0x2e, 0x47, 0x65, 0x74, 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x44, 0x69, 0x72, 0x65,
+ 0x63, 0x74, 0x6f, 0x72, 0x79, 0x53, 0x69, 0x7a, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74,
+ 0x1a, 0x26, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x47, 0x65, 0x74, 0x4f, 0x62, 0x6a,
+ 0x65, 0x63, 0x74, 0x44, 0x69, 0x72, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x79, 0x53, 0x69, 0x7a, 0x65,
+ 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x06, 0xfa, 0x97, 0x28, 0x02, 0x08, 0x02,
+ 0x12, 0x5d, 0x0a, 0x10, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69,
+ 0x74, 0x6f, 0x72, 0x79, 0x12, 0x1f, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x52, 0x65,
+ 0x6d, 0x6f, 0x76, 0x65, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x52, 0x65,
+ 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x20, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x52,
+ 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x52,
0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x06, 0xfa, 0x97, 0x28, 0x02, 0x08, 0x01, 0x12,
- 0x60, 0x0a, 0x11, 0x47, 0x65, 0x74, 0x46, 0x69, 0x6c, 0x65, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62,
- 0x75, 0x74, 0x65, 0x73, 0x12, 0x20, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x47, 0x65,
- 0x74, 0x46, 0x69, 0x6c, 0x65, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x52,
- 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x21, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e,
- 0x47, 0x65, 0x74, 0x46, 0x69, 0x6c, 0x65, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65,
- 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x06, 0xfa, 0x97, 0x28, 0x02, 0x08,
- 0x02, 0x42, 0x34, 0x5a, 0x32, 0x67, 0x69, 0x74, 0x6c, 0x61, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f,
- 0x67, 0x69, 0x74, 0x6c, 0x61, 0x62, 0x2d, 0x6f, 0x72, 0x67, 0x2f, 0x67, 0x69, 0x74, 0x61, 0x6c,
- 0x79, 0x2f, 0x76, 0x31, 0x36, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x67, 0x6f, 0x2f, 0x67,
- 0x69, 0x74, 0x61, 0x6c, 0x79, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
+ 0x66, 0x0a, 0x13, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x65, 0x52, 0x65, 0x70, 0x6f,
+ 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x12, 0x22, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e,
+ 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x65, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74,
+ 0x6f, 0x72, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x23, 0x2e, 0x67, 0x69, 0x74,
+ 0x61, 0x6c, 0x79, 0x2e, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x65, 0x52, 0x65, 0x70,
+ 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22,
+ 0x06, 0xfa, 0x97, 0x28, 0x02, 0x08, 0x01, 0x12, 0x63, 0x0a, 0x12, 0x4f, 0x70, 0x74, 0x69, 0x6d,
+ 0x69, 0x7a, 0x65, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x12, 0x21, 0x2e,
+ 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x4f, 0x70, 0x74, 0x69, 0x6d, 0x69, 0x7a, 0x65, 0x52,
+ 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74,
+ 0x1a, 0x22, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x4f, 0x70, 0x74, 0x69, 0x6d, 0x69,
+ 0x7a, 0x65, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x52, 0x65, 0x73, 0x70,
+ 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x06, 0xfa, 0x97, 0x28, 0x02, 0x08, 0x03, 0x12, 0x72, 0x0a, 0x17,
+ 0x50, 0x72, 0x75, 0x6e, 0x65, 0x55, 0x6e, 0x72, 0x65, 0x61, 0x63, 0x68, 0x61, 0x62, 0x6c, 0x65,
+ 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x73, 0x12, 0x26, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79,
+ 0x2e, 0x50, 0x72, 0x75, 0x6e, 0x65, 0x55, 0x6e, 0x72, 0x65, 0x61, 0x63, 0x68, 0x61, 0x62, 0x6c,
+ 0x65, 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a,
+ 0x27, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x50, 0x72, 0x75, 0x6e, 0x65, 0x55, 0x6e,
+ 0x72, 0x65, 0x61, 0x63, 0x68, 0x61, 0x62, 0x6c, 0x65, 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x73,
+ 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x06, 0xfa, 0x97, 0x28, 0x02, 0x08, 0x03,
+ 0x12, 0x51, 0x0a, 0x0b, 0x53, 0x65, 0x74, 0x46, 0x75, 0x6c, 0x6c, 0x50, 0x61, 0x74, 0x68, 0x12,
+ 0x1a, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x53, 0x65, 0x74, 0x46, 0x75, 0x6c, 0x6c,
+ 0x50, 0x61, 0x74, 0x68, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x67, 0x69,
+ 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x53, 0x65, 0x74, 0x46, 0x75, 0x6c, 0x6c, 0x50, 0x61, 0x74, 0x68,
+ 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x09, 0xfa, 0x97, 0x28, 0x02, 0x08, 0x01,
+ 0x88, 0x02, 0x01, 0x12, 0x48, 0x0a, 0x08, 0x46, 0x75, 0x6c, 0x6c, 0x50, 0x61, 0x74, 0x68, 0x12,
+ 0x17, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x46, 0x75, 0x6c, 0x6c, 0x50, 0x61, 0x74,
+ 0x68, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x18, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c,
+ 0x79, 0x2e, 0x46, 0x75, 0x6c, 0x6c, 0x50, 0x61, 0x74, 0x68, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e,
+ 0x73, 0x65, 0x22, 0x09, 0xfa, 0x97, 0x28, 0x02, 0x08, 0x02, 0x88, 0x02, 0x01, 0x12, 0x4a, 0x0a,
+ 0x09, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x41, 0x6c, 0x6c, 0x12, 0x18, 0x2e, 0x67, 0x69, 0x74,
+ 0x61, 0x6c, 0x79, 0x2e, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x41, 0x6c, 0x6c, 0x52, 0x65, 0x71,
+ 0x75, 0x65, 0x73, 0x74, 0x1a, 0x19, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x52, 0x65,
+ 0x6d, 0x6f, 0x76, 0x65, 0x41, 0x6c, 0x6c, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22,
+ 0x08, 0xfa, 0x97, 0x28, 0x04, 0x08, 0x01, 0x10, 0x02, 0x12, 0x5d, 0x0a, 0x10, 0x42, 0x61, 0x63,
+ 0x6b, 0x75, 0x70, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x12, 0x1f, 0x2e,
+ 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x52, 0x65, 0x70,
+ 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x20,
+ 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x52, 0x65,
+ 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65,
+ 0x22, 0x06, 0xfa, 0x97, 0x28, 0x02, 0x08, 0x02, 0x12, 0x60, 0x0a, 0x11, 0x52, 0x65, 0x73, 0x74,
+ 0x6f, 0x72, 0x65, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x12, 0x20, 0x2e,
+ 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x52, 0x65, 0x73, 0x74, 0x6f, 0x72, 0x65, 0x52, 0x65,
+ 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a,
+ 0x21, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x52, 0x65, 0x73, 0x74, 0x6f, 0x72, 0x65,
+ 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e,
+ 0x73, 0x65, 0x22, 0x06, 0xfa, 0x97, 0x28, 0x02, 0x08, 0x01, 0x12, 0x60, 0x0a, 0x11, 0x47, 0x65,
+ 0x74, 0x46, 0x69, 0x6c, 0x65, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x12,
+ 0x20, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x47, 0x65, 0x74, 0x46, 0x69, 0x6c, 0x65,
+ 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73,
+ 0x74, 0x1a, 0x21, 0x2e, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x47, 0x65, 0x74, 0x46, 0x69,
+ 0x6c, 0x65, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70,
+ 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x06, 0xfa, 0x97, 0x28, 0x02, 0x08, 0x02, 0x42, 0x34, 0x5a, 0x32,
+ 0x67, 0x69, 0x74, 0x6c, 0x61, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x67, 0x69, 0x74, 0x6c, 0x61,
+ 0x62, 0x2d, 0x6f, 0x72, 0x67, 0x2f, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2f, 0x76, 0x31, 0x36,
+ 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x67, 0x6f, 0x2f, 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79,
+ 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
}
var (
@@ -6572,7 +6460,7 @@ func file_repository_proto_rawDescGZIP() []byte {
}
var file_repository_proto_enumTypes = make([]protoimpl.EnumInfo, 3)
-var file_repository_proto_msgTypes = make([]protoimpl.MessageInfo, 97)
+var file_repository_proto_msgTypes = make([]protoimpl.MessageInfo, 95)
var file_repository_proto_goTypes = []interface{}{
(GetArchiveRequest_Format)(0), // 0: gitaly.GetArchiveRequest.Format
(GetRawChangesResponse_RawChange_Operation)(0), // 1: gitaly.GetRawChangesResponse.RawChange.Operation
@@ -6648,190 +6536,185 @@ var file_repository_proto_goTypes = []interface{}{
(*GetObjectDirectorySizeResponse)(nil), // 71: gitaly.GetObjectDirectorySizeResponse
(*RemoveRepositoryRequest)(nil), // 72: gitaly.RemoveRepositoryRequest
(*RemoveRepositoryResponse)(nil), // 73: gitaly.RemoveRepositoryResponse
- (*RenameRepositoryRequest)(nil), // 74: gitaly.RenameRepositoryRequest
- (*RenameRepositoryResponse)(nil), // 75: gitaly.RenameRepositoryResponse
- (*ReplicateRepositoryRequest)(nil), // 76: gitaly.ReplicateRepositoryRequest
- (*ReplicateRepositoryResponse)(nil), // 77: gitaly.ReplicateRepositoryResponse
- (*OptimizeRepositoryRequest)(nil), // 78: gitaly.OptimizeRepositoryRequest
- (*OptimizeRepositoryResponse)(nil), // 79: gitaly.OptimizeRepositoryResponse
- (*PruneUnreachableObjectsRequest)(nil), // 80: gitaly.PruneUnreachableObjectsRequest
- (*PruneUnreachableObjectsResponse)(nil), // 81: gitaly.PruneUnreachableObjectsResponse
- (*SetFullPathRequest)(nil), // 82: gitaly.SetFullPathRequest
- (*SetFullPathResponse)(nil), // 83: gitaly.SetFullPathResponse
- (*FullPathRequest)(nil), // 84: gitaly.FullPathRequest
- (*FullPathResponse)(nil), // 85: gitaly.FullPathResponse
- (*RemoveAllRequest)(nil), // 86: gitaly.RemoveAllRequest
- (*RemoveAllResponse)(nil), // 87: gitaly.RemoveAllResponse
- (*BackupRepositoryRequest)(nil), // 88: gitaly.BackupRepositoryRequest
- (*BackupRepositoryResponse)(nil), // 89: gitaly.BackupRepositoryResponse
- (*RestoreRepositoryRequest)(nil), // 90: gitaly.RestoreRepositoryRequest
- (*RestoreRepositoryResponse)(nil), // 91: gitaly.RestoreRepositoryResponse
- (*GetFileAttributesRequest)(nil), // 92: gitaly.GetFileAttributesRequest
- (*GetFileAttributesResponse)(nil), // 93: gitaly.GetFileAttributesResponse
- (*RepositoryInfoResponse_ReferencesInfo)(nil), // 94: gitaly.RepositoryInfoResponse.ReferencesInfo
- (*RepositoryInfoResponse_ObjectsInfo)(nil), // 95: gitaly.RepositoryInfoResponse.ObjectsInfo
- (*GetRawChangesResponse_RawChange)(nil), // 96: gitaly.GetRawChangesResponse.RawChange
- (*BackupRepositoryResponse_SkippedError)(nil), // 97: gitaly.BackupRepositoryResponse.SkippedError
- (*RestoreRepositoryResponse_SkippedError)(nil), // 98: gitaly.RestoreRepositoryResponse.SkippedError
- (*GetFileAttributesResponse_AttributeInfo)(nil), // 99: gitaly.GetFileAttributesResponse.AttributeInfo
- (*Repository)(nil), // 100: gitaly.Repository
- (ObjectFormat)(0), // 101: gitaly.ObjectFormat
+ (*ReplicateRepositoryRequest)(nil), // 74: gitaly.ReplicateRepositoryRequest
+ (*ReplicateRepositoryResponse)(nil), // 75: gitaly.ReplicateRepositoryResponse
+ (*OptimizeRepositoryRequest)(nil), // 76: gitaly.OptimizeRepositoryRequest
+ (*OptimizeRepositoryResponse)(nil), // 77: gitaly.OptimizeRepositoryResponse
+ (*PruneUnreachableObjectsRequest)(nil), // 78: gitaly.PruneUnreachableObjectsRequest
+ (*PruneUnreachableObjectsResponse)(nil), // 79: gitaly.PruneUnreachableObjectsResponse
+ (*SetFullPathRequest)(nil), // 80: gitaly.SetFullPathRequest
+ (*SetFullPathResponse)(nil), // 81: gitaly.SetFullPathResponse
+ (*FullPathRequest)(nil), // 82: gitaly.FullPathRequest
+ (*FullPathResponse)(nil), // 83: gitaly.FullPathResponse
+ (*RemoveAllRequest)(nil), // 84: gitaly.RemoveAllRequest
+ (*RemoveAllResponse)(nil), // 85: gitaly.RemoveAllResponse
+ (*BackupRepositoryRequest)(nil), // 86: gitaly.BackupRepositoryRequest
+ (*BackupRepositoryResponse)(nil), // 87: gitaly.BackupRepositoryResponse
+ (*RestoreRepositoryRequest)(nil), // 88: gitaly.RestoreRepositoryRequest
+ (*RestoreRepositoryResponse)(nil), // 89: gitaly.RestoreRepositoryResponse
+ (*GetFileAttributesRequest)(nil), // 90: gitaly.GetFileAttributesRequest
+ (*GetFileAttributesResponse)(nil), // 91: gitaly.GetFileAttributesResponse
+ (*RepositoryInfoResponse_ReferencesInfo)(nil), // 92: gitaly.RepositoryInfoResponse.ReferencesInfo
+ (*RepositoryInfoResponse_ObjectsInfo)(nil), // 93: gitaly.RepositoryInfoResponse.ObjectsInfo
+ (*GetRawChangesResponse_RawChange)(nil), // 94: gitaly.GetRawChangesResponse.RawChange
+ (*BackupRepositoryResponse_SkippedError)(nil), // 95: gitaly.BackupRepositoryResponse.SkippedError
+ (*RestoreRepositoryResponse_SkippedError)(nil), // 96: gitaly.RestoreRepositoryResponse.SkippedError
+ (*GetFileAttributesResponse_AttributeInfo)(nil), // 97: gitaly.GetFileAttributesResponse.AttributeInfo
+ (*Repository)(nil), // 98: gitaly.Repository
+ (ObjectFormat)(0), // 99: gitaly.ObjectFormat
}
var file_repository_proto_depIdxs = []int32{
- 100, // 0: gitaly.RepositoryExistsRequest.repository:type_name -> gitaly.Repository
- 100, // 1: gitaly.RepositorySizeRequest.repository:type_name -> gitaly.Repository
- 100, // 2: gitaly.RepositoryInfoRequest.repository:type_name -> gitaly.Repository
- 94, // 3: gitaly.RepositoryInfoResponse.references:type_name -> gitaly.RepositoryInfoResponse.ReferencesInfo
- 95, // 4: gitaly.RepositoryInfoResponse.objects:type_name -> gitaly.RepositoryInfoResponse.ObjectsInfo
- 100, // 5: gitaly.ObjectsSizeRequest.repository:type_name -> gitaly.Repository
- 100, // 6: gitaly.ObjectFormatRequest.repository:type_name -> gitaly.Repository
- 101, // 7: gitaly.ObjectFormatResponse.format:type_name -> gitaly.ObjectFormat
- 100, // 8: gitaly.ApplyGitattributesRequest.repository:type_name -> gitaly.Repository
- 100, // 9: gitaly.FetchBundleRequest.repository:type_name -> gitaly.Repository
- 100, // 10: gitaly.FetchRemoteRequest.repository:type_name -> gitaly.Repository
+ 98, // 0: gitaly.RepositoryExistsRequest.repository:type_name -> gitaly.Repository
+ 98, // 1: gitaly.RepositorySizeRequest.repository:type_name -> gitaly.Repository
+ 98, // 2: gitaly.RepositoryInfoRequest.repository:type_name -> gitaly.Repository
+ 92, // 3: gitaly.RepositoryInfoResponse.references:type_name -> gitaly.RepositoryInfoResponse.ReferencesInfo
+ 93, // 4: gitaly.RepositoryInfoResponse.objects:type_name -> gitaly.RepositoryInfoResponse.ObjectsInfo
+ 98, // 5: gitaly.ObjectsSizeRequest.repository:type_name -> gitaly.Repository
+ 98, // 6: gitaly.ObjectFormatRequest.repository:type_name -> gitaly.Repository
+ 99, // 7: gitaly.ObjectFormatResponse.format:type_name -> gitaly.ObjectFormat
+ 98, // 8: gitaly.ApplyGitattributesRequest.repository:type_name -> gitaly.Repository
+ 98, // 9: gitaly.FetchBundleRequest.repository:type_name -> gitaly.Repository
+ 98, // 10: gitaly.FetchRemoteRequest.repository:type_name -> gitaly.Repository
69, // 11: gitaly.FetchRemoteRequest.remote_params:type_name -> gitaly.Remote
- 100, // 12: gitaly.CreateRepositoryRequest.repository:type_name -> gitaly.Repository
- 101, // 13: gitaly.CreateRepositoryRequest.object_format:type_name -> gitaly.ObjectFormat
- 100, // 14: gitaly.GetArchiveRequest.repository:type_name -> gitaly.Repository
+ 98, // 12: gitaly.CreateRepositoryRequest.repository:type_name -> gitaly.Repository
+ 99, // 13: gitaly.CreateRepositoryRequest.object_format:type_name -> gitaly.ObjectFormat
+ 98, // 14: gitaly.GetArchiveRequest.repository:type_name -> gitaly.Repository
0, // 15: gitaly.GetArchiveRequest.format:type_name -> gitaly.GetArchiveRequest.Format
- 100, // 16: gitaly.HasLocalBranchesRequest.repository:type_name -> gitaly.Repository
- 100, // 17: gitaly.FetchSourceBranchRequest.repository:type_name -> gitaly.Repository
- 100, // 18: gitaly.FetchSourceBranchRequest.source_repository:type_name -> gitaly.Repository
- 100, // 19: gitaly.FsckRequest.repository:type_name -> gitaly.Repository
- 100, // 20: gitaly.WriteRefRequest.repository:type_name -> gitaly.Repository
- 100, // 21: gitaly.FindMergeBaseRequest.repository:type_name -> gitaly.Repository
- 100, // 22: gitaly.CreateForkRequest.repository:type_name -> gitaly.Repository
- 100, // 23: gitaly.CreateForkRequest.source_repository:type_name -> gitaly.Repository
- 100, // 24: gitaly.CreateRepositoryFromURLRequest.repository:type_name -> gitaly.Repository
- 100, // 25: gitaly.CreateBundleRequest.repository:type_name -> gitaly.Repository
- 100, // 26: gitaly.CreateBundleFromRefListRequest.repository:type_name -> gitaly.Repository
- 100, // 27: gitaly.GetConfigRequest.repository:type_name -> gitaly.Repository
- 100, // 28: gitaly.RestoreCustomHooksRequest.repository:type_name -> gitaly.Repository
- 100, // 29: gitaly.SetCustomHooksRequest.repository:type_name -> gitaly.Repository
- 100, // 30: gitaly.BackupCustomHooksRequest.repository:type_name -> gitaly.Repository
- 100, // 31: gitaly.GetCustomHooksRequest.repository:type_name -> gitaly.Repository
- 100, // 32: gitaly.CreateRepositoryFromBundleRequest.repository:type_name -> gitaly.Repository
- 100, // 33: gitaly.FindLicenseRequest.repository:type_name -> gitaly.Repository
- 100, // 34: gitaly.GetInfoAttributesRequest.repository:type_name -> gitaly.Repository
- 100, // 35: gitaly.CalculateChecksumRequest.repository:type_name -> gitaly.Repository
- 100, // 36: gitaly.GetSnapshotRequest.repository:type_name -> gitaly.Repository
- 100, // 37: gitaly.CreateRepositoryFromSnapshotRequest.repository:type_name -> gitaly.Repository
- 100, // 38: gitaly.GetRawChangesRequest.repository:type_name -> gitaly.Repository
- 96, // 39: gitaly.GetRawChangesResponse.raw_changes:type_name -> gitaly.GetRawChangesResponse.RawChange
- 100, // 40: gitaly.SearchFilesByNameRequest.repository:type_name -> gitaly.Repository
- 100, // 41: gitaly.SearchFilesByContentRequest.repository:type_name -> gitaly.Repository
- 100, // 42: gitaly.GetObjectDirectorySizeRequest.repository:type_name -> gitaly.Repository
- 100, // 43: gitaly.RemoveRepositoryRequest.repository:type_name -> gitaly.Repository
- 100, // 44: gitaly.RenameRepositoryRequest.repository:type_name -> gitaly.Repository
- 100, // 45: gitaly.ReplicateRepositoryRequest.repository:type_name -> gitaly.Repository
- 100, // 46: gitaly.ReplicateRepositoryRequest.source:type_name -> gitaly.Repository
- 100, // 47: gitaly.OptimizeRepositoryRequest.repository:type_name -> gitaly.Repository
- 2, // 48: gitaly.OptimizeRepositoryRequest.strategy:type_name -> gitaly.OptimizeRepositoryRequest.Strategy
- 100, // 49: gitaly.PruneUnreachableObjectsRequest.repository:type_name -> gitaly.Repository
- 100, // 50: gitaly.SetFullPathRequest.repository:type_name -> gitaly.Repository
- 100, // 51: gitaly.FullPathRequest.repository:type_name -> gitaly.Repository
- 100, // 52: gitaly.BackupRepositoryRequest.repository:type_name -> gitaly.Repository
- 100, // 53: gitaly.BackupRepositoryRequest.vanity_repository:type_name -> gitaly.Repository
- 100, // 54: gitaly.RestoreRepositoryRequest.repository:type_name -> gitaly.Repository
- 100, // 55: gitaly.RestoreRepositoryRequest.vanity_repository:type_name -> gitaly.Repository
- 100, // 56: gitaly.GetFileAttributesRequest.repository:type_name -> gitaly.Repository
- 99, // 57: gitaly.GetFileAttributesResponse.attribute_infos:type_name -> gitaly.GetFileAttributesResponse.AttributeInfo
- 1, // 58: gitaly.GetRawChangesResponse.RawChange.operation:type_name -> gitaly.GetRawChangesResponse.RawChange.Operation
- 3, // 59: gitaly.RepositoryService.RepositoryExists:input_type -> gitaly.RepositoryExistsRequest
- 5, // 60: gitaly.RepositoryService.RepositorySize:input_type -> gitaly.RepositorySizeRequest
- 7, // 61: gitaly.RepositoryService.RepositoryInfo:input_type -> gitaly.RepositoryInfoRequest
- 9, // 62: gitaly.RepositoryService.ObjectsSize:input_type -> gitaly.ObjectsSizeRequest
- 11, // 63: gitaly.RepositoryService.ObjectFormat:input_type -> gitaly.ObjectFormatRequest
- 13, // 64: gitaly.RepositoryService.ApplyGitattributes:input_type -> gitaly.ApplyGitattributesRequest
- 17, // 65: gitaly.RepositoryService.FetchRemote:input_type -> gitaly.FetchRemoteRequest
- 19, // 66: gitaly.RepositoryService.CreateRepository:input_type -> gitaly.CreateRepositoryRequest
- 21, // 67: gitaly.RepositoryService.GetArchive:input_type -> gitaly.GetArchiveRequest
- 23, // 68: gitaly.RepositoryService.HasLocalBranches:input_type -> gitaly.HasLocalBranchesRequest
- 25, // 69: gitaly.RepositoryService.FetchSourceBranch:input_type -> gitaly.FetchSourceBranchRequest
- 27, // 70: gitaly.RepositoryService.Fsck:input_type -> gitaly.FsckRequest
- 29, // 71: gitaly.RepositoryService.WriteRef:input_type -> gitaly.WriteRefRequest
- 31, // 72: gitaly.RepositoryService.FindMergeBase:input_type -> gitaly.FindMergeBaseRequest
- 33, // 73: gitaly.RepositoryService.CreateFork:input_type -> gitaly.CreateForkRequest
- 35, // 74: gitaly.RepositoryService.CreateRepositoryFromURL:input_type -> gitaly.CreateRepositoryFromURLRequest
- 37, // 75: gitaly.RepositoryService.CreateBundle:input_type -> gitaly.CreateBundleRequest
- 39, // 76: gitaly.RepositoryService.CreateBundleFromRefList:input_type -> gitaly.CreateBundleFromRefListRequest
- 15, // 77: gitaly.RepositoryService.FetchBundle:input_type -> gitaly.FetchBundleRequest
- 51, // 78: gitaly.RepositoryService.CreateRepositoryFromBundle:input_type -> gitaly.CreateRepositoryFromBundleRequest
- 41, // 79: gitaly.RepositoryService.GetConfig:input_type -> gitaly.GetConfigRequest
- 53, // 80: gitaly.RepositoryService.FindLicense:input_type -> gitaly.FindLicenseRequest
- 55, // 81: gitaly.RepositoryService.GetInfoAttributes:input_type -> gitaly.GetInfoAttributesRequest
- 57, // 82: gitaly.RepositoryService.CalculateChecksum:input_type -> gitaly.CalculateChecksumRequest
- 59, // 83: gitaly.RepositoryService.GetSnapshot:input_type -> gitaly.GetSnapshotRequest
- 61, // 84: gitaly.RepositoryService.CreateRepositoryFromSnapshot:input_type -> gitaly.CreateRepositoryFromSnapshotRequest
- 63, // 85: gitaly.RepositoryService.GetRawChanges:input_type -> gitaly.GetRawChangesRequest
- 67, // 86: gitaly.RepositoryService.SearchFilesByContent:input_type -> gitaly.SearchFilesByContentRequest
- 65, // 87: gitaly.RepositoryService.SearchFilesByName:input_type -> gitaly.SearchFilesByNameRequest
- 43, // 88: gitaly.RepositoryService.RestoreCustomHooks:input_type -> gitaly.RestoreCustomHooksRequest
- 44, // 89: gitaly.RepositoryService.SetCustomHooks:input_type -> gitaly.SetCustomHooksRequest
- 47, // 90: gitaly.RepositoryService.BackupCustomHooks:input_type -> gitaly.BackupCustomHooksRequest
- 48, // 91: gitaly.RepositoryService.GetCustomHooks:input_type -> gitaly.GetCustomHooksRequest
- 70, // 92: gitaly.RepositoryService.GetObjectDirectorySize:input_type -> gitaly.GetObjectDirectorySizeRequest
- 72, // 93: gitaly.RepositoryService.RemoveRepository:input_type -> gitaly.RemoveRepositoryRequest
- 74, // 94: gitaly.RepositoryService.RenameRepository:input_type -> gitaly.RenameRepositoryRequest
- 76, // 95: gitaly.RepositoryService.ReplicateRepository:input_type -> gitaly.ReplicateRepositoryRequest
- 78, // 96: gitaly.RepositoryService.OptimizeRepository:input_type -> gitaly.OptimizeRepositoryRequest
- 80, // 97: gitaly.RepositoryService.PruneUnreachableObjects:input_type -> gitaly.PruneUnreachableObjectsRequest
- 82, // 98: gitaly.RepositoryService.SetFullPath:input_type -> gitaly.SetFullPathRequest
- 84, // 99: gitaly.RepositoryService.FullPath:input_type -> gitaly.FullPathRequest
- 86, // 100: gitaly.RepositoryService.RemoveAll:input_type -> gitaly.RemoveAllRequest
- 88, // 101: gitaly.RepositoryService.BackupRepository:input_type -> gitaly.BackupRepositoryRequest
- 90, // 102: gitaly.RepositoryService.RestoreRepository:input_type -> gitaly.RestoreRepositoryRequest
- 92, // 103: gitaly.RepositoryService.GetFileAttributes:input_type -> gitaly.GetFileAttributesRequest
- 4, // 104: gitaly.RepositoryService.RepositoryExists:output_type -> gitaly.RepositoryExistsResponse
- 6, // 105: gitaly.RepositoryService.RepositorySize:output_type -> gitaly.RepositorySizeResponse
- 8, // 106: gitaly.RepositoryService.RepositoryInfo:output_type -> gitaly.RepositoryInfoResponse
- 10, // 107: gitaly.RepositoryService.ObjectsSize:output_type -> gitaly.ObjectsSizeResponse
- 12, // 108: gitaly.RepositoryService.ObjectFormat:output_type -> gitaly.ObjectFormatResponse
- 14, // 109: gitaly.RepositoryService.ApplyGitattributes:output_type -> gitaly.ApplyGitattributesResponse
- 18, // 110: gitaly.RepositoryService.FetchRemote:output_type -> gitaly.FetchRemoteResponse
- 20, // 111: gitaly.RepositoryService.CreateRepository:output_type -> gitaly.CreateRepositoryResponse
- 22, // 112: gitaly.RepositoryService.GetArchive:output_type -> gitaly.GetArchiveResponse
- 24, // 113: gitaly.RepositoryService.HasLocalBranches:output_type -> gitaly.HasLocalBranchesResponse
- 26, // 114: gitaly.RepositoryService.FetchSourceBranch:output_type -> gitaly.FetchSourceBranchResponse
- 28, // 115: gitaly.RepositoryService.Fsck:output_type -> gitaly.FsckResponse
- 30, // 116: gitaly.RepositoryService.WriteRef:output_type -> gitaly.WriteRefResponse
- 32, // 117: gitaly.RepositoryService.FindMergeBase:output_type -> gitaly.FindMergeBaseResponse
- 34, // 118: gitaly.RepositoryService.CreateFork:output_type -> gitaly.CreateForkResponse
- 36, // 119: gitaly.RepositoryService.CreateRepositoryFromURL:output_type -> gitaly.CreateRepositoryFromURLResponse
- 38, // 120: gitaly.RepositoryService.CreateBundle:output_type -> gitaly.CreateBundleResponse
- 40, // 121: gitaly.RepositoryService.CreateBundleFromRefList:output_type -> gitaly.CreateBundleFromRefListResponse
- 16, // 122: gitaly.RepositoryService.FetchBundle:output_type -> gitaly.FetchBundleResponse
- 52, // 123: gitaly.RepositoryService.CreateRepositoryFromBundle:output_type -> gitaly.CreateRepositoryFromBundleResponse
- 42, // 124: gitaly.RepositoryService.GetConfig:output_type -> gitaly.GetConfigResponse
- 54, // 125: gitaly.RepositoryService.FindLicense:output_type -> gitaly.FindLicenseResponse
- 56, // 126: gitaly.RepositoryService.GetInfoAttributes:output_type -> gitaly.GetInfoAttributesResponse
- 58, // 127: gitaly.RepositoryService.CalculateChecksum:output_type -> gitaly.CalculateChecksumResponse
- 60, // 128: gitaly.RepositoryService.GetSnapshot:output_type -> gitaly.GetSnapshotResponse
- 62, // 129: gitaly.RepositoryService.CreateRepositoryFromSnapshot:output_type -> gitaly.CreateRepositoryFromSnapshotResponse
- 64, // 130: gitaly.RepositoryService.GetRawChanges:output_type -> gitaly.GetRawChangesResponse
- 68, // 131: gitaly.RepositoryService.SearchFilesByContent:output_type -> gitaly.SearchFilesByContentResponse
- 66, // 132: gitaly.RepositoryService.SearchFilesByName:output_type -> gitaly.SearchFilesByNameResponse
- 45, // 133: gitaly.RepositoryService.RestoreCustomHooks:output_type -> gitaly.RestoreCustomHooksResponse
- 46, // 134: gitaly.RepositoryService.SetCustomHooks:output_type -> gitaly.SetCustomHooksResponse
- 49, // 135: gitaly.RepositoryService.BackupCustomHooks:output_type -> gitaly.BackupCustomHooksResponse
- 50, // 136: gitaly.RepositoryService.GetCustomHooks:output_type -> gitaly.GetCustomHooksResponse
- 71, // 137: gitaly.RepositoryService.GetObjectDirectorySize:output_type -> gitaly.GetObjectDirectorySizeResponse
- 73, // 138: gitaly.RepositoryService.RemoveRepository:output_type -> gitaly.RemoveRepositoryResponse
- 75, // 139: gitaly.RepositoryService.RenameRepository:output_type -> gitaly.RenameRepositoryResponse
- 77, // 140: gitaly.RepositoryService.ReplicateRepository:output_type -> gitaly.ReplicateRepositoryResponse
- 79, // 141: gitaly.RepositoryService.OptimizeRepository:output_type -> gitaly.OptimizeRepositoryResponse
- 81, // 142: gitaly.RepositoryService.PruneUnreachableObjects:output_type -> gitaly.PruneUnreachableObjectsResponse
- 83, // 143: gitaly.RepositoryService.SetFullPath:output_type -> gitaly.SetFullPathResponse
- 85, // 144: gitaly.RepositoryService.FullPath:output_type -> gitaly.FullPathResponse
- 87, // 145: gitaly.RepositoryService.RemoveAll:output_type -> gitaly.RemoveAllResponse
- 89, // 146: gitaly.RepositoryService.BackupRepository:output_type -> gitaly.BackupRepositoryResponse
- 91, // 147: gitaly.RepositoryService.RestoreRepository:output_type -> gitaly.RestoreRepositoryResponse
- 93, // 148: gitaly.RepositoryService.GetFileAttributes:output_type -> gitaly.GetFileAttributesResponse
- 104, // [104:149] is the sub-list for method output_type
- 59, // [59:104] is the sub-list for method input_type
- 59, // [59:59] is the sub-list for extension type_name
- 59, // [59:59] is the sub-list for extension extendee
- 0, // [0:59] is the sub-list for field type_name
+ 98, // 16: gitaly.HasLocalBranchesRequest.repository:type_name -> gitaly.Repository
+ 98, // 17: gitaly.FetchSourceBranchRequest.repository:type_name -> gitaly.Repository
+ 98, // 18: gitaly.FetchSourceBranchRequest.source_repository:type_name -> gitaly.Repository
+ 98, // 19: gitaly.FsckRequest.repository:type_name -> gitaly.Repository
+ 98, // 20: gitaly.WriteRefRequest.repository:type_name -> gitaly.Repository
+ 98, // 21: gitaly.FindMergeBaseRequest.repository:type_name -> gitaly.Repository
+ 98, // 22: gitaly.CreateForkRequest.repository:type_name -> gitaly.Repository
+ 98, // 23: gitaly.CreateForkRequest.source_repository:type_name -> gitaly.Repository
+ 98, // 24: gitaly.CreateRepositoryFromURLRequest.repository:type_name -> gitaly.Repository
+ 98, // 25: gitaly.CreateBundleRequest.repository:type_name -> gitaly.Repository
+ 98, // 26: gitaly.CreateBundleFromRefListRequest.repository:type_name -> gitaly.Repository
+ 98, // 27: gitaly.GetConfigRequest.repository:type_name -> gitaly.Repository
+ 98, // 28: gitaly.RestoreCustomHooksRequest.repository:type_name -> gitaly.Repository
+ 98, // 29: gitaly.SetCustomHooksRequest.repository:type_name -> gitaly.Repository
+ 98, // 30: gitaly.BackupCustomHooksRequest.repository:type_name -> gitaly.Repository
+ 98, // 31: gitaly.GetCustomHooksRequest.repository:type_name -> gitaly.Repository
+ 98, // 32: gitaly.CreateRepositoryFromBundleRequest.repository:type_name -> gitaly.Repository
+ 98, // 33: gitaly.FindLicenseRequest.repository:type_name -> gitaly.Repository
+ 98, // 34: gitaly.GetInfoAttributesRequest.repository:type_name -> gitaly.Repository
+ 98, // 35: gitaly.CalculateChecksumRequest.repository:type_name -> gitaly.Repository
+ 98, // 36: gitaly.GetSnapshotRequest.repository:type_name -> gitaly.Repository
+ 98, // 37: gitaly.CreateRepositoryFromSnapshotRequest.repository:type_name -> gitaly.Repository
+ 98, // 38: gitaly.GetRawChangesRequest.repository:type_name -> gitaly.Repository
+ 94, // 39: gitaly.GetRawChangesResponse.raw_changes:type_name -> gitaly.GetRawChangesResponse.RawChange
+ 98, // 40: gitaly.SearchFilesByNameRequest.repository:type_name -> gitaly.Repository
+ 98, // 41: gitaly.SearchFilesByContentRequest.repository:type_name -> gitaly.Repository
+ 98, // 42: gitaly.GetObjectDirectorySizeRequest.repository:type_name -> gitaly.Repository
+ 98, // 43: gitaly.RemoveRepositoryRequest.repository:type_name -> gitaly.Repository
+ 98, // 44: gitaly.ReplicateRepositoryRequest.repository:type_name -> gitaly.Repository
+ 98, // 45: gitaly.ReplicateRepositoryRequest.source:type_name -> gitaly.Repository
+ 98, // 46: gitaly.OptimizeRepositoryRequest.repository:type_name -> gitaly.Repository
+ 2, // 47: gitaly.OptimizeRepositoryRequest.strategy:type_name -> gitaly.OptimizeRepositoryRequest.Strategy
+ 98, // 48: gitaly.PruneUnreachableObjectsRequest.repository:type_name -> gitaly.Repository
+ 98, // 49: gitaly.SetFullPathRequest.repository:type_name -> gitaly.Repository
+ 98, // 50: gitaly.FullPathRequest.repository:type_name -> gitaly.Repository
+ 98, // 51: gitaly.BackupRepositoryRequest.repository:type_name -> gitaly.Repository
+ 98, // 52: gitaly.BackupRepositoryRequest.vanity_repository:type_name -> gitaly.Repository
+ 98, // 53: gitaly.RestoreRepositoryRequest.repository:type_name -> gitaly.Repository
+ 98, // 54: gitaly.RestoreRepositoryRequest.vanity_repository:type_name -> gitaly.Repository
+ 98, // 55: gitaly.GetFileAttributesRequest.repository:type_name -> gitaly.Repository
+ 97, // 56: gitaly.GetFileAttributesResponse.attribute_infos:type_name -> gitaly.GetFileAttributesResponse.AttributeInfo
+ 1, // 57: gitaly.GetRawChangesResponse.RawChange.operation:type_name -> gitaly.GetRawChangesResponse.RawChange.Operation
+ 3, // 58: gitaly.RepositoryService.RepositoryExists:input_type -> gitaly.RepositoryExistsRequest
+ 5, // 59: gitaly.RepositoryService.RepositorySize:input_type -> gitaly.RepositorySizeRequest
+ 7, // 60: gitaly.RepositoryService.RepositoryInfo:input_type -> gitaly.RepositoryInfoRequest
+ 9, // 61: gitaly.RepositoryService.ObjectsSize:input_type -> gitaly.ObjectsSizeRequest
+ 11, // 62: gitaly.RepositoryService.ObjectFormat:input_type -> gitaly.ObjectFormatRequest
+ 13, // 63: gitaly.RepositoryService.ApplyGitattributes:input_type -> gitaly.ApplyGitattributesRequest
+ 17, // 64: gitaly.RepositoryService.FetchRemote:input_type -> gitaly.FetchRemoteRequest
+ 19, // 65: gitaly.RepositoryService.CreateRepository:input_type -> gitaly.CreateRepositoryRequest
+ 21, // 66: gitaly.RepositoryService.GetArchive:input_type -> gitaly.GetArchiveRequest
+ 23, // 67: gitaly.RepositoryService.HasLocalBranches:input_type -> gitaly.HasLocalBranchesRequest
+ 25, // 68: gitaly.RepositoryService.FetchSourceBranch:input_type -> gitaly.FetchSourceBranchRequest
+ 27, // 69: gitaly.RepositoryService.Fsck:input_type -> gitaly.FsckRequest
+ 29, // 70: gitaly.RepositoryService.WriteRef:input_type -> gitaly.WriteRefRequest
+ 31, // 71: gitaly.RepositoryService.FindMergeBase:input_type -> gitaly.FindMergeBaseRequest
+ 33, // 72: gitaly.RepositoryService.CreateFork:input_type -> gitaly.CreateForkRequest
+ 35, // 73: gitaly.RepositoryService.CreateRepositoryFromURL:input_type -> gitaly.CreateRepositoryFromURLRequest
+ 37, // 74: gitaly.RepositoryService.CreateBundle:input_type -> gitaly.CreateBundleRequest
+ 39, // 75: gitaly.RepositoryService.CreateBundleFromRefList:input_type -> gitaly.CreateBundleFromRefListRequest
+ 15, // 76: gitaly.RepositoryService.FetchBundle:input_type -> gitaly.FetchBundleRequest
+ 51, // 77: gitaly.RepositoryService.CreateRepositoryFromBundle:input_type -> gitaly.CreateRepositoryFromBundleRequest
+ 41, // 78: gitaly.RepositoryService.GetConfig:input_type -> gitaly.GetConfigRequest
+ 53, // 79: gitaly.RepositoryService.FindLicense:input_type -> gitaly.FindLicenseRequest
+ 55, // 80: gitaly.RepositoryService.GetInfoAttributes:input_type -> gitaly.GetInfoAttributesRequest
+ 57, // 81: gitaly.RepositoryService.CalculateChecksum:input_type -> gitaly.CalculateChecksumRequest
+ 59, // 82: gitaly.RepositoryService.GetSnapshot:input_type -> gitaly.GetSnapshotRequest
+ 61, // 83: gitaly.RepositoryService.CreateRepositoryFromSnapshot:input_type -> gitaly.CreateRepositoryFromSnapshotRequest
+ 63, // 84: gitaly.RepositoryService.GetRawChanges:input_type -> gitaly.GetRawChangesRequest
+ 67, // 85: gitaly.RepositoryService.SearchFilesByContent:input_type -> gitaly.SearchFilesByContentRequest
+ 65, // 86: gitaly.RepositoryService.SearchFilesByName:input_type -> gitaly.SearchFilesByNameRequest
+ 43, // 87: gitaly.RepositoryService.RestoreCustomHooks:input_type -> gitaly.RestoreCustomHooksRequest
+ 44, // 88: gitaly.RepositoryService.SetCustomHooks:input_type -> gitaly.SetCustomHooksRequest
+ 47, // 89: gitaly.RepositoryService.BackupCustomHooks:input_type -> gitaly.BackupCustomHooksRequest
+ 48, // 90: gitaly.RepositoryService.GetCustomHooks:input_type -> gitaly.GetCustomHooksRequest
+ 70, // 91: gitaly.RepositoryService.GetObjectDirectorySize:input_type -> gitaly.GetObjectDirectorySizeRequest
+ 72, // 92: gitaly.RepositoryService.RemoveRepository:input_type -> gitaly.RemoveRepositoryRequest
+ 74, // 93: gitaly.RepositoryService.ReplicateRepository:input_type -> gitaly.ReplicateRepositoryRequest
+ 76, // 94: gitaly.RepositoryService.OptimizeRepository:input_type -> gitaly.OptimizeRepositoryRequest
+ 78, // 95: gitaly.RepositoryService.PruneUnreachableObjects:input_type -> gitaly.PruneUnreachableObjectsRequest
+ 80, // 96: gitaly.RepositoryService.SetFullPath:input_type -> gitaly.SetFullPathRequest
+ 82, // 97: gitaly.RepositoryService.FullPath:input_type -> gitaly.FullPathRequest
+ 84, // 98: gitaly.RepositoryService.RemoveAll:input_type -> gitaly.RemoveAllRequest
+ 86, // 99: gitaly.RepositoryService.BackupRepository:input_type -> gitaly.BackupRepositoryRequest
+ 88, // 100: gitaly.RepositoryService.RestoreRepository:input_type -> gitaly.RestoreRepositoryRequest
+ 90, // 101: gitaly.RepositoryService.GetFileAttributes:input_type -> gitaly.GetFileAttributesRequest
+ 4, // 102: gitaly.RepositoryService.RepositoryExists:output_type -> gitaly.RepositoryExistsResponse
+ 6, // 103: gitaly.RepositoryService.RepositorySize:output_type -> gitaly.RepositorySizeResponse
+ 8, // 104: gitaly.RepositoryService.RepositoryInfo:output_type -> gitaly.RepositoryInfoResponse
+ 10, // 105: gitaly.RepositoryService.ObjectsSize:output_type -> gitaly.ObjectsSizeResponse
+ 12, // 106: gitaly.RepositoryService.ObjectFormat:output_type -> gitaly.ObjectFormatResponse
+ 14, // 107: gitaly.RepositoryService.ApplyGitattributes:output_type -> gitaly.ApplyGitattributesResponse
+ 18, // 108: gitaly.RepositoryService.FetchRemote:output_type -> gitaly.FetchRemoteResponse
+ 20, // 109: gitaly.RepositoryService.CreateRepository:output_type -> gitaly.CreateRepositoryResponse
+ 22, // 110: gitaly.RepositoryService.GetArchive:output_type -> gitaly.GetArchiveResponse
+ 24, // 111: gitaly.RepositoryService.HasLocalBranches:output_type -> gitaly.HasLocalBranchesResponse
+ 26, // 112: gitaly.RepositoryService.FetchSourceBranch:output_type -> gitaly.FetchSourceBranchResponse
+ 28, // 113: gitaly.RepositoryService.Fsck:output_type -> gitaly.FsckResponse
+ 30, // 114: gitaly.RepositoryService.WriteRef:output_type -> gitaly.WriteRefResponse
+ 32, // 115: gitaly.RepositoryService.FindMergeBase:output_type -> gitaly.FindMergeBaseResponse
+ 34, // 116: gitaly.RepositoryService.CreateFork:output_type -> gitaly.CreateForkResponse
+ 36, // 117: gitaly.RepositoryService.CreateRepositoryFromURL:output_type -> gitaly.CreateRepositoryFromURLResponse
+ 38, // 118: gitaly.RepositoryService.CreateBundle:output_type -> gitaly.CreateBundleResponse
+ 40, // 119: gitaly.RepositoryService.CreateBundleFromRefList:output_type -> gitaly.CreateBundleFromRefListResponse
+ 16, // 120: gitaly.RepositoryService.FetchBundle:output_type -> gitaly.FetchBundleResponse
+ 52, // 121: gitaly.RepositoryService.CreateRepositoryFromBundle:output_type -> gitaly.CreateRepositoryFromBundleResponse
+ 42, // 122: gitaly.RepositoryService.GetConfig:output_type -> gitaly.GetConfigResponse
+ 54, // 123: gitaly.RepositoryService.FindLicense:output_type -> gitaly.FindLicenseResponse
+ 56, // 124: gitaly.RepositoryService.GetInfoAttributes:output_type -> gitaly.GetInfoAttributesResponse
+ 58, // 125: gitaly.RepositoryService.CalculateChecksum:output_type -> gitaly.CalculateChecksumResponse
+ 60, // 126: gitaly.RepositoryService.GetSnapshot:output_type -> gitaly.GetSnapshotResponse
+ 62, // 127: gitaly.RepositoryService.CreateRepositoryFromSnapshot:output_type -> gitaly.CreateRepositoryFromSnapshotResponse
+ 64, // 128: gitaly.RepositoryService.GetRawChanges:output_type -> gitaly.GetRawChangesResponse
+ 68, // 129: gitaly.RepositoryService.SearchFilesByContent:output_type -> gitaly.SearchFilesByContentResponse
+ 66, // 130: gitaly.RepositoryService.SearchFilesByName:output_type -> gitaly.SearchFilesByNameResponse
+ 45, // 131: gitaly.RepositoryService.RestoreCustomHooks:output_type -> gitaly.RestoreCustomHooksResponse
+ 46, // 132: gitaly.RepositoryService.SetCustomHooks:output_type -> gitaly.SetCustomHooksResponse
+ 49, // 133: gitaly.RepositoryService.BackupCustomHooks:output_type -> gitaly.BackupCustomHooksResponse
+ 50, // 134: gitaly.RepositoryService.GetCustomHooks:output_type -> gitaly.GetCustomHooksResponse
+ 71, // 135: gitaly.RepositoryService.GetObjectDirectorySize:output_type -> gitaly.GetObjectDirectorySizeResponse
+ 73, // 136: gitaly.RepositoryService.RemoveRepository:output_type -> gitaly.RemoveRepositoryResponse
+ 75, // 137: gitaly.RepositoryService.ReplicateRepository:output_type -> gitaly.ReplicateRepositoryResponse
+ 77, // 138: gitaly.RepositoryService.OptimizeRepository:output_type -> gitaly.OptimizeRepositoryResponse
+ 79, // 139: gitaly.RepositoryService.PruneUnreachableObjects:output_type -> gitaly.PruneUnreachableObjectsResponse
+ 81, // 140: gitaly.RepositoryService.SetFullPath:output_type -> gitaly.SetFullPathResponse
+ 83, // 141: gitaly.RepositoryService.FullPath:output_type -> gitaly.FullPathResponse
+ 85, // 142: gitaly.RepositoryService.RemoveAll:output_type -> gitaly.RemoveAllResponse
+ 87, // 143: gitaly.RepositoryService.BackupRepository:output_type -> gitaly.BackupRepositoryResponse
+ 89, // 144: gitaly.RepositoryService.RestoreRepository:output_type -> gitaly.RestoreRepositoryResponse
+ 91, // 145: gitaly.RepositoryService.GetFileAttributes:output_type -> gitaly.GetFileAttributesResponse
+ 102, // [102:146] is the sub-list for method output_type
+ 58, // [58:102] is the sub-list for method input_type
+ 58, // [58:58] is the sub-list for extension type_name
+ 58, // [58:58] is the sub-list for extension extendee
+ 0, // [0:58] is the sub-list for field type_name
}
func init() { file_repository_proto_init() }
@@ -7695,30 +7578,6 @@ func file_repository_proto_init() {
}
}
file_repository_proto_msgTypes[71].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*RenameRepositoryRequest); i {
- case 0:
- return &v.state
- case 1:
- return &v.sizeCache
- case 2:
- return &v.unknownFields
- default:
- return nil
- }
- }
- file_repository_proto_msgTypes[72].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*RenameRepositoryResponse); i {
- case 0:
- return &v.state
- case 1:
- return &v.sizeCache
- case 2:
- return &v.unknownFields
- default:
- return nil
- }
- }
- file_repository_proto_msgTypes[73].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*ReplicateRepositoryRequest); i {
case 0:
return &v.state
@@ -7730,7 +7589,7 @@ func file_repository_proto_init() {
return nil
}
}
- file_repository_proto_msgTypes[74].Exporter = func(v interface{}, i int) interface{} {
+ file_repository_proto_msgTypes[72].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*ReplicateRepositoryResponse); i {
case 0:
return &v.state
@@ -7742,7 +7601,7 @@ func file_repository_proto_init() {
return nil
}
}
- file_repository_proto_msgTypes[75].Exporter = func(v interface{}, i int) interface{} {
+ file_repository_proto_msgTypes[73].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*OptimizeRepositoryRequest); i {
case 0:
return &v.state
@@ -7754,7 +7613,7 @@ func file_repository_proto_init() {
return nil
}
}
- file_repository_proto_msgTypes[76].Exporter = func(v interface{}, i int) interface{} {
+ file_repository_proto_msgTypes[74].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*OptimizeRepositoryResponse); i {
case 0:
return &v.state
@@ -7766,7 +7625,7 @@ func file_repository_proto_init() {
return nil
}
}
- file_repository_proto_msgTypes[77].Exporter = func(v interface{}, i int) interface{} {
+ file_repository_proto_msgTypes[75].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*PruneUnreachableObjectsRequest); i {
case 0:
return &v.state
@@ -7778,7 +7637,7 @@ func file_repository_proto_init() {
return nil
}
}
- file_repository_proto_msgTypes[78].Exporter = func(v interface{}, i int) interface{} {
+ file_repository_proto_msgTypes[76].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*PruneUnreachableObjectsResponse); i {
case 0:
return &v.state
@@ -7790,7 +7649,7 @@ func file_repository_proto_init() {
return nil
}
}
- file_repository_proto_msgTypes[79].Exporter = func(v interface{}, i int) interface{} {
+ file_repository_proto_msgTypes[77].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*SetFullPathRequest); i {
case 0:
return &v.state
@@ -7802,7 +7661,7 @@ func file_repository_proto_init() {
return nil
}
}
- file_repository_proto_msgTypes[80].Exporter = func(v interface{}, i int) interface{} {
+ file_repository_proto_msgTypes[78].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*SetFullPathResponse); i {
case 0:
return &v.state
@@ -7814,7 +7673,7 @@ func file_repository_proto_init() {
return nil
}
}
- file_repository_proto_msgTypes[81].Exporter = func(v interface{}, i int) interface{} {
+ file_repository_proto_msgTypes[79].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*FullPathRequest); i {
case 0:
return &v.state
@@ -7826,7 +7685,7 @@ func file_repository_proto_init() {
return nil
}
}
- file_repository_proto_msgTypes[82].Exporter = func(v interface{}, i int) interface{} {
+ file_repository_proto_msgTypes[80].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*FullPathResponse); i {
case 0:
return &v.state
@@ -7838,7 +7697,7 @@ func file_repository_proto_init() {
return nil
}
}
- file_repository_proto_msgTypes[83].Exporter = func(v interface{}, i int) interface{} {
+ file_repository_proto_msgTypes[81].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*RemoveAllRequest); i {
case 0:
return &v.state
@@ -7850,7 +7709,7 @@ func file_repository_proto_init() {
return nil
}
}
- file_repository_proto_msgTypes[84].Exporter = func(v interface{}, i int) interface{} {
+ file_repository_proto_msgTypes[82].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*RemoveAllResponse); i {
case 0:
return &v.state
@@ -7862,7 +7721,7 @@ func file_repository_proto_init() {
return nil
}
}
- file_repository_proto_msgTypes[85].Exporter = func(v interface{}, i int) interface{} {
+ file_repository_proto_msgTypes[83].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*BackupRepositoryRequest); i {
case 0:
return &v.state
@@ -7874,7 +7733,7 @@ func file_repository_proto_init() {
return nil
}
}
- file_repository_proto_msgTypes[86].Exporter = func(v interface{}, i int) interface{} {
+ file_repository_proto_msgTypes[84].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*BackupRepositoryResponse); i {
case 0:
return &v.state
@@ -7886,7 +7745,7 @@ func file_repository_proto_init() {
return nil
}
}
- file_repository_proto_msgTypes[87].Exporter = func(v interface{}, i int) interface{} {
+ file_repository_proto_msgTypes[85].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*RestoreRepositoryRequest); i {
case 0:
return &v.state
@@ -7898,7 +7757,7 @@ func file_repository_proto_init() {
return nil
}
}
- file_repository_proto_msgTypes[88].Exporter = func(v interface{}, i int) interface{} {
+ file_repository_proto_msgTypes[86].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*RestoreRepositoryResponse); i {
case 0:
return &v.state
@@ -7910,7 +7769,7 @@ func file_repository_proto_init() {
return nil
}
}
- file_repository_proto_msgTypes[89].Exporter = func(v interface{}, i int) interface{} {
+ file_repository_proto_msgTypes[87].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*GetFileAttributesRequest); i {
case 0:
return &v.state
@@ -7922,7 +7781,7 @@ func file_repository_proto_init() {
return nil
}
}
- file_repository_proto_msgTypes[90].Exporter = func(v interface{}, i int) interface{} {
+ file_repository_proto_msgTypes[88].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*GetFileAttributesResponse); i {
case 0:
return &v.state
@@ -7934,7 +7793,7 @@ func file_repository_proto_init() {
return nil
}
}
- file_repository_proto_msgTypes[91].Exporter = func(v interface{}, i int) interface{} {
+ file_repository_proto_msgTypes[89].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*RepositoryInfoResponse_ReferencesInfo); i {
case 0:
return &v.state
@@ -7946,7 +7805,7 @@ func file_repository_proto_init() {
return nil
}
}
- file_repository_proto_msgTypes[92].Exporter = func(v interface{}, i int) interface{} {
+ file_repository_proto_msgTypes[90].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*RepositoryInfoResponse_ObjectsInfo); i {
case 0:
return &v.state
@@ -7958,7 +7817,7 @@ func file_repository_proto_init() {
return nil
}
}
- file_repository_proto_msgTypes[93].Exporter = func(v interface{}, i int) interface{} {
+ file_repository_proto_msgTypes[91].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*GetRawChangesResponse_RawChange); i {
case 0:
return &v.state
@@ -7970,7 +7829,7 @@ func file_repository_proto_init() {
return nil
}
}
- file_repository_proto_msgTypes[94].Exporter = func(v interface{}, i int) interface{} {
+ file_repository_proto_msgTypes[92].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*BackupRepositoryResponse_SkippedError); i {
case 0:
return &v.state
@@ -7982,7 +7841,7 @@ func file_repository_proto_init() {
return nil
}
}
- file_repository_proto_msgTypes[95].Exporter = func(v interface{}, i int) interface{} {
+ file_repository_proto_msgTypes[93].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*RestoreRepositoryResponse_SkippedError); i {
case 0:
return &v.state
@@ -7994,7 +7853,7 @@ func file_repository_proto_init() {
return nil
}
}
- file_repository_proto_msgTypes[96].Exporter = func(v interface{}, i int) interface{} {
+ file_repository_proto_msgTypes[94].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*GetFileAttributesResponse_AttributeInfo); i {
case 0:
return &v.state
@@ -8013,7 +7872,7 @@ func file_repository_proto_init() {
GoPackagePath: reflect.TypeOf(x{}).PkgPath(),
RawDescriptor: file_repository_proto_rawDesc,
NumEnums: 3,
- NumMessages: 97,
+ NumMessages: 95,
NumExtensions: 0,
NumServices: 1,
},
diff --git a/proto/go/gitalypb/repository_grpc.pb.go b/proto/go/gitalypb/repository_grpc.pb.go
index 4987ffa68..b46e4ab3a 100644
--- a/proto/go/gitalypb/repository_grpc.pb.go
+++ b/proto/go/gitalypb/repository_grpc.pb.go
@@ -146,8 +146,6 @@ type RepositoryServiceClient interface {
// eventually remove it. This ensures that even on networked filesystems the
// data is actually removed even if there's someone still handling the data.
RemoveRepository(ctx context.Context, in *RemoveRepositoryRequest, opts ...grpc.CallOption) (*RemoveRepositoryResponse, error)
- // RenameRepository ...
- RenameRepository(ctx context.Context, in *RenameRepositoryRequest, opts ...grpc.CallOption) (*RenameRepositoryResponse, error)
// ReplicateRepository replicates data from a source repository to target repository. On the
// target repository, this operation ensures synchronization of the following components:
//
@@ -904,15 +902,6 @@ func (c *repositoryServiceClient) RemoveRepository(ctx context.Context, in *Remo
return out, nil
}
-func (c *repositoryServiceClient) RenameRepository(ctx context.Context, in *RenameRepositoryRequest, opts ...grpc.CallOption) (*RenameRepositoryResponse, error) {
- out := new(RenameRepositoryResponse)
- err := c.cc.Invoke(ctx, "/gitaly.RepositoryService/RenameRepository", in, out, opts...)
- if err != nil {
- return nil, err
- }
- return out, nil
-}
-
func (c *repositoryServiceClient) ReplicateRepository(ctx context.Context, in *ReplicateRepositoryRequest, opts ...grpc.CallOption) (*ReplicateRepositoryResponse, error) {
out := new(ReplicateRepositoryResponse)
err := c.cc.Invoke(ctx, "/gitaly.RepositoryService/ReplicateRepository", in, out, opts...)
@@ -1124,8 +1113,6 @@ type RepositoryServiceServer interface {
// eventually remove it. This ensures that even on networked filesystems the
// data is actually removed even if there's someone still handling the data.
RemoveRepository(context.Context, *RemoveRepositoryRequest) (*RemoveRepositoryResponse, error)
- // RenameRepository ...
- RenameRepository(context.Context, *RenameRepositoryRequest) (*RenameRepositoryResponse, error)
// ReplicateRepository replicates data from a source repository to target repository. On the
// target repository, this operation ensures synchronization of the following components:
//
@@ -1288,9 +1275,6 @@ func (UnimplementedRepositoryServiceServer) GetObjectDirectorySize(context.Conte
func (UnimplementedRepositoryServiceServer) RemoveRepository(context.Context, *RemoveRepositoryRequest) (*RemoveRepositoryResponse, error) {
return nil, status.Errorf(codes.Unimplemented, "method RemoveRepository not implemented")
}
-func (UnimplementedRepositoryServiceServer) RenameRepository(context.Context, *RenameRepositoryRequest) (*RenameRepositoryResponse, error) {
- return nil, status.Errorf(codes.Unimplemented, "method RenameRepository not implemented")
-}
func (UnimplementedRepositoryServiceServer) ReplicateRepository(context.Context, *ReplicateRepositoryRequest) (*ReplicateRepositoryResponse, error) {
return nil, status.Errorf(codes.Unimplemented, "method ReplicateRepository not implemented")
}
@@ -2039,24 +2023,6 @@ func _RepositoryService_RemoveRepository_Handler(srv interface{}, ctx context.Co
return interceptor(ctx, in, info, handler)
}
-func _RepositoryService_RenameRepository_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
- in := new(RenameRepositoryRequest)
- if err := dec(in); err != nil {
- return nil, err
- }
- if interceptor == nil {
- return srv.(RepositoryServiceServer).RenameRepository(ctx, in)
- }
- info := &grpc.UnaryServerInfo{
- Server: srv,
- FullMethod: "/gitaly.RepositoryService/RenameRepository",
- }
- handler := func(ctx context.Context, req interface{}) (interface{}, error) {
- return srv.(RepositoryServiceServer).RenameRepository(ctx, req.(*RenameRepositoryRequest))
- }
- return interceptor(ctx, in, info, handler)
-}
-
func _RepositoryService_ReplicateRepository_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
in := new(ReplicateRepositoryRequest)
if err := dec(in); err != nil {
@@ -2303,10 +2269,6 @@ var RepositoryService_ServiceDesc = grpc.ServiceDesc{
Handler: _RepositoryService_RemoveRepository_Handler,
},
{
- MethodName: "RenameRepository",
- Handler: _RepositoryService_RenameRepository_Handler,
- },
- {
MethodName: "ReplicateRepository",
Handler: _RepositoryService_ReplicateRepository_Handler,
},
diff --git a/proto/repository.proto b/proto/repository.proto
index ebd5fcb07..27c5ce666 100644
--- a/proto/repository.proto
+++ b/proto/repository.proto
@@ -309,13 +309,6 @@ service RepositoryService {
};
}
- // RenameRepository ...
- rpc RenameRepository(RenameRepositoryRequest) returns (RenameRepositoryResponse) {
- option (op_type) = {
- op: MUTATOR
- };
- }
-
// ReplicateRepository replicates data from a source repository to target repository. On the
// target repository, this operation ensures synchronization of the following components:
//
@@ -1168,18 +1161,6 @@ message RemoveRepositoryRequest {
message RemoveRepositoryResponse {
}
-// RenameRepositoryRequest ...
-message RenameRepositoryRequest {
- // repository ...
- Repository repository = 1 [(target_repository)=true];
- // relative_path ...
- string relative_path = 2;
-}
-
-// RenameRepositoryResponse ...
-message RenameRepositoryResponse{
-}
-
// ReplicateRepositoryRequest is a request for the ReplicateRepository RPC.
message ReplicateRepositoryRequest {
// repository is the target repository that the RPC replicates to.