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:
authorEric Ju <eju@gitlab.com>2023-11-27 19:46:05 +0300
committerEric Ju <eju@gitlab.com>2023-11-27 19:46:05 +0300
commitcd5a3412db6fdbaaad4ac135968b27316289f46e (patch)
tree5e081b263110ae29cca960100c6403a8520ea51d
parentb0c19f6861fe387e6a61e613bf097b088cab5329 (diff)
repository: Remove ApplyGitattributesej-issue-5347_remove-ApplyGitattributes
As we are using `.gitattributes` files from the root tree instead of writing into info/gitattributes, ApplyGitAttributes GRPC is no longer needed. We remove it from the code base in this commit.
-rw-r--r--internal/gitaly/service/repository/apply_gitattributes.go156
-rw-r--r--internal/gitaly/service/repository/apply_gitattributes_test.go327
-rw-r--r--internal/gitaly/service/repository/replicate_test.go12
-rw-r--r--internal/gitaly/storage/storagemgr/middleware.go3
-rw-r--r--internal/grpc/protoregistry/registry_test.go1
-rw-r--r--internal/praefect/coordinator.go1
-rw-r--r--proto/go/gitalypb/repository.pb.go2727
-rw-r--r--proto/go/gitalypb/repository_grpc.pb.go43
-rw-r--r--proto/repository.proto21
9 files changed, 1305 insertions, 1986 deletions
diff --git a/internal/gitaly/service/repository/apply_gitattributes.go b/internal/gitaly/service/repository/apply_gitattributes.go
deleted file mode 100644
index f9dffccbf..000000000
--- a/internal/gitaly/service/repository/apply_gitattributes.go
+++ /dev/null
@@ -1,156 +0,0 @@
-package repository
-
-import (
- "context"
- "errors"
- "fmt"
- "io"
- "os"
- "path/filepath"
-
- "gitlab.com/gitlab-org/gitaly/v16/internal/git"
- "gitlab.com/gitlab-org/gitaly/v16/internal/git/catfile"
- "gitlab.com/gitlab-org/gitaly/v16/internal/git/localrepo"
- "gitlab.com/gitlab-org/gitaly/v16/internal/gitaly/transaction"
- "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/internal/transaction/txinfo"
- "gitlab.com/gitlab-org/gitaly/v16/internal/transaction/voting"
- "gitlab.com/gitlab-org/gitaly/v16/proto/go/gitalypb"
-)
-
-const attributesFileMode os.FileMode = perm.SharedFile
-
-func (s *server) applyGitattributes(ctx context.Context, repo *localrepo.Repo, objectReader catfile.ObjectContentReader, repoPath string, revision []byte) (returnedErr error) {
- infoPath := filepath.Join(repoPath, "info")
- attributesPath := filepath.Join(infoPath, "attributes")
-
- _, err := repo.ResolveRevision(ctx, git.Revision(revision)+"^{commit}")
- if err != nil {
- if errors.Is(err, git.ErrReferenceNotFound) {
- return structerr.NewInvalidArgument("revision does not exist")
- }
-
- return err
- }
-
- blobObj, err := objectReader.Object(ctx, git.Revision(fmt.Sprintf("%s:.gitattributes", revision)))
- if err != nil && !errors.As(err, &catfile.NotFoundError{}) {
- return err
- }
-
- // Create /info folder if it doesn't exist
- if err := os.MkdirAll(infoPath, perm.SharedDir); err != nil {
- return err
- }
-
- if errors.As(err, &catfile.NotFoundError{}) || blobObj.Type != "blob" {
- locker, err := safe.NewLockingFileWriter(attributesPath, safe.LockingFileWriterConfig{
- FileWriterConfig: safe.FileWriterConfig{FileMode: attributesFileMode},
- })
- if err != nil {
- return fmt.Errorf("creating gitattributes lock: %w", err)
- }
- defer func() {
- if err := locker.Close(); err != nil {
- s.logger.WithError(err).ErrorContext(ctx, "unlocking gitattributes")
- }
- }()
-
- if err := locker.Lock(); err != nil {
- return fmt.Errorf("locking gitattributes: %w", err)
- }
-
- // We use the zero OID as placeholder to vote on removal of the
- // gitattributes file.
- if err := s.vote(ctx, git.ObjectHashSHA1.ZeroOID, voting.Prepared); err != nil {
- return fmt.Errorf("preimage vote: %w", err)
- }
-
- if err := os.Remove(attributesPath); err != nil && !os.IsNotExist(err) {
- return err
- }
-
- if err := s.vote(ctx, git.ObjectHashSHA1.ZeroOID, voting.Committed); err != nil {
- return fmt.Errorf("postimage vote: %w", err)
- }
-
- return nil
- }
-
- writer, err := safe.NewLockingFileWriter(attributesPath, safe.LockingFileWriterConfig{
- FileWriterConfig: safe.FileWriterConfig{FileMode: attributesFileMode},
- })
- if err != nil {
- return fmt.Errorf("creating gitattributes writer: %w", err)
- }
- defer func() {
- if err := writer.Close(); err != nil && returnedErr == nil {
- if !errors.Is(err, safe.ErrAlreadyDone) {
- returnedErr = err
- }
- }
- }()
-
- if _, err := io.Copy(writer, blobObj); err != nil {
- return err
- }
-
- if err := transaction.CommitLockedFile(ctx, s.txManager, writer); err != nil {
- return fmt.Errorf("committing gitattributes: %w", err)
- }
-
- return nil
-}
-
-func (s *server) vote(ctx context.Context, oid git.ObjectID, phase voting.Phase) error {
- tx, err := txinfo.TransactionFromContext(ctx)
- if errors.Is(err, txinfo.ErrTransactionNotFound) {
- return nil
- }
-
- hash, err := oid.Bytes()
- if err != nil {
- return fmt.Errorf("vote with invalid object ID: %w", err)
- }
-
- vote, err := voting.VoteFromHash(hash)
- if err != nil {
- return fmt.Errorf("cannot convert OID to vote: %w", err)
- }
-
- if err := s.txManager.Vote(ctx, tx, vote, phase); err != nil {
- return fmt.Errorf("vote failed: %w", err)
- }
-
- return nil
-}
-
-func (s *server) ApplyGitattributes(ctx context.Context, in *gitalypb.ApplyGitattributesRequest) (*gitalypb.ApplyGitattributesResponse, error) {
- repository := in.GetRepository()
- if err := s.locator.ValidateRepository(repository); err != nil {
- return nil, structerr.NewInvalidArgument("%w", err)
- }
- repo := s.localrepo(repository)
- repoPath, err := s.locator.GetRepoPath(repo)
- if err != nil {
- return nil, err
- }
-
- if err := git.ValidateRevision(in.GetRevision()); err != nil {
- return nil, structerr.NewInvalidArgument("revision: %w", err)
- }
-
- objectReader, cancel, err := s.catfileCache.ObjectReader(ctx, repo)
- if err != nil {
- return nil, err
- }
- defer cancel()
-
- if err := s.applyGitattributes(ctx, repo, objectReader, repoPath, in.GetRevision()); err != nil {
- return nil, err
- }
-
- return &gitalypb.ApplyGitattributesResponse{}, nil
-}
diff --git a/internal/gitaly/service/repository/apply_gitattributes_test.go b/internal/gitaly/service/repository/apply_gitattributes_test.go
deleted file mode 100644
index c5b741bfd..000000000
--- a/internal/gitaly/service/repository/apply_gitattributes_test.go
+++ /dev/null
@@ -1,327 +0,0 @@
-package repository
-
-import (
- "bytes"
- "context"
- "os"
- "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/storage"
- "gitlab.com/gitlab-org/gitaly/v16/internal/grpc/backchannel"
- "gitlab.com/gitlab-org/gitaly/v16/internal/grpc/metadata"
- "gitlab.com/gitlab-org/gitaly/v16/internal/helper/perm"
- "gitlab.com/gitlab-org/gitaly/v16/internal/structerr"
- "gitlab.com/gitlab-org/gitaly/v16/internal/testhelper"
- "gitlab.com/gitlab-org/gitaly/v16/internal/testhelper/testcfg"
- "gitlab.com/gitlab-org/gitaly/v16/internal/transaction/txinfo"
- "gitlab.com/gitlab-org/gitaly/v16/internal/transaction/voting"
- "gitlab.com/gitlab-org/gitaly/v16/proto/go/gitalypb"
- "google.golang.org/grpc"
-)
-
-func TestApplyGitattributes_successful(t *testing.T) {
- t.Parallel()
-
- ctx := testhelper.Context(t)
- cfg, client := setupRepositoryService(t)
-
- repo, repoPath := gittest.CreateRepository(t, ctx, cfg)
- gitattributesContent := "pattern attr=value"
- commitWithGitattributes := gittest.WriteCommit(t, cfg, repoPath, gittest.WithTreeEntries(
- gittest.TreeEntry{Path: ".gitattributes", Mode: "100644", Content: gitattributesContent},
- ))
- commitWithoutGitattributes := gittest.WriteCommit(t, cfg, repoPath)
-
- infoPath := filepath.Join(repoPath, "info")
- attributesPath := filepath.Join(infoPath, "attributes")
-
- for _, tc := range []struct {
- desc string
- revision []byte
- expectedContent []byte
- }{
- {
- desc: "With a .gitattributes file",
- revision: []byte(commitWithGitattributes),
- expectedContent: []byte(gitattributesContent),
- },
- {
- desc: "Without a .gitattributes file",
- revision: []byte(commitWithoutGitattributes),
- expectedContent: nil,
- },
- } {
- t.Run(tc.desc, func(t *testing.T) {
- t.Run("without 'info' directory", func(t *testing.T) {
- require.NoError(t, os.RemoveAll(infoPath))
- requireApplyGitattributes(t, ctx, client, repo, attributesPath, tc.revision, tc.expectedContent)
- })
-
- t.Run("without 'info/attributes' directory", func(t *testing.T) {
- require.NoError(t, os.RemoveAll(infoPath))
- require.NoError(t, os.Mkdir(infoPath, perm.SharedDir))
- requireApplyGitattributes(t, ctx, client, repo, attributesPath, tc.revision, tc.expectedContent)
- })
-
- t.Run("with preexisting 'info/attributes'", func(t *testing.T) {
- require.NoError(t, os.RemoveAll(infoPath))
- require.NoError(t, os.Mkdir(infoPath, perm.SharedDir))
- require.NoError(t, os.WriteFile(attributesPath, []byte("*.docx diff=word"), perm.SharedFile))
- requireApplyGitattributes(t, ctx, client, repo, attributesPath, tc.revision, tc.expectedContent)
- })
- })
- }
-}
-
-type testTransactionServer struct {
- gitalypb.UnimplementedRefTransactionServer
- vote func(*gitalypb.VoteTransactionRequest) (*gitalypb.VoteTransactionResponse, error)
-}
-
-func (s *testTransactionServer) VoteTransaction(ctx context.Context, in *gitalypb.VoteTransactionRequest) (*gitalypb.VoteTransactionResponse, error) {
- if s.vote != nil {
- return s.vote(in)
- }
- return nil, nil
-}
-
-func TestApplyGitattributes_transactional(t *testing.T) {
- t.Parallel()
-
- ctx := testhelper.Context(t)
- cfg := testcfg.Build(t)
-
- repo, repoPath := gittest.CreateRepository(t, ctx, cfg, gittest.CreateRepositoryConfig{
- SkipCreationViaService: true,
- })
- gitattributesContent := "pattern attr=value"
- commitWithGitattributes := gittest.WriteCommit(t, cfg, repoPath, gittest.WithTreeEntries(
- gittest.TreeEntry{Path: ".gitattributes", Mode: "100644", Content: gitattributesContent},
- ))
- commitWithoutGitattributes := gittest.WriteCommit(t, cfg, repoPath)
-
- transactionServer := &testTransactionServer{}
- runRepositoryService(t, cfg)
-
- // We're using internal listener in order to route around
- // Praefect in our tests. Otherwise Praefect would replace our
- // carefully crafted transaction and server information.
- logger := testhelper.SharedLogger(t)
-
- client := newMuxedRepositoryClient(t, ctx, cfg, "unix://"+cfg.InternalSocketPath(),
- backchannel.NewClientHandshaker(
- logger,
- func() backchannel.Server {
- srv := grpc.NewServer()
- gitalypb.RegisterRefTransactionServer(srv, transactionServer)
- return srv
- },
- backchannel.DefaultConfiguration(),
- ),
- )
-
- for _, tc := range []struct {
- desc string
- revision []byte
- voteFn func(*testing.T, *gitalypb.VoteTransactionRequest) (*gitalypb.VoteTransactionResponse, error)
- shouldExist bool
- expectedErr error
- expectedVotes int
- }{
- {
- desc: "successful vote writes gitattributes",
- revision: []byte(commitWithGitattributes),
- voteFn: func(t *testing.T, request *gitalypb.VoteTransactionRequest) (*gitalypb.VoteTransactionResponse, error) {
- vote := voting.VoteFromData([]byte(gitattributesContent))
- expectedHash := vote.Bytes()
-
- require.Equal(t, expectedHash, request.ReferenceUpdatesHash)
- return &gitalypb.VoteTransactionResponse{
- State: gitalypb.VoteTransactionResponse_COMMIT,
- }, nil
- },
- shouldExist: true,
- expectedVotes: 2,
- },
- {
- desc: "aborted vote does not write gitattributes",
- revision: []byte(commitWithGitattributes),
- voteFn: func(t *testing.T, request *gitalypb.VoteTransactionRequest) (*gitalypb.VoteTransactionResponse, error) {
- return &gitalypb.VoteTransactionResponse{
- State: gitalypb.VoteTransactionResponse_ABORT,
- }, nil
- },
- shouldExist: false,
- expectedErr: func() error {
- return structerr.NewInternal("committing gitattributes: voting on locked file: preimage vote: transaction was aborted")
- }(),
- expectedVotes: 1,
- },
- {
- desc: "failing vote does not write gitattributes",
- revision: []byte(commitWithGitattributes),
- voteFn: func(t *testing.T, request *gitalypb.VoteTransactionRequest) (*gitalypb.VoteTransactionResponse, error) {
- return nil, structerr.NewFailedPrecondition("foobar")
- },
- shouldExist: false,
- expectedErr: func() error {
- return structerr.NewFailedPrecondition("committing gitattributes: voting on locked file: preimage vote: rpc error: code = FailedPrecondition desc = foobar")
- }(),
- expectedVotes: 1,
- },
- {
- desc: "commit without gitattributes performs vote",
- revision: []byte(commitWithoutGitattributes),
- voteFn: func(t *testing.T, request *gitalypb.VoteTransactionRequest) (*gitalypb.VoteTransactionResponse, error) {
- require.Equal(t, bytes.Repeat([]byte{0x00}, 20), request.ReferenceUpdatesHash)
- return &gitalypb.VoteTransactionResponse{
- State: gitalypb.VoteTransactionResponse_COMMIT,
- }, nil
- },
- shouldExist: false,
- expectedVotes: 2,
- },
- } {
- t.Run(tc.desc, func(t *testing.T) {
- infoPath := filepath.Join(repoPath, "info")
- require.NoError(t, os.RemoveAll(infoPath))
-
- ctx, err := txinfo.InjectTransaction(ctx, 1, "primary", true)
- require.NoError(t, err)
- ctx = metadata.IncomingToOutgoing(ctx)
-
- var votes int
- transactionServer.vote = func(request *gitalypb.VoteTransactionRequest) (*gitalypb.VoteTransactionResponse, error) {
- votes++
- return tc.voteFn(t, request)
- }
-
- //nolint:staticcheck
- _, err = client.ApplyGitattributes(ctx, &gitalypb.ApplyGitattributesRequest{
- Repository: repo,
- Revision: tc.revision,
- })
- testhelper.RequireGrpcError(t, tc.expectedErr, err)
-
- path := filepath.Join(infoPath, "attributes")
- if tc.shouldExist {
- content := testhelper.MustReadFile(t, path)
- require.Equal(t, []byte(gitattributesContent), content)
- } else {
- require.NoFileExists(t, path)
- }
- require.Equal(t, tc.expectedVotes, votes)
- })
- }
-}
-
-func TestApplyGitattributes_failure(t *testing.T) {
- t.Parallel()
-
- ctx := testhelper.Context(t)
- cfg, client := setupRepositoryService(t)
- repo, _ := gittest.CreateRepository(t, ctx, cfg)
-
- for _, tc := range []struct {
- desc string
- repo *gitalypb.Repository
- revision []byte
- expectedErr error
- }{
- {
- desc: "no repository provided",
- repo: nil,
- revision: nil,
- expectedErr: structerr.NewInvalidArgument("%w", storage.ErrRepositoryNotSet),
- },
- {
- desc: "unknown storage provided",
- repo: &gitalypb.Repository{
- RelativePath: "stub",
- StorageName: "foo",
- },
- revision: []byte("master"),
- expectedErr: testhelper.ToInterceptedMetadata(structerr.NewInvalidArgument(
- "%w", storage.NewStorageNotFoundError("foo"),
- )),
- },
- {
- desc: "storage not provided",
- repo: &gitalypb.Repository{
- RelativePath: repo.GetRelativePath(),
- },
- revision: []byte("master"),
- expectedErr: structerr.NewInvalidArgument("%w", storage.ErrStorageNotSet),
- },
- {
- desc: "repository doesn't exist on disk",
- repo: &gitalypb.Repository{
- StorageName: repo.GetStorageName(),
- RelativePath: "bar",
- },
- revision: []byte("master"),
- expectedErr: testhelper.ToInterceptedMetadata(
- structerr.New("%w", storage.NewRepositoryNotFoundError(cfg.Storages[0].Name, "bar")),
- ),
- },
- {
- desc: "no revision provided",
- repo: repo,
- revision: []byte(""),
- expectedErr: structerr.NewInvalidArgument("revision: empty revision"),
- },
- {
- desc: "unknown revision",
- repo: repo,
- revision: []byte("not-existing-ref"),
- expectedErr: structerr.NewInvalidArgument("revision does not exist"),
- },
- {
- desc: "invalid revision",
- repo: repo,
- revision: []byte("--output=/meow"),
- expectedErr: structerr.NewInvalidArgument("revision: revision can't start with '-'"),
- },
- } {
- t.Run(tc.desc, func(t *testing.T) {
- //nolint:staticcheck
- _, err := client.ApplyGitattributes(ctx, &gitalypb.ApplyGitattributesRequest{
- Repository: tc.repo,
- Revision: tc.revision,
- })
- testhelper.RequireGrpcError(t, tc.expectedErr, err)
- })
- }
-}
-
-func requireApplyGitattributes(
- t *testing.T,
- ctx context.Context,
- client gitalypb.RepositoryServiceClient,
- repo *gitalypb.Repository,
- attributesPath string,
- revision, expectedContent []byte,
-) {
- t.Helper()
-
- //nolint:staticcheck
- response, err := client.ApplyGitattributes(ctx, &gitalypb.ApplyGitattributesRequest{
- Repository: repo,
- Revision: revision,
- })
- require.NoError(t, err)
- testhelper.ProtoEqual(t, &gitalypb.ApplyGitattributesResponse{}, response)
-
- if expectedContent == nil {
- require.NoFileExists(t, attributesPath)
- } else {
- require.Equal(t, expectedContent, testhelper.MustReadFile(t, attributesPath))
-
- info, err := os.Stat(attributesPath)
- require.NoError(t, err)
- require.Equal(t, attributesFileMode, info.Mode())
- }
-}
diff --git a/internal/gitaly/service/repository/replicate_test.go b/internal/gitaly/service/repository/replicate_test.go
index 525032858..9d1840726 100644
--- a/internal/gitaly/service/repository/replicate_test.go
+++ b/internal/gitaly/service/repository/replicate_test.go
@@ -41,6 +41,18 @@ import (
"google.golang.org/protobuf/proto"
)
+type testTransactionServer struct {
+ gitalypb.UnimplementedRefTransactionServer
+ vote func(*gitalypb.VoteTransactionRequest) (*gitalypb.VoteTransactionResponse, error)
+}
+
+func (s *testTransactionServer) VoteTransaction(ctx context.Context, in *gitalypb.VoteTransactionRequest) (*gitalypb.VoteTransactionResponse, error) {
+ if s.vote != nil {
+ return s.vote(in)
+ }
+ return nil, nil
+}
+
func TestReplicateRepository(t *testing.T) {
testhelper.SkipWithWAL(t, `
ReplicateRepository is replicating git attributes as a separate file. WAL doesn't
diff --git a/internal/gitaly/storage/storagemgr/middleware.go b/internal/gitaly/storage/storagemgr/middleware.go
index ac36e972c..0df6c6479 100644
--- a/internal/gitaly/storage/storagemgr/middleware.go
+++ b/internal/gitaly/storage/storagemgr/middleware.go
@@ -50,9 +50,6 @@ var nonTransactionalRPCs = map[string]struct{}{
// Below RPCs implement functionality which isn't going to be supported by WAL.
// Handle these as non-transactional. Their usage must be removed prior to enabling WAL.
//
- // Attributes are going to be read from HEAD. Writing out a separate attributes file
- // won't be supported.
- "/gitaly.RepositoryService/ApplyGitattributes": {},
// 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": {},
diff --git a/internal/grpc/protoregistry/registry_test.go b/internal/grpc/protoregistry/registry_test.go
index de0ee38b3..aec830f97 100644
--- a/internal/grpc/protoregistry/registry_test.go
+++ b/internal/grpc/protoregistry/registry_test.go
@@ -90,7 +90,6 @@ func TestNewProtoRegistry(t *testing.T) {
"UpdateRemoteMirror": OpAccessor,
},
"RepositoryService": {
- "ApplyGitattributes": OpMutator,
"BackupCustomHooks": OpAccessor,
"CalculateChecksum": OpAccessor,
"CreateBundle": OpAccessor,
diff --git a/internal/praefect/coordinator.go b/internal/praefect/coordinator.go
index 47cd76a27..23f4c4aed 100644
--- a/internal/praefect/coordinator.go
+++ b/internal/praefect/coordinator.go
@@ -72,7 +72,6 @@ var transactionRPCs = map[string]transactionsCondition{
"/gitaly.OperationService/UserUpdateSubmodule": transactionsEnabled,
"/gitaly.RefService/DeleteRefs": transactionsEnabled,
"/gitaly.RefService/UpdateReferences": transactionsEnabled,
- "/gitaly.RepositoryService/ApplyGitattributes": transactionsEnabled,
"/gitaly.RepositoryService/CreateFork": transactionsEnabled,
"/gitaly.RepositoryService/CreateRepository": transactionsEnabled,
"/gitaly.RepositoryService/CreateRepositoryFromBundle": transactionsEnabled,
diff --git a/proto/go/gitalypb/repository.pb.go b/proto/go/gitalypb/repository.pb.go
index 01e2dfe9e..877c24ca7 100644
--- a/proto/go/gitalypb/repository.pb.go
+++ b/proto/go/gitalypb/repository.pb.go
@@ -74,7 +74,7 @@ func (x GetArchiveRequest_Format) Number() protoreflect.EnumNumber {
// Deprecated: Use GetArchiveRequest_Format.Descriptor instead.
func (GetArchiveRequest_Format) EnumDescriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{18, 0}
+ return file_repository_proto_rawDescGZIP(), []int{16, 0}
}
// Operation ...
@@ -143,7 +143,7 @@ func (x GetRawChangesResponse_RawChange_Operation) Number() protoreflect.EnumNum
// Deprecated: Use GetRawChangesResponse_RawChange_Operation.Descriptor instead.
func (GetRawChangesResponse_RawChange_Operation) EnumDescriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{61, 0, 0}
+ return file_repository_proto_rawDescGZIP(), []int{59, 0, 0}
}
// Strategy determines how the repository shall be optimized.
@@ -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{73, 0}
+ return file_repository_proto_rawDescGZIP(), []int{71, 0}
}
// RepositoryExistsRequest ...
@@ -731,103 +731,6 @@ func (x *ObjectFormatResponse) GetFormat() ObjectFormat {
return ObjectFormat_OBJECT_FORMAT_UNSPECIFIED
}
-// ApplyGitattributesRequest ...
-type ApplyGitattributesRequest struct {
- state protoimpl.MessageState
- sizeCache protoimpl.SizeCache
- unknownFields protoimpl.UnknownFields
-
- // repository ...
- Repository *Repository `protobuf:"bytes,1,opt,name=repository,proto3" json:"repository,omitempty"`
- // revision ...
- Revision []byte `protobuf:"bytes,2,opt,name=revision,proto3" json:"revision,omitempty"`
-}
-
-func (x *ApplyGitattributesRequest) Reset() {
- *x = ApplyGitattributesRequest{}
- if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[10]
- ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
- ms.StoreMessageInfo(mi)
- }
-}
-
-func (x *ApplyGitattributesRequest) String() string {
- return protoimpl.X.MessageStringOf(x)
-}
-
-func (*ApplyGitattributesRequest) ProtoMessage() {}
-
-func (x *ApplyGitattributesRequest) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[10]
- 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 ApplyGitattributesRequest.ProtoReflect.Descriptor instead.
-func (*ApplyGitattributesRequest) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{10}
-}
-
-func (x *ApplyGitattributesRequest) GetRepository() *Repository {
- if x != nil {
- return x.Repository
- }
- return nil
-}
-
-func (x *ApplyGitattributesRequest) GetRevision() []byte {
- if x != nil {
- return x.Revision
- }
- return nil
-}
-
-// ApplyGitattributesResponse ...
-type ApplyGitattributesResponse struct {
- state protoimpl.MessageState
- sizeCache protoimpl.SizeCache
- unknownFields protoimpl.UnknownFields
-}
-
-func (x *ApplyGitattributesResponse) Reset() {
- *x = ApplyGitattributesResponse{}
- if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[11]
- ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
- ms.StoreMessageInfo(mi)
- }
-}
-
-func (x *ApplyGitattributesResponse) String() string {
- return protoimpl.X.MessageStringOf(x)
-}
-
-func (*ApplyGitattributesResponse) ProtoMessage() {}
-
-func (x *ApplyGitattributesResponse) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[11]
- 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 ApplyGitattributesResponse.ProtoReflect.Descriptor instead.
-func (*ApplyGitattributesResponse) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{11}
-}
-
// FetchBundleRequest ...
type FetchBundleRequest struct {
state protoimpl.MessageState
@@ -845,7 +748,7 @@ type FetchBundleRequest struct {
func (x *FetchBundleRequest) Reset() {
*x = FetchBundleRequest{}
if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[12]
+ mi := &file_repository_proto_msgTypes[10]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -858,7 +761,7 @@ func (x *FetchBundleRequest) String() string {
func (*FetchBundleRequest) ProtoMessage() {}
func (x *FetchBundleRequest) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[12]
+ mi := &file_repository_proto_msgTypes[10]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -871,7 +774,7 @@ func (x *FetchBundleRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use FetchBundleRequest.ProtoReflect.Descriptor instead.
func (*FetchBundleRequest) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{12}
+ return file_repository_proto_rawDescGZIP(), []int{10}
}
func (x *FetchBundleRequest) GetRepository() *Repository {
@@ -905,7 +808,7 @@ type FetchBundleResponse struct {
func (x *FetchBundleResponse) Reset() {
*x = FetchBundleResponse{}
if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[13]
+ mi := &file_repository_proto_msgTypes[11]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -918,7 +821,7 @@ func (x *FetchBundleResponse) String() string {
func (*FetchBundleResponse) ProtoMessage() {}
func (x *FetchBundleResponse) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[13]
+ mi := &file_repository_proto_msgTypes[11]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -931,7 +834,7 @@ func (x *FetchBundleResponse) ProtoReflect() protoreflect.Message {
// Deprecated: Use FetchBundleResponse.ProtoReflect.Descriptor instead.
func (*FetchBundleResponse) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{13}
+ return file_repository_proto_rawDescGZIP(), []int{11}
}
// FetchRemoteRequest ...
@@ -967,7 +870,7 @@ type FetchRemoteRequest struct {
func (x *FetchRemoteRequest) Reset() {
*x = FetchRemoteRequest{}
if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[14]
+ mi := &file_repository_proto_msgTypes[12]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -980,7 +883,7 @@ func (x *FetchRemoteRequest) String() string {
func (*FetchRemoteRequest) ProtoMessage() {}
func (x *FetchRemoteRequest) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[14]
+ mi := &file_repository_proto_msgTypes[12]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -993,7 +896,7 @@ func (x *FetchRemoteRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use FetchRemoteRequest.ProtoReflect.Descriptor instead.
func (*FetchRemoteRequest) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{14}
+ return file_repository_proto_rawDescGZIP(), []int{12}
}
func (x *FetchRemoteRequest) GetRepository() *Repository {
@@ -1074,7 +977,7 @@ type FetchRemoteResponse struct {
func (x *FetchRemoteResponse) Reset() {
*x = FetchRemoteResponse{}
if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[15]
+ mi := &file_repository_proto_msgTypes[13]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -1087,7 +990,7 @@ func (x *FetchRemoteResponse) String() string {
func (*FetchRemoteResponse) ProtoMessage() {}
func (x *FetchRemoteResponse) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[15]
+ mi := &file_repository_proto_msgTypes[13]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -1100,7 +1003,7 @@ func (x *FetchRemoteResponse) ProtoReflect() protoreflect.Message {
// Deprecated: Use FetchRemoteResponse.ProtoReflect.Descriptor instead.
func (*FetchRemoteResponse) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{15}
+ return file_repository_proto_rawDescGZIP(), []int{13}
}
func (x *FetchRemoteResponse) GetTagsChanged() bool {
@@ -1131,7 +1034,7 @@ type CreateRepositoryRequest struct {
func (x *CreateRepositoryRequest) Reset() {
*x = CreateRepositoryRequest{}
if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[16]
+ mi := &file_repository_proto_msgTypes[14]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -1144,7 +1047,7 @@ func (x *CreateRepositoryRequest) String() string {
func (*CreateRepositoryRequest) ProtoMessage() {}
func (x *CreateRepositoryRequest) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[16]
+ mi := &file_repository_proto_msgTypes[14]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -1157,7 +1060,7 @@ func (x *CreateRepositoryRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use CreateRepositoryRequest.ProtoReflect.Descriptor instead.
func (*CreateRepositoryRequest) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{16}
+ return file_repository_proto_rawDescGZIP(), []int{14}
}
func (x *CreateRepositoryRequest) GetRepository() *Repository {
@@ -1191,7 +1094,7 @@ type CreateRepositoryResponse struct {
func (x *CreateRepositoryResponse) Reset() {
*x = CreateRepositoryResponse{}
if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[17]
+ mi := &file_repository_proto_msgTypes[15]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -1204,7 +1107,7 @@ func (x *CreateRepositoryResponse) String() string {
func (*CreateRepositoryResponse) ProtoMessage() {}
func (x *CreateRepositoryResponse) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[17]
+ mi := &file_repository_proto_msgTypes[15]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -1217,7 +1120,7 @@ func (x *CreateRepositoryResponse) ProtoReflect() protoreflect.Message {
// Deprecated: Use CreateRepositoryResponse.ProtoReflect.Descriptor instead.
func (*CreateRepositoryResponse) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{17}
+ return file_repository_proto_rawDescGZIP(), []int{15}
}
// GetArchiveRequest ...
@@ -1252,7 +1155,7 @@ type GetArchiveRequest struct {
func (x *GetArchiveRequest) Reset() {
*x = GetArchiveRequest{}
if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[18]
+ mi := &file_repository_proto_msgTypes[16]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -1265,7 +1168,7 @@ func (x *GetArchiveRequest) String() string {
func (*GetArchiveRequest) ProtoMessage() {}
func (x *GetArchiveRequest) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[18]
+ mi := &file_repository_proto_msgTypes[16]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -1278,7 +1181,7 @@ func (x *GetArchiveRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use GetArchiveRequest.ProtoReflect.Descriptor instead.
func (*GetArchiveRequest) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{18}
+ return file_repository_proto_rawDescGZIP(), []int{16}
}
func (x *GetArchiveRequest) GetRepository() *Repository {
@@ -1350,7 +1253,7 @@ type GetArchiveResponse struct {
func (x *GetArchiveResponse) Reset() {
*x = GetArchiveResponse{}
if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[19]
+ mi := &file_repository_proto_msgTypes[17]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -1363,7 +1266,7 @@ func (x *GetArchiveResponse) String() string {
func (*GetArchiveResponse) ProtoMessage() {}
func (x *GetArchiveResponse) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[19]
+ mi := &file_repository_proto_msgTypes[17]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -1376,7 +1279,7 @@ func (x *GetArchiveResponse) ProtoReflect() protoreflect.Message {
// Deprecated: Use GetArchiveResponse.ProtoReflect.Descriptor instead.
func (*GetArchiveResponse) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{19}
+ return file_repository_proto_rawDescGZIP(), []int{17}
}
func (x *GetArchiveResponse) GetData() []byte {
@@ -1399,7 +1302,7 @@ type HasLocalBranchesRequest struct {
func (x *HasLocalBranchesRequest) Reset() {
*x = HasLocalBranchesRequest{}
if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[20]
+ mi := &file_repository_proto_msgTypes[18]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -1412,7 +1315,7 @@ func (x *HasLocalBranchesRequest) String() string {
func (*HasLocalBranchesRequest) ProtoMessage() {}
func (x *HasLocalBranchesRequest) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[20]
+ mi := &file_repository_proto_msgTypes[18]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -1425,7 +1328,7 @@ func (x *HasLocalBranchesRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use HasLocalBranchesRequest.ProtoReflect.Descriptor instead.
func (*HasLocalBranchesRequest) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{20}
+ return file_repository_proto_rawDescGZIP(), []int{18}
}
func (x *HasLocalBranchesRequest) GetRepository() *Repository {
@@ -1448,7 +1351,7 @@ type HasLocalBranchesResponse struct {
func (x *HasLocalBranchesResponse) Reset() {
*x = HasLocalBranchesResponse{}
if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[21]
+ mi := &file_repository_proto_msgTypes[19]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -1461,7 +1364,7 @@ func (x *HasLocalBranchesResponse) String() string {
func (*HasLocalBranchesResponse) ProtoMessage() {}
func (x *HasLocalBranchesResponse) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[21]
+ mi := &file_repository_proto_msgTypes[19]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -1474,7 +1377,7 @@ func (x *HasLocalBranchesResponse) ProtoReflect() protoreflect.Message {
// Deprecated: Use HasLocalBranchesResponse.ProtoReflect.Descriptor instead.
func (*HasLocalBranchesResponse) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{21}
+ return file_repository_proto_rawDescGZIP(), []int{19}
}
func (x *HasLocalBranchesResponse) GetValue() bool {
@@ -1506,7 +1409,7 @@ type FetchSourceBranchRequest struct {
func (x *FetchSourceBranchRequest) Reset() {
*x = FetchSourceBranchRequest{}
if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[22]
+ mi := &file_repository_proto_msgTypes[20]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -1519,7 +1422,7 @@ func (x *FetchSourceBranchRequest) String() string {
func (*FetchSourceBranchRequest) ProtoMessage() {}
func (x *FetchSourceBranchRequest) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[22]
+ mi := &file_repository_proto_msgTypes[20]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -1532,7 +1435,7 @@ func (x *FetchSourceBranchRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use FetchSourceBranchRequest.ProtoReflect.Descriptor instead.
func (*FetchSourceBranchRequest) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{22}
+ return file_repository_proto_rawDescGZIP(), []int{20}
}
func (x *FetchSourceBranchRequest) GetRepository() *Repository {
@@ -1577,7 +1480,7 @@ type FetchSourceBranchResponse struct {
func (x *FetchSourceBranchResponse) Reset() {
*x = FetchSourceBranchResponse{}
if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[23]
+ mi := &file_repository_proto_msgTypes[21]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -1590,7 +1493,7 @@ func (x *FetchSourceBranchResponse) String() string {
func (*FetchSourceBranchResponse) ProtoMessage() {}
func (x *FetchSourceBranchResponse) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[23]
+ mi := &file_repository_proto_msgTypes[21]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -1603,7 +1506,7 @@ func (x *FetchSourceBranchResponse) ProtoReflect() protoreflect.Message {
// Deprecated: Use FetchSourceBranchResponse.ProtoReflect.Descriptor instead.
func (*FetchSourceBranchResponse) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{23}
+ return file_repository_proto_rawDescGZIP(), []int{21}
}
func (x *FetchSourceBranchResponse) GetResult() bool {
@@ -1626,7 +1529,7 @@ type FsckRequest struct {
func (x *FsckRequest) Reset() {
*x = FsckRequest{}
if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[24]
+ mi := &file_repository_proto_msgTypes[22]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -1639,7 +1542,7 @@ func (x *FsckRequest) String() string {
func (*FsckRequest) ProtoMessage() {}
func (x *FsckRequest) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[24]
+ mi := &file_repository_proto_msgTypes[22]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -1652,7 +1555,7 @@ func (x *FsckRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use FsckRequest.ProtoReflect.Descriptor instead.
func (*FsckRequest) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{24}
+ return file_repository_proto_rawDescGZIP(), []int{22}
}
func (x *FsckRequest) GetRepository() *Repository {
@@ -1675,7 +1578,7 @@ type FsckResponse struct {
func (x *FsckResponse) Reset() {
*x = FsckResponse{}
if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[25]
+ mi := &file_repository_proto_msgTypes[23]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -1688,7 +1591,7 @@ func (x *FsckResponse) String() string {
func (*FsckResponse) ProtoMessage() {}
func (x *FsckResponse) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[25]
+ mi := &file_repository_proto_msgTypes[23]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -1701,7 +1604,7 @@ func (x *FsckResponse) ProtoReflect() protoreflect.Message {
// Deprecated: Use FsckResponse.ProtoReflect.Descriptor instead.
func (*FsckResponse) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{25}
+ return file_repository_proto_rawDescGZIP(), []int{23}
}
func (x *FsckResponse) GetError() []byte {
@@ -1732,7 +1635,7 @@ type WriteRefRequest struct {
func (x *WriteRefRequest) Reset() {
*x = WriteRefRequest{}
if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[26]
+ mi := &file_repository_proto_msgTypes[24]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -1745,7 +1648,7 @@ func (x *WriteRefRequest) String() string {
func (*WriteRefRequest) ProtoMessage() {}
func (x *WriteRefRequest) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[26]
+ mi := &file_repository_proto_msgTypes[24]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -1758,7 +1661,7 @@ func (x *WriteRefRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use WriteRefRequest.ProtoReflect.Descriptor instead.
func (*WriteRefRequest) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{26}
+ return file_repository_proto_rawDescGZIP(), []int{24}
}
func (x *WriteRefRequest) GetRepository() *Repository {
@@ -1806,7 +1709,7 @@ type WriteRefResponse struct {
func (x *WriteRefResponse) Reset() {
*x = WriteRefResponse{}
if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[27]
+ mi := &file_repository_proto_msgTypes[25]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -1819,7 +1722,7 @@ func (x *WriteRefResponse) String() string {
func (*WriteRefResponse) ProtoMessage() {}
func (x *WriteRefResponse) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[27]
+ mi := &file_repository_proto_msgTypes[25]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -1832,7 +1735,7 @@ func (x *WriteRefResponse) ProtoReflect() protoreflect.Message {
// Deprecated: Use WriteRefResponse.ProtoReflect.Descriptor instead.
func (*WriteRefResponse) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{27}
+ return file_repository_proto_rawDescGZIP(), []int{25}
}
// FindMergeBaseRequest ...
@@ -1853,7 +1756,7 @@ type FindMergeBaseRequest struct {
func (x *FindMergeBaseRequest) Reset() {
*x = FindMergeBaseRequest{}
if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[28]
+ mi := &file_repository_proto_msgTypes[26]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -1866,7 +1769,7 @@ func (x *FindMergeBaseRequest) String() string {
func (*FindMergeBaseRequest) ProtoMessage() {}
func (x *FindMergeBaseRequest) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[28]
+ mi := &file_repository_proto_msgTypes[26]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -1879,7 +1782,7 @@ func (x *FindMergeBaseRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use FindMergeBaseRequest.ProtoReflect.Descriptor instead.
func (*FindMergeBaseRequest) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{28}
+ return file_repository_proto_rawDescGZIP(), []int{26}
}
func (x *FindMergeBaseRequest) GetRepository() *Repository {
@@ -1909,7 +1812,7 @@ type FindMergeBaseResponse struct {
func (x *FindMergeBaseResponse) Reset() {
*x = FindMergeBaseResponse{}
if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[29]
+ mi := &file_repository_proto_msgTypes[27]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -1922,7 +1825,7 @@ func (x *FindMergeBaseResponse) String() string {
func (*FindMergeBaseResponse) ProtoMessage() {}
func (x *FindMergeBaseResponse) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[29]
+ mi := &file_repository_proto_msgTypes[27]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -1935,7 +1838,7 @@ func (x *FindMergeBaseResponse) ProtoReflect() protoreflect.Message {
// Deprecated: Use FindMergeBaseResponse.ProtoReflect.Descriptor instead.
func (*FindMergeBaseResponse) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{29}
+ return file_repository_proto_rawDescGZIP(), []int{27}
}
func (x *FindMergeBaseResponse) GetBase() string {
@@ -1968,7 +1871,7 @@ type CreateForkRequest struct {
func (x *CreateForkRequest) Reset() {
*x = CreateForkRequest{}
if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[30]
+ mi := &file_repository_proto_msgTypes[28]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -1981,7 +1884,7 @@ func (x *CreateForkRequest) String() string {
func (*CreateForkRequest) ProtoMessage() {}
func (x *CreateForkRequest) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[30]
+ mi := &file_repository_proto_msgTypes[28]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -1994,7 +1897,7 @@ func (x *CreateForkRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use CreateForkRequest.ProtoReflect.Descriptor instead.
func (*CreateForkRequest) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{30}
+ return file_repository_proto_rawDescGZIP(), []int{28}
}
func (x *CreateForkRequest) GetRepository() *Repository {
@@ -2028,7 +1931,7 @@ type CreateForkResponse struct {
func (x *CreateForkResponse) Reset() {
*x = CreateForkResponse{}
if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[31]
+ mi := &file_repository_proto_msgTypes[29]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -2041,7 +1944,7 @@ func (x *CreateForkResponse) String() string {
func (*CreateForkResponse) ProtoMessage() {}
func (x *CreateForkResponse) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[31]
+ mi := &file_repository_proto_msgTypes[29]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -2054,7 +1957,7 @@ func (x *CreateForkResponse) ProtoReflect() protoreflect.Message {
// Deprecated: Use CreateForkResponse.ProtoReflect.Descriptor instead.
func (*CreateForkResponse) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{31}
+ return file_repository_proto_rawDescGZIP(), []int{29}
}
// CreateRepositoryFromURLRequest ...
@@ -2088,7 +1991,7 @@ type CreateRepositoryFromURLRequest struct {
func (x *CreateRepositoryFromURLRequest) Reset() {
*x = CreateRepositoryFromURLRequest{}
if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[32]
+ mi := &file_repository_proto_msgTypes[30]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -2101,7 +2004,7 @@ func (x *CreateRepositoryFromURLRequest) String() string {
func (*CreateRepositoryFromURLRequest) ProtoMessage() {}
func (x *CreateRepositoryFromURLRequest) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[32]
+ mi := &file_repository_proto_msgTypes[30]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -2114,7 +2017,7 @@ func (x *CreateRepositoryFromURLRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use CreateRepositoryFromURLRequest.ProtoReflect.Descriptor instead.
func (*CreateRepositoryFromURLRequest) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{32}
+ return file_repository_proto_rawDescGZIP(), []int{30}
}
func (x *CreateRepositoryFromURLRequest) GetRepository() *Repository {
@@ -2162,7 +2065,7 @@ type CreateRepositoryFromURLResponse struct {
func (x *CreateRepositoryFromURLResponse) Reset() {
*x = CreateRepositoryFromURLResponse{}
if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[33]
+ mi := &file_repository_proto_msgTypes[31]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -2175,7 +2078,7 @@ func (x *CreateRepositoryFromURLResponse) String() string {
func (*CreateRepositoryFromURLResponse) ProtoMessage() {}
func (x *CreateRepositoryFromURLResponse) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[33]
+ mi := &file_repository_proto_msgTypes[31]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -2188,7 +2091,7 @@ func (x *CreateRepositoryFromURLResponse) ProtoReflect() protoreflect.Message {
// Deprecated: Use CreateRepositoryFromURLResponse.ProtoReflect.Descriptor instead.
func (*CreateRepositoryFromURLResponse) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{33}
+ return file_repository_proto_rawDescGZIP(), []int{31}
}
// CreateBundleRequest ...
@@ -2204,7 +2107,7 @@ type CreateBundleRequest struct {
func (x *CreateBundleRequest) Reset() {
*x = CreateBundleRequest{}
if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[34]
+ mi := &file_repository_proto_msgTypes[32]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -2217,7 +2120,7 @@ func (x *CreateBundleRequest) String() string {
func (*CreateBundleRequest) ProtoMessage() {}
func (x *CreateBundleRequest) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[34]
+ mi := &file_repository_proto_msgTypes[32]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -2230,7 +2133,7 @@ func (x *CreateBundleRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use CreateBundleRequest.ProtoReflect.Descriptor instead.
func (*CreateBundleRequest) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{34}
+ return file_repository_proto_rawDescGZIP(), []int{32}
}
func (x *CreateBundleRequest) GetRepository() *Repository {
@@ -2253,7 +2156,7 @@ type CreateBundleResponse struct {
func (x *CreateBundleResponse) Reset() {
*x = CreateBundleResponse{}
if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[35]
+ mi := &file_repository_proto_msgTypes[33]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -2266,7 +2169,7 @@ func (x *CreateBundleResponse) String() string {
func (*CreateBundleResponse) ProtoMessage() {}
func (x *CreateBundleResponse) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[35]
+ mi := &file_repository_proto_msgTypes[33]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -2279,7 +2182,7 @@ func (x *CreateBundleResponse) ProtoReflect() protoreflect.Message {
// Deprecated: Use CreateBundleResponse.ProtoReflect.Descriptor instead.
func (*CreateBundleResponse) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{35}
+ return file_repository_proto_rawDescGZIP(), []int{33}
}
func (x *CreateBundleResponse) GetData() []byte {
@@ -2306,7 +2209,7 @@ type CreateBundleFromRefListRequest struct {
func (x *CreateBundleFromRefListRequest) Reset() {
*x = CreateBundleFromRefListRequest{}
if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[36]
+ mi := &file_repository_proto_msgTypes[34]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -2319,7 +2222,7 @@ func (x *CreateBundleFromRefListRequest) String() string {
func (*CreateBundleFromRefListRequest) ProtoMessage() {}
func (x *CreateBundleFromRefListRequest) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[36]
+ mi := &file_repository_proto_msgTypes[34]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -2332,7 +2235,7 @@ func (x *CreateBundleFromRefListRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use CreateBundleFromRefListRequest.ProtoReflect.Descriptor instead.
func (*CreateBundleFromRefListRequest) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{36}
+ return file_repository_proto_rawDescGZIP(), []int{34}
}
func (x *CreateBundleFromRefListRequest) GetRepository() *Repository {
@@ -2362,7 +2265,7 @@ type CreateBundleFromRefListResponse struct {
func (x *CreateBundleFromRefListResponse) Reset() {
*x = CreateBundleFromRefListResponse{}
if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[37]
+ mi := &file_repository_proto_msgTypes[35]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -2375,7 +2278,7 @@ func (x *CreateBundleFromRefListResponse) String() string {
func (*CreateBundleFromRefListResponse) ProtoMessage() {}
func (x *CreateBundleFromRefListResponse) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[37]
+ mi := &file_repository_proto_msgTypes[35]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -2388,7 +2291,7 @@ func (x *CreateBundleFromRefListResponse) ProtoReflect() protoreflect.Message {
// Deprecated: Use CreateBundleFromRefListResponse.ProtoReflect.Descriptor instead.
func (*CreateBundleFromRefListResponse) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{37}
+ return file_repository_proto_rawDescGZIP(), []int{35}
}
func (x *CreateBundleFromRefListResponse) GetData() []byte {
@@ -2412,7 +2315,7 @@ type GetConfigRequest struct {
func (x *GetConfigRequest) Reset() {
*x = GetConfigRequest{}
if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[38]
+ mi := &file_repository_proto_msgTypes[36]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -2425,7 +2328,7 @@ func (x *GetConfigRequest) String() string {
func (*GetConfigRequest) ProtoMessage() {}
func (x *GetConfigRequest) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[38]
+ mi := &file_repository_proto_msgTypes[36]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -2438,7 +2341,7 @@ func (x *GetConfigRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use GetConfigRequest.ProtoReflect.Descriptor instead.
func (*GetConfigRequest) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{38}
+ return file_repository_proto_rawDescGZIP(), []int{36}
}
func (x *GetConfigRequest) GetRepository() *Repository {
@@ -2461,7 +2364,7 @@ type GetConfigResponse struct {
func (x *GetConfigResponse) Reset() {
*x = GetConfigResponse{}
if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[39]
+ mi := &file_repository_proto_msgTypes[37]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -2474,7 +2377,7 @@ func (x *GetConfigResponse) String() string {
func (*GetConfigResponse) ProtoMessage() {}
func (x *GetConfigResponse) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[39]
+ mi := &file_repository_proto_msgTypes[37]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -2487,7 +2390,7 @@ func (x *GetConfigResponse) ProtoReflect() protoreflect.Message {
// Deprecated: Use GetConfigResponse.ProtoReflect.Descriptor instead.
func (*GetConfigResponse) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{39}
+ return file_repository_proto_rawDescGZIP(), []int{37}
}
func (x *GetConfigResponse) GetData() []byte {
@@ -2512,7 +2415,7 @@ type RestoreCustomHooksRequest struct {
func (x *RestoreCustomHooksRequest) Reset() {
*x = RestoreCustomHooksRequest{}
if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[40]
+ mi := &file_repository_proto_msgTypes[38]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -2525,7 +2428,7 @@ func (x *RestoreCustomHooksRequest) String() string {
func (*RestoreCustomHooksRequest) ProtoMessage() {}
func (x *RestoreCustomHooksRequest) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[40]
+ mi := &file_repository_proto_msgTypes[38]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -2538,7 +2441,7 @@ func (x *RestoreCustomHooksRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use RestoreCustomHooksRequest.ProtoReflect.Descriptor instead.
func (*RestoreCustomHooksRequest) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{40}
+ return file_repository_proto_rawDescGZIP(), []int{38}
}
func (x *RestoreCustomHooksRequest) GetRepository() *Repository {
@@ -2570,7 +2473,7 @@ type SetCustomHooksRequest struct {
func (x *SetCustomHooksRequest) Reset() {
*x = SetCustomHooksRequest{}
if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[41]
+ mi := &file_repository_proto_msgTypes[39]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -2583,7 +2486,7 @@ func (x *SetCustomHooksRequest) String() string {
func (*SetCustomHooksRequest) ProtoMessage() {}
func (x *SetCustomHooksRequest) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[41]
+ mi := &file_repository_proto_msgTypes[39]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -2596,7 +2499,7 @@ func (x *SetCustomHooksRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use SetCustomHooksRequest.ProtoReflect.Descriptor instead.
func (*SetCustomHooksRequest) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{41}
+ return file_repository_proto_rawDescGZIP(), []int{39}
}
func (x *SetCustomHooksRequest) GetRepository() *Repository {
@@ -2623,7 +2526,7 @@ type RestoreCustomHooksResponse struct {
func (x *RestoreCustomHooksResponse) Reset() {
*x = RestoreCustomHooksResponse{}
if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[42]
+ mi := &file_repository_proto_msgTypes[40]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -2636,7 +2539,7 @@ func (x *RestoreCustomHooksResponse) String() string {
func (*RestoreCustomHooksResponse) ProtoMessage() {}
func (x *RestoreCustomHooksResponse) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[42]
+ mi := &file_repository_proto_msgTypes[40]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -2649,7 +2552,7 @@ func (x *RestoreCustomHooksResponse) ProtoReflect() protoreflect.Message {
// Deprecated: Use RestoreCustomHooksResponse.ProtoReflect.Descriptor instead.
func (*RestoreCustomHooksResponse) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{42}
+ return file_repository_proto_rawDescGZIP(), []int{40}
}
// SetCustomHooksResponse ...
@@ -2662,7 +2565,7 @@ type SetCustomHooksResponse struct {
func (x *SetCustomHooksResponse) Reset() {
*x = SetCustomHooksResponse{}
if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[43]
+ mi := &file_repository_proto_msgTypes[41]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -2675,7 +2578,7 @@ func (x *SetCustomHooksResponse) String() string {
func (*SetCustomHooksResponse) ProtoMessage() {}
func (x *SetCustomHooksResponse) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[43]
+ mi := &file_repository_proto_msgTypes[41]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -2688,7 +2591,7 @@ func (x *SetCustomHooksResponse) ProtoReflect() protoreflect.Message {
// Deprecated: Use SetCustomHooksResponse.ProtoReflect.Descriptor instead.
func (*SetCustomHooksResponse) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{43}
+ return file_repository_proto_rawDescGZIP(), []int{41}
}
// BackupCustomHooksRequest ...
@@ -2704,7 +2607,7 @@ type BackupCustomHooksRequest struct {
func (x *BackupCustomHooksRequest) Reset() {
*x = BackupCustomHooksRequest{}
if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[44]
+ mi := &file_repository_proto_msgTypes[42]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -2717,7 +2620,7 @@ func (x *BackupCustomHooksRequest) String() string {
func (*BackupCustomHooksRequest) ProtoMessage() {}
func (x *BackupCustomHooksRequest) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[44]
+ mi := &file_repository_proto_msgTypes[42]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -2730,7 +2633,7 @@ func (x *BackupCustomHooksRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use BackupCustomHooksRequest.ProtoReflect.Descriptor instead.
func (*BackupCustomHooksRequest) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{44}
+ return file_repository_proto_rawDescGZIP(), []int{42}
}
func (x *BackupCustomHooksRequest) GetRepository() *Repository {
@@ -2753,7 +2656,7 @@ type GetCustomHooksRequest struct {
func (x *GetCustomHooksRequest) Reset() {
*x = GetCustomHooksRequest{}
if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[45]
+ mi := &file_repository_proto_msgTypes[43]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -2766,7 +2669,7 @@ func (x *GetCustomHooksRequest) String() string {
func (*GetCustomHooksRequest) ProtoMessage() {}
func (x *GetCustomHooksRequest) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[45]
+ mi := &file_repository_proto_msgTypes[43]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -2779,7 +2682,7 @@ func (x *GetCustomHooksRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use GetCustomHooksRequest.ProtoReflect.Descriptor instead.
func (*GetCustomHooksRequest) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{45}
+ return file_repository_proto_rawDescGZIP(), []int{43}
}
func (x *GetCustomHooksRequest) GetRepository() *Repository {
@@ -2802,7 +2705,7 @@ type BackupCustomHooksResponse struct {
func (x *BackupCustomHooksResponse) Reset() {
*x = BackupCustomHooksResponse{}
if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[46]
+ mi := &file_repository_proto_msgTypes[44]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -2815,7 +2718,7 @@ func (x *BackupCustomHooksResponse) String() string {
func (*BackupCustomHooksResponse) ProtoMessage() {}
func (x *BackupCustomHooksResponse) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[46]
+ mi := &file_repository_proto_msgTypes[44]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -2828,7 +2731,7 @@ func (x *BackupCustomHooksResponse) ProtoReflect() protoreflect.Message {
// Deprecated: Use BackupCustomHooksResponse.ProtoReflect.Descriptor instead.
func (*BackupCustomHooksResponse) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{46}
+ return file_repository_proto_rawDescGZIP(), []int{44}
}
func (x *BackupCustomHooksResponse) GetData() []byte {
@@ -2851,7 +2754,7 @@ type GetCustomHooksResponse struct {
func (x *GetCustomHooksResponse) Reset() {
*x = GetCustomHooksResponse{}
if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[47]
+ mi := &file_repository_proto_msgTypes[45]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -2864,7 +2767,7 @@ func (x *GetCustomHooksResponse) String() string {
func (*GetCustomHooksResponse) ProtoMessage() {}
func (x *GetCustomHooksResponse) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[47]
+ mi := &file_repository_proto_msgTypes[45]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -2877,7 +2780,7 @@ func (x *GetCustomHooksResponse) ProtoReflect() protoreflect.Message {
// Deprecated: Use GetCustomHooksResponse.ProtoReflect.Descriptor instead.
func (*GetCustomHooksResponse) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{47}
+ return file_repository_proto_rawDescGZIP(), []int{45}
}
func (x *GetCustomHooksResponse) GetData() []byte {
@@ -2903,7 +2806,7 @@ type CreateRepositoryFromBundleRequest struct {
func (x *CreateRepositoryFromBundleRequest) Reset() {
*x = CreateRepositoryFromBundleRequest{}
if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[48]
+ mi := &file_repository_proto_msgTypes[46]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -2916,7 +2819,7 @@ func (x *CreateRepositoryFromBundleRequest) String() string {
func (*CreateRepositoryFromBundleRequest) ProtoMessage() {}
func (x *CreateRepositoryFromBundleRequest) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[48]
+ mi := &file_repository_proto_msgTypes[46]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -2929,7 +2832,7 @@ func (x *CreateRepositoryFromBundleRequest) ProtoReflect() protoreflect.Message
// Deprecated: Use CreateRepositoryFromBundleRequest.ProtoReflect.Descriptor instead.
func (*CreateRepositoryFromBundleRequest) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{48}
+ return file_repository_proto_rawDescGZIP(), []int{46}
}
func (x *CreateRepositoryFromBundleRequest) GetRepository() *Repository {
@@ -2956,7 +2859,7 @@ type CreateRepositoryFromBundleResponse struct {
func (x *CreateRepositoryFromBundleResponse) Reset() {
*x = CreateRepositoryFromBundleResponse{}
if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[49]
+ mi := &file_repository_proto_msgTypes[47]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -2969,7 +2872,7 @@ func (x *CreateRepositoryFromBundleResponse) String() string {
func (*CreateRepositoryFromBundleResponse) ProtoMessage() {}
func (x *CreateRepositoryFromBundleResponse) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[49]
+ mi := &file_repository_proto_msgTypes[47]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -2982,7 +2885,7 @@ func (x *CreateRepositoryFromBundleResponse) ProtoReflect() protoreflect.Message
// Deprecated: Use CreateRepositoryFromBundleResponse.ProtoReflect.Descriptor instead.
func (*CreateRepositoryFromBundleResponse) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{49}
+ return file_repository_proto_rawDescGZIP(), []int{47}
}
// FindLicenseRequest asks to detect the license for the given repository.
@@ -2998,7 +2901,7 @@ type FindLicenseRequest struct {
func (x *FindLicenseRequest) Reset() {
*x = FindLicenseRequest{}
if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[50]
+ mi := &file_repository_proto_msgTypes[48]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -3011,7 +2914,7 @@ func (x *FindLicenseRequest) String() string {
func (*FindLicenseRequest) ProtoMessage() {}
func (x *FindLicenseRequest) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[50]
+ mi := &file_repository_proto_msgTypes[48]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -3024,7 +2927,7 @@ func (x *FindLicenseRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use FindLicenseRequest.ProtoReflect.Descriptor instead.
func (*FindLicenseRequest) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{50}
+ return file_repository_proto_rawDescGZIP(), []int{48}
}
func (x *FindLicenseRequest) GetRepository() *Repository {
@@ -3062,7 +2965,7 @@ type FindLicenseResponse struct {
func (x *FindLicenseResponse) Reset() {
*x = FindLicenseResponse{}
if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[51]
+ mi := &file_repository_proto_msgTypes[49]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -3075,7 +2978,7 @@ func (x *FindLicenseResponse) String() string {
func (*FindLicenseResponse) ProtoMessage() {}
func (x *FindLicenseResponse) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[51]
+ mi := &file_repository_proto_msgTypes[49]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -3088,7 +2991,7 @@ func (x *FindLicenseResponse) ProtoReflect() protoreflect.Message {
// Deprecated: Use FindLicenseResponse.ProtoReflect.Descriptor instead.
func (*FindLicenseResponse) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{51}
+ return file_repository_proto_rawDescGZIP(), []int{49}
}
func (x *FindLicenseResponse) GetLicenseShortName() string {
@@ -3139,7 +3042,7 @@ type GetInfoAttributesRequest struct {
func (x *GetInfoAttributesRequest) Reset() {
*x = GetInfoAttributesRequest{}
if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[52]
+ mi := &file_repository_proto_msgTypes[50]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -3152,7 +3055,7 @@ func (x *GetInfoAttributesRequest) String() string {
func (*GetInfoAttributesRequest) ProtoMessage() {}
func (x *GetInfoAttributesRequest) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[52]
+ mi := &file_repository_proto_msgTypes[50]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -3165,7 +3068,7 @@ func (x *GetInfoAttributesRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use GetInfoAttributesRequest.ProtoReflect.Descriptor instead.
func (*GetInfoAttributesRequest) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{52}
+ return file_repository_proto_rawDescGZIP(), []int{50}
}
func (x *GetInfoAttributesRequest) GetRepository() *Repository {
@@ -3188,7 +3091,7 @@ type GetInfoAttributesResponse struct {
func (x *GetInfoAttributesResponse) Reset() {
*x = GetInfoAttributesResponse{}
if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[53]
+ mi := &file_repository_proto_msgTypes[51]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -3201,7 +3104,7 @@ func (x *GetInfoAttributesResponse) String() string {
func (*GetInfoAttributesResponse) ProtoMessage() {}
func (x *GetInfoAttributesResponse) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[53]
+ mi := &file_repository_proto_msgTypes[51]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -3214,7 +3117,7 @@ func (x *GetInfoAttributesResponse) ProtoReflect() protoreflect.Message {
// Deprecated: Use GetInfoAttributesResponse.ProtoReflect.Descriptor instead.
func (*GetInfoAttributesResponse) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{53}
+ return file_repository_proto_rawDescGZIP(), []int{51}
}
func (x *GetInfoAttributesResponse) GetAttributes() []byte {
@@ -3237,7 +3140,7 @@ type CalculateChecksumRequest struct {
func (x *CalculateChecksumRequest) Reset() {
*x = CalculateChecksumRequest{}
if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[54]
+ mi := &file_repository_proto_msgTypes[52]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -3250,7 +3153,7 @@ func (x *CalculateChecksumRequest) String() string {
func (*CalculateChecksumRequest) ProtoMessage() {}
func (x *CalculateChecksumRequest) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[54]
+ mi := &file_repository_proto_msgTypes[52]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -3263,7 +3166,7 @@ func (x *CalculateChecksumRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use CalculateChecksumRequest.ProtoReflect.Descriptor instead.
func (*CalculateChecksumRequest) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{54}
+ return file_repository_proto_rawDescGZIP(), []int{52}
}
func (x *CalculateChecksumRequest) GetRepository() *Repository {
@@ -3286,7 +3189,7 @@ type CalculateChecksumResponse struct {
func (x *CalculateChecksumResponse) Reset() {
*x = CalculateChecksumResponse{}
if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[55]
+ mi := &file_repository_proto_msgTypes[53]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -3299,7 +3202,7 @@ func (x *CalculateChecksumResponse) String() string {
func (*CalculateChecksumResponse) ProtoMessage() {}
func (x *CalculateChecksumResponse) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[55]
+ mi := &file_repository_proto_msgTypes[53]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -3312,7 +3215,7 @@ func (x *CalculateChecksumResponse) ProtoReflect() protoreflect.Message {
// Deprecated: Use CalculateChecksumResponse.ProtoReflect.Descriptor instead.
func (*CalculateChecksumResponse) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{55}
+ return file_repository_proto_rawDescGZIP(), []int{53}
}
func (x *CalculateChecksumResponse) GetChecksum() string {
@@ -3335,7 +3238,7 @@ type GetSnapshotRequest struct {
func (x *GetSnapshotRequest) Reset() {
*x = GetSnapshotRequest{}
if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[56]
+ mi := &file_repository_proto_msgTypes[54]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -3348,7 +3251,7 @@ func (x *GetSnapshotRequest) String() string {
func (*GetSnapshotRequest) ProtoMessage() {}
func (x *GetSnapshotRequest) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[56]
+ mi := &file_repository_proto_msgTypes[54]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -3361,7 +3264,7 @@ func (x *GetSnapshotRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use GetSnapshotRequest.ProtoReflect.Descriptor instead.
func (*GetSnapshotRequest) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{56}
+ return file_repository_proto_rawDescGZIP(), []int{54}
}
func (x *GetSnapshotRequest) GetRepository() *Repository {
@@ -3384,7 +3287,7 @@ type GetSnapshotResponse struct {
func (x *GetSnapshotResponse) Reset() {
*x = GetSnapshotResponse{}
if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[57]
+ mi := &file_repository_proto_msgTypes[55]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -3397,7 +3300,7 @@ func (x *GetSnapshotResponse) String() string {
func (*GetSnapshotResponse) ProtoMessage() {}
func (x *GetSnapshotResponse) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[57]
+ mi := &file_repository_proto_msgTypes[55]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -3410,7 +3313,7 @@ func (x *GetSnapshotResponse) ProtoReflect() protoreflect.Message {
// Deprecated: Use GetSnapshotResponse.ProtoReflect.Descriptor instead.
func (*GetSnapshotResponse) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{57}
+ return file_repository_proto_rawDescGZIP(), []int{55}
}
func (x *GetSnapshotResponse) GetData() []byte {
@@ -3444,7 +3347,7 @@ type CreateRepositoryFromSnapshotRequest struct {
func (x *CreateRepositoryFromSnapshotRequest) Reset() {
*x = CreateRepositoryFromSnapshotRequest{}
if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[58]
+ mi := &file_repository_proto_msgTypes[56]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -3457,7 +3360,7 @@ func (x *CreateRepositoryFromSnapshotRequest) String() string {
func (*CreateRepositoryFromSnapshotRequest) ProtoMessage() {}
func (x *CreateRepositoryFromSnapshotRequest) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[58]
+ mi := &file_repository_proto_msgTypes[56]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -3470,7 +3373,7 @@ func (x *CreateRepositoryFromSnapshotRequest) ProtoReflect() protoreflect.Messag
// Deprecated: Use CreateRepositoryFromSnapshotRequest.ProtoReflect.Descriptor instead.
func (*CreateRepositoryFromSnapshotRequest) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{58}
+ return file_repository_proto_rawDescGZIP(), []int{56}
}
func (x *CreateRepositoryFromSnapshotRequest) GetRepository() *Repository {
@@ -3511,7 +3414,7 @@ type CreateRepositoryFromSnapshotResponse struct {
func (x *CreateRepositoryFromSnapshotResponse) Reset() {
*x = CreateRepositoryFromSnapshotResponse{}
if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[59]
+ mi := &file_repository_proto_msgTypes[57]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -3524,7 +3427,7 @@ func (x *CreateRepositoryFromSnapshotResponse) String() string {
func (*CreateRepositoryFromSnapshotResponse) ProtoMessage() {}
func (x *CreateRepositoryFromSnapshotResponse) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[59]
+ mi := &file_repository_proto_msgTypes[57]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -3537,7 +3440,7 @@ func (x *CreateRepositoryFromSnapshotResponse) ProtoReflect() protoreflect.Messa
// Deprecated: Use CreateRepositoryFromSnapshotResponse.ProtoReflect.Descriptor instead.
func (*CreateRepositoryFromSnapshotResponse) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{59}
+ return file_repository_proto_rawDescGZIP(), []int{57}
}
// GetRawChangesRequest ...
@@ -3557,7 +3460,7 @@ type GetRawChangesRequest struct {
func (x *GetRawChangesRequest) Reset() {
*x = GetRawChangesRequest{}
if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[60]
+ mi := &file_repository_proto_msgTypes[58]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -3570,7 +3473,7 @@ func (x *GetRawChangesRequest) String() string {
func (*GetRawChangesRequest) ProtoMessage() {}
func (x *GetRawChangesRequest) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[60]
+ mi := &file_repository_proto_msgTypes[58]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -3583,7 +3486,7 @@ func (x *GetRawChangesRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use GetRawChangesRequest.ProtoReflect.Descriptor instead.
func (*GetRawChangesRequest) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{60}
+ return file_repository_proto_rawDescGZIP(), []int{58}
}
func (x *GetRawChangesRequest) GetRepository() *Repository {
@@ -3620,7 +3523,7 @@ type GetRawChangesResponse struct {
func (x *GetRawChangesResponse) Reset() {
*x = GetRawChangesResponse{}
if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[61]
+ mi := &file_repository_proto_msgTypes[59]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -3633,7 +3536,7 @@ func (x *GetRawChangesResponse) String() string {
func (*GetRawChangesResponse) ProtoMessage() {}
func (x *GetRawChangesResponse) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[61]
+ mi := &file_repository_proto_msgTypes[59]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -3646,7 +3549,7 @@ func (x *GetRawChangesResponse) ProtoReflect() protoreflect.Message {
// Deprecated: Use GetRawChangesResponse.ProtoReflect.Descriptor instead.
func (*GetRawChangesResponse) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{61}
+ return file_repository_proto_rawDescGZIP(), []int{59}
}
func (x *GetRawChangesResponse) GetRawChanges() []*GetRawChangesResponse_RawChange {
@@ -3684,7 +3587,7 @@ type SearchFilesByNameRequest struct {
func (x *SearchFilesByNameRequest) Reset() {
*x = SearchFilesByNameRequest{}
if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[62]
+ mi := &file_repository_proto_msgTypes[60]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -3697,7 +3600,7 @@ func (x *SearchFilesByNameRequest) String() string {
func (*SearchFilesByNameRequest) ProtoMessage() {}
func (x *SearchFilesByNameRequest) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[62]
+ mi := &file_repository_proto_msgTypes[60]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -3710,7 +3613,7 @@ func (x *SearchFilesByNameRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use SearchFilesByNameRequest.ProtoReflect.Descriptor instead.
func (*SearchFilesByNameRequest) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{62}
+ return file_repository_proto_rawDescGZIP(), []int{60}
}
func (x *SearchFilesByNameRequest) GetRepository() *Repository {
@@ -3768,7 +3671,7 @@ type SearchFilesByNameResponse struct {
func (x *SearchFilesByNameResponse) Reset() {
*x = SearchFilesByNameResponse{}
if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[63]
+ mi := &file_repository_proto_msgTypes[61]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -3781,7 +3684,7 @@ func (x *SearchFilesByNameResponse) String() string {
func (*SearchFilesByNameResponse) ProtoMessage() {}
func (x *SearchFilesByNameResponse) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[63]
+ mi := &file_repository_proto_msgTypes[61]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -3794,7 +3697,7 @@ func (x *SearchFilesByNameResponse) ProtoReflect() protoreflect.Message {
// Deprecated: Use SearchFilesByNameResponse.ProtoReflect.Descriptor instead.
func (*SearchFilesByNameResponse) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{63}
+ return file_repository_proto_rawDescGZIP(), []int{61}
}
func (x *SearchFilesByNameResponse) GetFiles() [][]byte {
@@ -3823,7 +3726,7 @@ type SearchFilesByContentRequest struct {
func (x *SearchFilesByContentRequest) Reset() {
*x = SearchFilesByContentRequest{}
if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[64]
+ mi := &file_repository_proto_msgTypes[62]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -3836,7 +3739,7 @@ func (x *SearchFilesByContentRequest) String() string {
func (*SearchFilesByContentRequest) ProtoMessage() {}
func (x *SearchFilesByContentRequest) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[64]
+ mi := &file_repository_proto_msgTypes[62]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -3849,7 +3752,7 @@ func (x *SearchFilesByContentRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use SearchFilesByContentRequest.ProtoReflect.Descriptor instead.
func (*SearchFilesByContentRequest) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{64}
+ return file_repository_proto_rawDescGZIP(), []int{62}
}
func (x *SearchFilesByContentRequest) GetRepository() *Repository {
@@ -3897,7 +3800,7 @@ type SearchFilesByContentResponse struct {
func (x *SearchFilesByContentResponse) Reset() {
*x = SearchFilesByContentResponse{}
if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[65]
+ mi := &file_repository_proto_msgTypes[63]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -3910,7 +3813,7 @@ func (x *SearchFilesByContentResponse) String() string {
func (*SearchFilesByContentResponse) ProtoMessage() {}
func (x *SearchFilesByContentResponse) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[65]
+ mi := &file_repository_proto_msgTypes[63]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -3923,7 +3826,7 @@ func (x *SearchFilesByContentResponse) ProtoReflect() protoreflect.Message {
// Deprecated: Use SearchFilesByContentResponse.ProtoReflect.Descriptor instead.
func (*SearchFilesByContentResponse) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{65}
+ return file_repository_proto_rawDescGZIP(), []int{63}
}
func (x *SearchFilesByContentResponse) GetMatches() [][]byte {
@@ -3982,7 +3885,7 @@ type Remote struct {
func (x *Remote) Reset() {
*x = Remote{}
if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[66]
+ mi := &file_repository_proto_msgTypes[64]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -3995,7 +3898,7 @@ func (x *Remote) String() string {
func (*Remote) ProtoMessage() {}
func (x *Remote) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[66]
+ mi := &file_repository_proto_msgTypes[64]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -4008,7 +3911,7 @@ func (x *Remote) ProtoReflect() protoreflect.Message {
// Deprecated: Use Remote.ProtoReflect.Descriptor instead.
func (*Remote) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{66}
+ return file_repository_proto_rawDescGZIP(), []int{64}
}
func (x *Remote) GetUrl() string {
@@ -4052,7 +3955,7 @@ type GetObjectDirectorySizeRequest struct {
func (x *GetObjectDirectorySizeRequest) Reset() {
*x = GetObjectDirectorySizeRequest{}
if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[67]
+ mi := &file_repository_proto_msgTypes[65]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -4065,7 +3968,7 @@ func (x *GetObjectDirectorySizeRequest) String() string {
func (*GetObjectDirectorySizeRequest) ProtoMessage() {}
func (x *GetObjectDirectorySizeRequest) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[67]
+ mi := &file_repository_proto_msgTypes[65]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -4078,7 +3981,7 @@ func (x *GetObjectDirectorySizeRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use GetObjectDirectorySizeRequest.ProtoReflect.Descriptor instead.
func (*GetObjectDirectorySizeRequest) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{67}
+ return file_repository_proto_rawDescGZIP(), []int{65}
}
func (x *GetObjectDirectorySizeRequest) GetRepository() *Repository {
@@ -4101,7 +4004,7 @@ type GetObjectDirectorySizeResponse struct {
func (x *GetObjectDirectorySizeResponse) Reset() {
*x = GetObjectDirectorySizeResponse{}
if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[68]
+ mi := &file_repository_proto_msgTypes[66]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -4114,7 +4017,7 @@ func (x *GetObjectDirectorySizeResponse) String() string {
func (*GetObjectDirectorySizeResponse) ProtoMessage() {}
func (x *GetObjectDirectorySizeResponse) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[68]
+ mi := &file_repository_proto_msgTypes[66]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -4127,7 +4030,7 @@ func (x *GetObjectDirectorySizeResponse) ProtoReflect() protoreflect.Message {
// Deprecated: Use GetObjectDirectorySizeResponse.ProtoReflect.Descriptor instead.
func (*GetObjectDirectorySizeResponse) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{68}
+ return file_repository_proto_rawDescGZIP(), []int{66}
}
func (x *GetObjectDirectorySizeResponse) GetSize() int64 {
@@ -4150,7 +4053,7 @@ type RemoveRepositoryRequest struct {
func (x *RemoveRepositoryRequest) Reset() {
*x = RemoveRepositoryRequest{}
if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[69]
+ mi := &file_repository_proto_msgTypes[67]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -4163,7 +4066,7 @@ func (x *RemoveRepositoryRequest) String() string {
func (*RemoveRepositoryRequest) ProtoMessage() {}
func (x *RemoveRepositoryRequest) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[69]
+ mi := &file_repository_proto_msgTypes[67]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -4176,7 +4079,7 @@ func (x *RemoveRepositoryRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use RemoveRepositoryRequest.ProtoReflect.Descriptor instead.
func (*RemoveRepositoryRequest) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{69}
+ return file_repository_proto_rawDescGZIP(), []int{67}
}
func (x *RemoveRepositoryRequest) GetRepository() *Repository {
@@ -4196,7 +4099,7 @@ type RemoveRepositoryResponse struct {
func (x *RemoveRepositoryResponse) Reset() {
*x = RemoveRepositoryResponse{}
if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[70]
+ mi := &file_repository_proto_msgTypes[68]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -4209,7 +4112,7 @@ func (x *RemoveRepositoryResponse) String() string {
func (*RemoveRepositoryResponse) ProtoMessage() {}
func (x *RemoveRepositoryResponse) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[70]
+ mi := &file_repository_proto_msgTypes[68]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -4222,7 +4125,7 @@ func (x *RemoveRepositoryResponse) ProtoReflect() protoreflect.Message {
// Deprecated: Use RemoveRepositoryResponse.ProtoReflect.Descriptor instead.
func (*RemoveRepositoryResponse) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{70}
+ return file_repository_proto_rawDescGZIP(), []int{68}
}
// ReplicateRepositoryRequest is a request for the ReplicateRepository RPC.
@@ -4249,7 +4152,7 @@ type ReplicateRepositoryRequest struct {
func (x *ReplicateRepositoryRequest) Reset() {
*x = ReplicateRepositoryRequest{}
if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[71]
+ mi := &file_repository_proto_msgTypes[69]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -4262,7 +4165,7 @@ func (x *ReplicateRepositoryRequest) String() string {
func (*ReplicateRepositoryRequest) ProtoMessage() {}
func (x *ReplicateRepositoryRequest) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[71]
+ mi := &file_repository_proto_msgTypes[69]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -4275,7 +4178,7 @@ func (x *ReplicateRepositoryRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use ReplicateRepositoryRequest.ProtoReflect.Descriptor instead.
func (*ReplicateRepositoryRequest) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{71}
+ return file_repository_proto_rawDescGZIP(), []int{69}
}
func (x *ReplicateRepositoryRequest) GetRepository() *Repository {
@@ -4309,7 +4212,7 @@ type ReplicateRepositoryResponse struct {
func (x *ReplicateRepositoryResponse) Reset() {
*x = ReplicateRepositoryResponse{}
if protoimpl.UnsafeEnabled {
- mi := &file_repository_proto_msgTypes[72]
+ mi := &file_repository_proto_msgTypes[70]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -4322,7 +4225,7 @@ func (x *ReplicateRepositoryResponse) String() string {
func (*ReplicateRepositoryResponse) ProtoMessage() {}
func (x *ReplicateRepositoryResponse) ProtoReflect() protoreflect.Message {
- mi := &file_repository_proto_msgTypes[72]
+ mi := &file_repository_proto_msgTypes[70]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -4335,7 +4238,7 @@ func (x *ReplicateRepositoryResponse) ProtoReflect() protoreflect.Message {
// Deprecated: Use ReplicateRepositoryResponse.ProtoReflect.Descriptor instead.
func (*ReplicateRepositoryResponse) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{72}
+ return file_repository_proto_rawDescGZIP(), []int{70}
}
// OptimizeRepositoryRequest is a request for the OptimizeRepository RPC.
@@ -4353,7 +4256,7 @@ type OptimizeRepositoryRequest struct {
func (x *OptimizeRepositoryRequest) Reset() {
*x = OptimizeRepositoryRequest{}
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)
}
@@ -4366,7 +4269,7 @@ func (x *OptimizeRepositoryRequest) String() string {
func (*OptimizeRepositoryRequest) ProtoMessage() {}
func (x *OptimizeRepositoryRequest) 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 {
@@ -4379,7 +4282,7 @@ func (x *OptimizeRepositoryRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use OptimizeRepositoryRequest.ProtoReflect.Descriptor instead.
func (*OptimizeRepositoryRequest) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{73}
+ return file_repository_proto_rawDescGZIP(), []int{71}
}
func (x *OptimizeRepositoryRequest) GetRepository() *Repository {
@@ -4406,7 +4309,7 @@ type OptimizeRepositoryResponse struct {
func (x *OptimizeRepositoryResponse) Reset() {
*x = OptimizeRepositoryResponse{}
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 *OptimizeRepositoryResponse) String() string {
func (*OptimizeRepositoryResponse) ProtoMessage() {}
func (x *OptimizeRepositoryResponse) 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 *OptimizeRepositoryResponse) ProtoReflect() protoreflect.Message {
// Deprecated: Use OptimizeRepositoryResponse.ProtoReflect.Descriptor instead.
func (*OptimizeRepositoryResponse) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{74}
+ return file_repository_proto_rawDescGZIP(), []int{72}
}
// PruneUnreachableObjectsRequest is a request for the PruneUnreachableObjects
@@ -4449,7 +4352,7 @@ type PruneUnreachableObjectsRequest struct {
func (x *PruneUnreachableObjectsRequest) Reset() {
*x = PruneUnreachableObjectsRequest{}
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)
}
@@ -4462,7 +4365,7 @@ func (x *PruneUnreachableObjectsRequest) String() string {
func (*PruneUnreachableObjectsRequest) ProtoMessage() {}
func (x *PruneUnreachableObjectsRequest) 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 {
@@ -4475,7 +4378,7 @@ func (x *PruneUnreachableObjectsRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use PruneUnreachableObjectsRequest.ProtoReflect.Descriptor instead.
func (*PruneUnreachableObjectsRequest) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{75}
+ return file_repository_proto_rawDescGZIP(), []int{73}
}
func (x *PruneUnreachableObjectsRequest) GetRepository() *Repository {
@@ -4496,7 +4399,7 @@ type PruneUnreachableObjectsResponse struct {
func (x *PruneUnreachableObjectsResponse) Reset() {
*x = PruneUnreachableObjectsResponse{}
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)
}
@@ -4509,7 +4412,7 @@ func (x *PruneUnreachableObjectsResponse) String() string {
func (*PruneUnreachableObjectsResponse) ProtoMessage() {}
func (x *PruneUnreachableObjectsResponse) 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 {
@@ -4522,7 +4425,7 @@ func (x *PruneUnreachableObjectsResponse) ProtoReflect() protoreflect.Message {
// Deprecated: Use PruneUnreachableObjectsResponse.ProtoReflect.Descriptor instead.
func (*PruneUnreachableObjectsResponse) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{76}
+ return file_repository_proto_rawDescGZIP(), []int{74}
}
// SetFullPathRequest is a request for the SetFullPath RPC.
@@ -4540,7 +4443,7 @@ type SetFullPathRequest struct {
func (x *SetFullPathRequest) Reset() {
*x = SetFullPathRequest{}
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)
}
@@ -4553,7 +4456,7 @@ func (x *SetFullPathRequest) String() string {
func (*SetFullPathRequest) ProtoMessage() {}
func (x *SetFullPathRequest) 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 {
@@ -4566,7 +4469,7 @@ func (x *SetFullPathRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use SetFullPathRequest.ProtoReflect.Descriptor instead.
func (*SetFullPathRequest) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{77}
+ return file_repository_proto_rawDescGZIP(), []int{75}
}
func (x *SetFullPathRequest) GetRepository() *Repository {
@@ -4593,7 +4496,7 @@ type SetFullPathResponse struct {
func (x *SetFullPathResponse) Reset() {
*x = SetFullPathResponse{}
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 *SetFullPathResponse) String() string {
func (*SetFullPathResponse) ProtoMessage() {}
func (x *SetFullPathResponse) 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 *SetFullPathResponse) ProtoReflect() protoreflect.Message {
// Deprecated: Use SetFullPathResponse.ProtoReflect.Descriptor instead.
func (*SetFullPathResponse) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{78}
+ return file_repository_proto_rawDescGZIP(), []int{76}
}
// FullPathRequest is a request for the FullPath RPC.
@@ -4635,7 +4538,7 @@ type FullPathRequest struct {
func (x *FullPathRequest) Reset() {
*x = FullPathRequest{}
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)
}
@@ -4648,7 +4551,7 @@ func (x *FullPathRequest) String() string {
func (*FullPathRequest) ProtoMessage() {}
func (x *FullPathRequest) 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 {
@@ -4661,7 +4564,7 @@ func (x *FullPathRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use FullPathRequest.ProtoReflect.Descriptor instead.
func (*FullPathRequest) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{79}
+ return file_repository_proto_rawDescGZIP(), []int{77}
}
func (x *FullPathRequest) GetRepository() *Repository {
@@ -4684,7 +4587,7 @@ type FullPathResponse struct {
func (x *FullPathResponse) Reset() {
*x = FullPathResponse{}
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)
}
@@ -4697,7 +4600,7 @@ func (x *FullPathResponse) String() string {
func (*FullPathResponse) ProtoMessage() {}
func (x *FullPathResponse) 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 {
@@ -4710,7 +4613,7 @@ func (x *FullPathResponse) ProtoReflect() protoreflect.Message {
// Deprecated: Use FullPathResponse.ProtoReflect.Descriptor instead.
func (*FullPathResponse) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{80}
+ return file_repository_proto_rawDescGZIP(), []int{78}
}
func (x *FullPathResponse) GetPath() string {
@@ -4733,7 +4636,7 @@ type RemoveAllRequest struct {
func (x *RemoveAllRequest) Reset() {
*x = RemoveAllRequest{}
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)
}
@@ -4746,7 +4649,7 @@ func (x *RemoveAllRequest) String() string {
func (*RemoveAllRequest) ProtoMessage() {}
func (x *RemoveAllRequest) 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 {
@@ -4759,7 +4662,7 @@ func (x *RemoveAllRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use RemoveAllRequest.ProtoReflect.Descriptor instead.
func (*RemoveAllRequest) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{81}
+ return file_repository_proto_rawDescGZIP(), []int{79}
}
func (x *RemoveAllRequest) GetStorageName() string {
@@ -4779,7 +4682,7 @@ type RemoveAllResponse struct {
func (x *RemoveAllResponse) Reset() {
*x = RemoveAllResponse{}
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)
}
@@ -4792,7 +4695,7 @@ func (x *RemoveAllResponse) String() string {
func (*RemoveAllResponse) ProtoMessage() {}
func (x *RemoveAllResponse) 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 {
@@ -4805,7 +4708,7 @@ func (x *RemoveAllResponse) ProtoReflect() protoreflect.Message {
// Deprecated: Use RemoveAllResponse.ProtoReflect.Descriptor instead.
func (*RemoveAllResponse) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{82}
+ return file_repository_proto_rawDescGZIP(), []int{80}
}
// BackupRepositoryRequest is a request for the BackupRepository RPC.
@@ -4828,7 +4731,7 @@ type BackupRepositoryRequest struct {
func (x *BackupRepositoryRequest) Reset() {
*x = BackupRepositoryRequest{}
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)
}
@@ -4841,7 +4744,7 @@ func (x *BackupRepositoryRequest) String() string {
func (*BackupRepositoryRequest) ProtoMessage() {}
func (x *BackupRepositoryRequest) 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 {
@@ -4854,7 +4757,7 @@ func (x *BackupRepositoryRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use BackupRepositoryRequest.ProtoReflect.Descriptor instead.
func (*BackupRepositoryRequest) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{83}
+ return file_repository_proto_rawDescGZIP(), []int{81}
}
func (x *BackupRepositoryRequest) GetRepository() *Repository {
@@ -4895,7 +4798,7 @@ type BackupRepositoryResponse struct {
func (x *BackupRepositoryResponse) Reset() {
*x = BackupRepositoryResponse{}
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)
}
@@ -4908,7 +4811,7 @@ func (x *BackupRepositoryResponse) String() string {
func (*BackupRepositoryResponse) ProtoMessage() {}
func (x *BackupRepositoryResponse) 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 {
@@ -4921,7 +4824,7 @@ func (x *BackupRepositoryResponse) ProtoReflect() protoreflect.Message {
// Deprecated: Use BackupRepositoryResponse.ProtoReflect.Descriptor instead.
func (*BackupRepositoryResponse) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{84}
+ return file_repository_proto_rawDescGZIP(), []int{82}
}
// RestoreRepositoryRequest is a request for the RestoreRepository RPC.
@@ -4945,7 +4848,7 @@ type RestoreRepositoryRequest struct {
func (x *RestoreRepositoryRequest) Reset() {
*x = RestoreRepositoryRequest{}
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)
}
@@ -4958,7 +4861,7 @@ func (x *RestoreRepositoryRequest) String() string {
func (*RestoreRepositoryRequest) ProtoMessage() {}
func (x *RestoreRepositoryRequest) 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 {
@@ -4971,7 +4874,7 @@ func (x *RestoreRepositoryRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use RestoreRepositoryRequest.ProtoReflect.Descriptor instead.
func (*RestoreRepositoryRequest) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{85}
+ return file_repository_proto_rawDescGZIP(), []int{83}
}
func (x *RestoreRepositoryRequest) GetRepository() *Repository {
@@ -5012,7 +4915,7 @@ type RestoreRepositoryResponse struct {
func (x *RestoreRepositoryResponse) Reset() {
*x = RestoreRepositoryResponse{}
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)
}
@@ -5025,7 +4928,7 @@ func (x *RestoreRepositoryResponse) String() string {
func (*RestoreRepositoryResponse) ProtoMessage() {}
func (x *RestoreRepositoryResponse) 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 {
@@ -5038,7 +4941,7 @@ func (x *RestoreRepositoryResponse) ProtoReflect() protoreflect.Message {
// Deprecated: Use RestoreRepositoryResponse.ProtoReflect.Descriptor instead.
func (*RestoreRepositoryResponse) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{86}
+ return file_repository_proto_rawDescGZIP(), []int{84}
}
// GetFileAttributesRequest is a request for the GetFileAttributes RPC.
@@ -5060,7 +4963,7 @@ type GetFileAttributesRequest struct {
func (x *GetFileAttributesRequest) Reset() {
*x = GetFileAttributesRequest{}
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)
}
@@ -5073,7 +4976,7 @@ func (x *GetFileAttributesRequest) String() string {
func (*GetFileAttributesRequest) ProtoMessage() {}
func (x *GetFileAttributesRequest) 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 {
@@ -5086,7 +4989,7 @@ func (x *GetFileAttributesRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use GetFileAttributesRequest.ProtoReflect.Descriptor instead.
func (*GetFileAttributesRequest) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{87}
+ return file_repository_proto_rawDescGZIP(), []int{85}
}
func (x *GetFileAttributesRequest) GetRepository() *Repository {
@@ -5130,7 +5033,7 @@ type GetFileAttributesResponse struct {
func (x *GetFileAttributesResponse) Reset() {
*x = GetFileAttributesResponse{}
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)
}
@@ -5143,7 +5046,7 @@ func (x *GetFileAttributesResponse) String() string {
func (*GetFileAttributesResponse) ProtoMessage() {}
func (x *GetFileAttributesResponse) 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 {
@@ -5156,7 +5059,7 @@ func (x *GetFileAttributesResponse) ProtoReflect() protoreflect.Message {
// Deprecated: Use GetFileAttributesResponse.ProtoReflect.Descriptor instead.
func (*GetFileAttributesResponse) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{88}
+ return file_repository_proto_rawDescGZIP(), []int{86}
}
func (x *GetFileAttributesResponse) GetAttributeInfos() []*GetFileAttributesResponse_AttributeInfo {
@@ -5190,7 +5093,7 @@ type RepositoryInfoResponse_ReferencesInfo struct {
func (x *RepositoryInfoResponse_ReferencesInfo) Reset() {
*x = RepositoryInfoResponse_ReferencesInfo{}
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)
}
@@ -5203,7 +5106,7 @@ func (x *RepositoryInfoResponse_ReferencesInfo) String() string {
func (*RepositoryInfoResponse_ReferencesInfo) ProtoMessage() {}
func (x *RepositoryInfoResponse_ReferencesInfo) 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 {
@@ -5269,7 +5172,7 @@ type RepositoryInfoResponse_ObjectsInfo struct {
func (x *RepositoryInfoResponse_ObjectsInfo) Reset() {
*x = RepositoryInfoResponse_ObjectsInfo{}
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)
}
@@ -5282,7 +5185,7 @@ func (x *RepositoryInfoResponse_ObjectsInfo) String() string {
func (*RepositoryInfoResponse_ObjectsInfo) ProtoMessage() {}
func (x *RepositoryInfoResponse_ObjectsInfo) 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 {
@@ -5354,7 +5257,7 @@ type GetRawChangesResponse_RawChange struct {
func (x *GetRawChangesResponse_RawChange) Reset() {
*x = GetRawChangesResponse_RawChange{}
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)
}
@@ -5367,7 +5270,7 @@ func (x *GetRawChangesResponse_RawChange) String() string {
func (*GetRawChangesResponse_RawChange) ProtoMessage() {}
func (x *GetRawChangesResponse_RawChange) 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 {
@@ -5380,7 +5283,7 @@ func (x *GetRawChangesResponse_RawChange) ProtoReflect() protoreflect.Message {
// Deprecated: Use GetRawChangesResponse_RawChange.ProtoReflect.Descriptor instead.
func (*GetRawChangesResponse_RawChange) Descriptor() ([]byte, []int) {
- return file_repository_proto_rawDescGZIP(), []int{61, 0}
+ return file_repository_proto_rawDescGZIP(), []int{59, 0}
}
func (x *GetRawChangesResponse_RawChange) GetBlobId() string {
@@ -5449,7 +5352,7 @@ type BackupRepositoryResponse_SkippedError struct {
func (x *BackupRepositoryResponse_SkippedError) Reset() {
*x = BackupRepositoryResponse_SkippedError{}
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)
}
@@ -5462,7 +5365,7 @@ func (x *BackupRepositoryResponse_SkippedError) String() string {
func (*BackupRepositoryResponse_SkippedError) ProtoMessage() {}
func (x *BackupRepositoryResponse_SkippedError) 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 {
@@ -5475,7 +5378,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{84, 0}
+ return file_repository_proto_rawDescGZIP(), []int{82, 0}
}
// SkippedError is returned when the repository restore has been skipped.
@@ -5488,7 +5391,7 @@ type RestoreRepositoryResponse_SkippedError struct {
func (x *RestoreRepositoryResponse_SkippedError) Reset() {
*x = RestoreRepositoryResponse_SkippedError{}
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)
}
@@ -5501,7 +5404,7 @@ func (x *RestoreRepositoryResponse_SkippedError) String() string {
func (*RestoreRepositoryResponse_SkippedError) ProtoMessage() {}
func (x *RestoreRepositoryResponse_SkippedError) 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 {
@@ -5514,7 +5417,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{86, 0}
+ return file_repository_proto_rawDescGZIP(), []int{84, 0}
}
// AttributeInfo is a specified git attribute for a given path.
@@ -5536,7 +5439,7 @@ type GetFileAttributesResponse_AttributeInfo struct {
func (x *GetFileAttributesResponse_AttributeInfo) Reset() {
*x = GetFileAttributesResponse_AttributeInfo{}
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)
}
@@ -5549,7 +5452,7 @@ func (x *GetFileAttributesResponse_AttributeInfo) String() string {
func (*GetFileAttributesResponse_AttributeInfo) ProtoMessage() {}
func (x *GetFileAttributesResponse_AttributeInfo) 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 {
@@ -5562,7 +5465,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{88, 0}
+ return file_repository_proto_rawDescGZIP(), []int{86, 0}
}
func (x *GetFileAttributesResponse_AttributeInfo) GetPath() string {
@@ -5658,793 +5561,778 @@ var file_repository_proto_rawDesc = []byte{
0x6d, 0x61, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2c, 0x0a, 0x06, 0x66,
0x6f, 0x72, 0x6d, 0x61, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x14, 0x2e, 0x67, 0x69,
0x74, 0x61, 0x6c, 0x79, 0x2e, 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x46, 0x6f, 0x72, 0x6d, 0x61,
- 0x74, 0x52, 0x06, 0x66, 0x6f, 0x72, 0x6d, 0x61, 0x74, 0x22, 0x71, 0x0a, 0x19, 0x41, 0x70, 0x70,
- 0x6c, 0x79, 0x47, 0x69, 0x74, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x52,
+ 0x74, 0x52, 0x06, 0x66, 0x6f, 0x72, 0x6d, 0x61, 0x74, 0x22, 0x83, 0x01, 0x0a, 0x12, 0x46, 0x65,
+ 0x74, 0x63, 0x68, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 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, 0x64, 0x61,
+ 0x74, 0x61, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x04, 0x64, 0x61, 0x74, 0x61, 0x12, 0x1f,
+ 0x0a, 0x0b, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x5f, 0x68, 0x65, 0x61, 0x64, 0x18, 0x03, 0x20,
+ 0x01, 0x28, 0x08, 0x52, 0x0a, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x48, 0x65, 0x61, 0x64, 0x22,
+ 0x15, 0x0a, 0x13, 0x46, 0x65, 0x74, 0x63, 0x68, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x52, 0x65,
+ 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xe3, 0x02, 0x0a, 0x12, 0x46, 0x65, 0x74, 0x63, 0x68,
+ 0x52, 0x65, 0x6d, 0x6f, 0x74, 0x65, 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, 0x14, 0x0a, 0x05, 0x66, 0x6f, 0x72, 0x63, 0x65,
+ 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x05, 0x66, 0x6f, 0x72, 0x63, 0x65, 0x12, 0x17, 0x0a,
+ 0x07, 0x6e, 0x6f, 0x5f, 0x74, 0x61, 0x67, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x06,
+ 0x6e, 0x6f, 0x54, 0x61, 0x67, 0x73, 0x12, 0x18, 0x0a, 0x07, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75,
+ 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x05, 0x52, 0x07, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74,
+ 0x12, 0x17, 0x0a, 0x07, 0x73, 0x73, 0x68, 0x5f, 0x6b, 0x65, 0x79, 0x18, 0x06, 0x20, 0x01, 0x28,
+ 0x09, 0x52, 0x06, 0x73, 0x73, 0x68, 0x4b, 0x65, 0x79, 0x12, 0x1f, 0x0a, 0x0b, 0x6b, 0x6e, 0x6f,
+ 0x77, 0x6e, 0x5f, 0x68, 0x6f, 0x73, 0x74, 0x73, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a,
+ 0x6b, 0x6e, 0x6f, 0x77, 0x6e, 0x48, 0x6f, 0x73, 0x74, 0x73, 0x12, 0x19, 0x0a, 0x08, 0x6e, 0x6f,
+ 0x5f, 0x70, 0x72, 0x75, 0x6e, 0x65, 0x18, 0x09, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x6e, 0x6f,
+ 0x50, 0x72, 0x75, 0x6e, 0x65, 0x12, 0x33, 0x0a, 0x0d, 0x72, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x5f,
+ 0x70, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0e, 0x2e, 0x67,
+ 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x52, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x52, 0x0c, 0x72, 0x65,
+ 0x6d, 0x6f, 0x74, 0x65, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x12, 0x2c, 0x0a, 0x12, 0x63, 0x68,
+ 0x65, 0x63, 0x6b, 0x5f, 0x74, 0x61, 0x67, 0x73, 0x5f, 0x63, 0x68, 0x61, 0x6e, 0x67, 0x65, 0x64,
+ 0x18, 0x0b, 0x20, 0x01, 0x28, 0x08, 0x52, 0x10, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x54, 0x61, 0x67,
+ 0x73, 0x43, 0x68, 0x61, 0x6e, 0x67, 0x65, 0x64, 0x4a, 0x04, 0x08, 0x08, 0x10, 0x09, 0x4a, 0x04,
+ 0x08, 0x02, 0x10, 0x03, 0x52, 0x06, 0x72, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x22, 0x38, 0x0a, 0x13,
+ 0x46, 0x65, 0x74, 0x63, 0x68, 0x52, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f,
+ 0x6e, 0x73, 0x65, 0x12, 0x21, 0x0a, 0x0c, 0x74, 0x61, 0x67, 0x73, 0x5f, 0x63, 0x68, 0x61, 0x6e,
+ 0x67, 0x65, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0b, 0x74, 0x61, 0x67, 0x73, 0x43,
+ 0x68, 0x61, 0x6e, 0x67, 0x65, 0x64, 0x22, 0xb5, 0x01, 0x0a, 0x17, 0x43, 0x72, 0x65, 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, 0x25, 0x0a, 0x0e,
+ 0x64, 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, 0x5f, 0x62, 0x72, 0x61, 0x6e, 0x63, 0x68, 0x18, 0x02,
+ 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0d, 0x64, 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, 0x42, 0x72, 0x61,
+ 0x6e, 0x63, 0x68, 0x12, 0x39, 0x0a, 0x0d, 0x6f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x5f, 0x66, 0x6f,
+ 0x72, 0x6d, 0x61, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x14, 0x2e, 0x67, 0x69, 0x74,
+ 0x61, 0x6c, 0x79, 0x2e, 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x46, 0x6f, 0x72, 0x6d, 0x61, 0x74,
+ 0x52, 0x0c, 0x6f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x46, 0x6f, 0x72, 0x6d, 0x61, 0x74, 0x22, 0x1a,
+ 0x0a, 0x18, 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, 0xea, 0x02, 0x0a, 0x11, 0x47,
+ 0x65, 0x74, 0x41, 0x72, 0x63, 0x68, 0x69, 0x76, 0x65, 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, 0x1b, 0x0a, 0x09, 0x63, 0x6f,
+ 0x6d, 0x6d, 0x69, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x63,
+ 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x49, 0x64, 0x12, 0x16, 0x0a, 0x06, 0x70, 0x72, 0x65, 0x66, 0x69,
+ 0x78, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x70, 0x72, 0x65, 0x66, 0x69, 0x78, 0x12,
+ 0x38, 0x0a, 0x06, 0x66, 0x6f, 0x72, 0x6d, 0x61, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32,
+ 0x20, 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, 0x2e, 0x46, 0x6f, 0x72, 0x6d, 0x61,
+ 0x74, 0x52, 0x06, 0x66, 0x6f, 0x72, 0x6d, 0x61, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x70, 0x61, 0x74,
+ 0x68, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x04, 0x70, 0x61, 0x74, 0x68, 0x12, 0x18, 0x0a,
+ 0x07, 0x65, 0x78, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x18, 0x06, 0x20, 0x03, 0x28, 0x0c, 0x52, 0x07,
+ 0x65, 0x78, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x65, 0x6c, 0x69, 0x64, 0x65,
+ 0x5f, 0x70, 0x61, 0x74, 0x68, 0x18, 0x07, 0x20, 0x01, 0x28, 0x08, 0x52, 0x09, 0x65, 0x6c, 0x69,
+ 0x64, 0x65, 0x50, 0x61, 0x74, 0x68, 0x12, 0x2a, 0x0a, 0x11, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64,
+ 0x65, 0x5f, 0x6c, 0x66, 0x73, 0x5f, 0x62, 0x6c, 0x6f, 0x62, 0x73, 0x18, 0x08, 0x20, 0x01, 0x28,
+ 0x08, 0x52, 0x0f, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x4c, 0x66, 0x73, 0x42, 0x6c, 0x6f,
+ 0x62, 0x73, 0x22, 0x33, 0x0a, 0x06, 0x46, 0x6f, 0x72, 0x6d, 0x61, 0x74, 0x12, 0x07, 0x0a, 0x03,
+ 0x5a, 0x49, 0x50, 0x10, 0x00, 0x12, 0x07, 0x0a, 0x03, 0x54, 0x41, 0x52, 0x10, 0x01, 0x12, 0x0a,
+ 0x0a, 0x06, 0x54, 0x41, 0x52, 0x5f, 0x47, 0x5a, 0x10, 0x02, 0x12, 0x0b, 0x0a, 0x07, 0x54, 0x41,
+ 0x52, 0x5f, 0x42, 0x5a, 0x32, 0x10, 0x03, 0x22, 0x28, 0x0a, 0x12, 0x47, 0x65, 0x74, 0x41, 0x72,
+ 0x63, 0x68, 0x69, 0x76, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x12, 0x0a,
+ 0x04, 0x64, 0x61, 0x74, 0x61, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x04, 0x64, 0x61, 0x74,
+ 0x61, 0x22, 0x53, 0x0a, 0x17, 0x48, 0x61, 0x73, 0x4c, 0x6f, 0x63, 0x61, 0x6c, 0x42, 0x72, 0x61,
+ 0x6e, 0x63, 0x68, 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, 0x22, 0x30, 0x0a, 0x18, 0x48, 0x61, 0x73, 0x4c, 0x6f, 0x63,
+ 0x61, 0x6c, 0x42, 0x72, 0x61, 0x6e, 0x63, 0x68, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e,
+ 0x73, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28,
+ 0x08, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x22, 0xd9, 0x01, 0x0a, 0x18, 0x46, 0x65, 0x74,
+ 0x63, 0x68, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x42, 0x72, 0x61, 0x6e, 0x63, 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,
+ 0x3f, 0x0a, 0x11, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 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,
+ 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79,
+ 0x12, 0x23, 0x0a, 0x0d, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x62, 0x72, 0x61, 0x6e, 0x63,
+ 0x68, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0c, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x42,
+ 0x72, 0x61, 0x6e, 0x63, 0x68, 0x12, 0x1d, 0x0a, 0x0a, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x5f,
+ 0x72, 0x65, 0x66, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x09, 0x74, 0x61, 0x72, 0x67, 0x65,
+ 0x74, 0x52, 0x65, 0x66, 0x22, 0x33, 0x0a, 0x19, 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, 0x12, 0x16, 0x0a, 0x06, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28,
+ 0x08, 0x52, 0x06, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x22, 0x47, 0x0a, 0x0b, 0x46, 0x73, 0x63,
+ 0x6b, 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, 0x24, 0x0a, 0x0c, 0x46, 0x73, 0x63, 0x6b, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e,
+ 0x73, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28,
+ 0x0c, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x22, 0xb8, 0x01, 0x0a, 0x0f, 0x57, 0x72, 0x69,
+ 0x74, 0x65, 0x52, 0x65, 0x66, 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, 0x10, 0x0a, 0x03, 0x72, 0x65, 0x66, 0x18, 0x02, 0x20,
+ 0x01, 0x28, 0x0c, 0x52, 0x03, 0x72, 0x65, 0x66, 0x12, 0x1a, 0x0a, 0x08, 0x72, 0x65, 0x76, 0x69,
+ 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x08, 0x72, 0x65, 0x76, 0x69,
+ 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x21, 0x0a, 0x0c, 0x6f, 0x6c, 0x64, 0x5f, 0x72, 0x65, 0x76, 0x69,
+ 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0b, 0x6f, 0x6c, 0x64, 0x52,
+ 0x65, 0x76, 0x69, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x14, 0x0a, 0x05, 0x66, 0x6f, 0x72, 0x63, 0x65,
+ 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, 0x05, 0x66, 0x6f, 0x72, 0x63, 0x65, 0x4a, 0x04, 0x08,
+ 0x06, 0x10, 0x07, 0x22, 0x18, 0x0a, 0x10, 0x57, 0x72, 0x69, 0x74, 0x65, 0x52, 0x65, 0x66, 0x52,
+ 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x4a, 0x04, 0x08, 0x01, 0x10, 0x02, 0x22, 0x6e, 0x0a,
+ 0x14, 0x46, 0x69, 0x6e, 0x64, 0x4d, 0x65, 0x72, 0x67, 0x65, 0x42, 0x61, 0x73, 0x65, 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,
+ 0x1c, 0x0a, 0x09, 0x72, 0x65, 0x76, 0x69, 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x02, 0x20, 0x03,
+ 0x28, 0x0c, 0x52, 0x09, 0x72, 0x65, 0x76, 0x69, 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x22, 0x2b, 0x0a,
+ 0x15, 0x46, 0x69, 0x6e, 0x64, 0x4d, 0x65, 0x72, 0x67, 0x65, 0x42, 0x61, 0x73, 0x65, 0x52, 0x65,
+ 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x62, 0x61, 0x73, 0x65, 0x18, 0x01,
+ 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x62, 0x61, 0x73, 0x65, 0x22, 0xaa, 0x01, 0x0a, 0x11, 0x43,
+ 0x72, 0x65, 0x61, 0x74, 0x65, 0x46, 0x6f, 0x72, 0x6b, 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, 0x73, 0x6f,
+ 0x75, 0x72, 0x63, 0x65, 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, 0x73, 0x6f, 0x75, 0x72, 0x63,
+ 0x65, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x12, 0x1a, 0x0a, 0x08, 0x72,
+ 0x65, 0x76, 0x69, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x08, 0x72,
+ 0x65, 0x76, 0x69, 0x73, 0x69, 0x6f, 0x6e, 0x22, 0x14, 0x0a, 0x12, 0x43, 0x72, 0x65, 0x61, 0x74,
+ 0x65, 0x46, 0x6f, 0x72, 0x6b, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xfc, 0x01,
+ 0x0a, 0x1e, 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,
+ 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, 0x10, 0x0a, 0x03, 0x75, 0x72,
+ 0x6c, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x75, 0x72, 0x6c, 0x12, 0x3a, 0x0a, 0x19,
+ 0x68, 0x74, 0x74, 0x70, 0x5f, 0x61, 0x75, 0x74, 0x68, 0x6f, 0x72, 0x69, 0x7a, 0x61, 0x74, 0x69,
+ 0x6f, 0x6e, 0x5f, 0x68, 0x65, 0x61, 0x64, 0x65, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52,
+ 0x17, 0x68, 0x74, 0x74, 0x70, 0x41, 0x75, 0x74, 0x68, 0x6f, 0x72, 0x69, 0x7a, 0x61, 0x74, 0x69,
+ 0x6f, 0x6e, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x16, 0x0a, 0x06, 0x6d, 0x69, 0x72, 0x72,
+ 0x6f, 0x72, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, 0x06, 0x6d, 0x69, 0x72, 0x72, 0x6f, 0x72,
+ 0x12, 0x29, 0x0a, 0x10, 0x72, 0x65, 0x73, 0x6f, 0x6c, 0x76, 0x65, 0x64, 0x5f, 0x61, 0x64, 0x64,
+ 0x72, 0x65, 0x73, 0x73, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x72, 0x65, 0x73, 0x6f,
+ 0x6c, 0x76, 0x65, 0x64, 0x41, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x4a, 0x04, 0x08, 0x03, 0x10,
+ 0x04, 0x52, 0x09, 0x68, 0x74, 0x74, 0x70, 0x5f, 0x68, 0x6f, 0x73, 0x74, 0x22, 0x21, 0x0a, 0x1f,
+ 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,
+ 0x4f, 0x0a, 0x13, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 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, 0x22, 0x1c, 0x0a, 0x1a,
- 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, 0x83, 0x01, 0x0a, 0x12, 0x46,
- 0x65, 0x74, 0x63, 0x68, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 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, 0x64,
- 0x61, 0x74, 0x61, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x04, 0x64, 0x61, 0x74, 0x61, 0x12,
- 0x1f, 0x0a, 0x0b, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x5f, 0x68, 0x65, 0x61, 0x64, 0x18, 0x03,
- 0x20, 0x01, 0x28, 0x08, 0x52, 0x0a, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x48, 0x65, 0x61, 0x64,
- 0x22, 0x15, 0x0a, 0x13, 0x46, 0x65, 0x74, 0x63, 0x68, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x52,
- 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xe3, 0x02, 0x0a, 0x12, 0x46, 0x65, 0x74, 0x63,
- 0x68, 0x52, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x38,
+ 0x22, 0x2a, 0x0a, 0x14, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65,
+ 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x64, 0x61, 0x74, 0x61,
+ 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x04, 0x64, 0x61, 0x74, 0x61, 0x22, 0x76, 0x0a, 0x1e,
+ 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, 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, 0x14, 0x0a, 0x05, 0x66, 0x6f, 0x72, 0x63,
- 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x05, 0x66, 0x6f, 0x72, 0x63, 0x65, 0x12, 0x17,
- 0x0a, 0x07, 0x6e, 0x6f, 0x5f, 0x74, 0x61, 0x67, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52,
- 0x06, 0x6e, 0x6f, 0x54, 0x61, 0x67, 0x73, 0x12, 0x18, 0x0a, 0x07, 0x74, 0x69, 0x6d, 0x65, 0x6f,
- 0x75, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x05, 0x52, 0x07, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75,
- 0x74, 0x12, 0x17, 0x0a, 0x07, 0x73, 0x73, 0x68, 0x5f, 0x6b, 0x65, 0x79, 0x18, 0x06, 0x20, 0x01,
- 0x28, 0x09, 0x52, 0x06, 0x73, 0x73, 0x68, 0x4b, 0x65, 0x79, 0x12, 0x1f, 0x0a, 0x0b, 0x6b, 0x6e,
- 0x6f, 0x77, 0x6e, 0x5f, 0x68, 0x6f, 0x73, 0x74, 0x73, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52,
- 0x0a, 0x6b, 0x6e, 0x6f, 0x77, 0x6e, 0x48, 0x6f, 0x73, 0x74, 0x73, 0x12, 0x19, 0x0a, 0x08, 0x6e,
- 0x6f, 0x5f, 0x70, 0x72, 0x75, 0x6e, 0x65, 0x18, 0x09, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x6e,
- 0x6f, 0x50, 0x72, 0x75, 0x6e, 0x65, 0x12, 0x33, 0x0a, 0x0d, 0x72, 0x65, 0x6d, 0x6f, 0x74, 0x65,
- 0x5f, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0e, 0x2e,
- 0x67, 0x69, 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x52, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x52, 0x0c, 0x72,
- 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x12, 0x2c, 0x0a, 0x12, 0x63,
- 0x68, 0x65, 0x63, 0x6b, 0x5f, 0x74, 0x61, 0x67, 0x73, 0x5f, 0x63, 0x68, 0x61, 0x6e, 0x67, 0x65,
- 0x64, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x08, 0x52, 0x10, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x54, 0x61,
- 0x67, 0x73, 0x43, 0x68, 0x61, 0x6e, 0x67, 0x65, 0x64, 0x4a, 0x04, 0x08, 0x08, 0x10, 0x09, 0x4a,
- 0x04, 0x08, 0x02, 0x10, 0x03, 0x52, 0x06, 0x72, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x22, 0x38, 0x0a,
- 0x13, 0x46, 0x65, 0x74, 0x63, 0x68, 0x52, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70,
- 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x21, 0x0a, 0x0c, 0x74, 0x61, 0x67, 0x73, 0x5f, 0x63, 0x68, 0x61,
- 0x6e, 0x67, 0x65, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0b, 0x74, 0x61, 0x67, 0x73,
- 0x43, 0x68, 0x61, 0x6e, 0x67, 0x65, 0x64, 0x22, 0xb5, 0x01, 0x0a, 0x17, 0x43, 0x72, 0x65, 0x61,
- 0x74, 0x65, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x52, 0x65, 0x71, 0x75,
+ 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x12, 0x1a, 0x0a, 0x08, 0x70, 0x61, 0x74, 0x74,
+ 0x65, 0x72, 0x6e, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0c, 0x52, 0x08, 0x70, 0x61, 0x74, 0x74,
+ 0x65, 0x72, 0x6e, 0x73, 0x22, 0x35, 0x0a, 0x1f, 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, 0x12, 0x12, 0x0a, 0x04, 0x64, 0x61, 0x74, 0x61, 0x18,
+ 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x04, 0x64, 0x61, 0x74, 0x61, 0x22, 0x4c, 0x0a, 0x10, 0x47,
+ 0x65, 0x74, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 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, 0x27, 0x0a, 0x11, 0x47, 0x65, 0x74,
+ 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x12,
+ 0x0a, 0x04, 0x64, 0x61, 0x74, 0x61, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x04, 0x64, 0x61,
+ 0x74, 0x61, 0x22, 0x69, 0x0a, 0x19, 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, 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, 0x64, 0x61, 0x74,
+ 0x61, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x04, 0x64, 0x61, 0x74, 0x61, 0x22, 0x65, 0x0a,
+ 0x15, 0x53, 0x65, 0x74, 0x43, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x48, 0x6f, 0x6f, 0x6b, 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, 0x12, 0x0a, 0x04, 0x64, 0x61, 0x74, 0x61, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x04,
+ 0x64, 0x61, 0x74, 0x61, 0x22, 0x1c, 0x0a, 0x1a, 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, 0x18, 0x0a, 0x16, 0x53, 0x65, 0x74, 0x43, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x48,
+ 0x6f, 0x6f, 0x6b, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x54, 0x0a, 0x18,
+ 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x43, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x48, 0x6f, 0x6f, 0x6b,
+ 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, 0x51, 0x0a, 0x15, 0x47, 0x65, 0x74, 0x43, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x48,
+ 0x6f, 0x6f, 0x6b, 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, 0x2f, 0x0a, 0x19, 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, 0x12, 0x12, 0x0a, 0x04, 0x64, 0x61, 0x74, 0x61, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c,
+ 0x52, 0x04, 0x64, 0x61, 0x74, 0x61, 0x22, 0x2c, 0x0a, 0x16, 0x47, 0x65, 0x74, 0x43, 0x75, 0x73,
+ 0x74, 0x6f, 0x6d, 0x48, 0x6f, 0x6f, 0x6b, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65,
+ 0x12, 0x12, 0x0a, 0x04, 0x64, 0x61, 0x74, 0x61, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x04,
+ 0x64, 0x61, 0x74, 0x61, 0x22, 0x71, 0x0a, 0x21, 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, 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, 0x64, 0x61, 0x74, 0x61, 0x18, 0x02, 0x20, 0x01, 0x28,
+ 0x0c, 0x52, 0x04, 0x64, 0x61, 0x74, 0x61, 0x22, 0x24, 0x0a, 0x22, 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, 0x4e, 0x0a,
+ 0x12, 0x46, 0x69, 0x6e, 0x64, 0x4c, 0x69, 0x63, 0x65, 0x6e, 0x73, 0x65, 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, 0x25, 0x0a,
- 0x0e, 0x64, 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, 0x5f, 0x62, 0x72, 0x61, 0x6e, 0x63, 0x68, 0x18,
- 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0d, 0x64, 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, 0x42, 0x72,
- 0x61, 0x6e, 0x63, 0x68, 0x12, 0x39, 0x0a, 0x0d, 0x6f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x5f, 0x66,
- 0x6f, 0x72, 0x6d, 0x61, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x14, 0x2e, 0x67, 0x69,
- 0x74, 0x61, 0x6c, 0x79, 0x2e, 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x46, 0x6f, 0x72, 0x6d, 0x61,
- 0x74, 0x52, 0x0c, 0x6f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x46, 0x6f, 0x72, 0x6d, 0x61, 0x74, 0x22,
- 0x1a, 0x0a, 0x18, 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, 0xea, 0x02, 0x0a, 0x11,
- 0x47, 0x65, 0x74, 0x41, 0x72, 0x63, 0x68, 0x69, 0x76, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73,
+ 0x01, 0x52, 0x0a, 0x72, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x22, 0xd5, 0x01,
+ 0x0a, 0x13, 0x46, 0x69, 0x6e, 0x64, 0x4c, 0x69, 0x63, 0x65, 0x6e, 0x73, 0x65, 0x52, 0x65, 0x73,
+ 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2c, 0x0a, 0x12, 0x6c, 0x69, 0x63, 0x65, 0x6e, 0x73, 0x65,
+ 0x5f, 0x73, 0x68, 0x6f, 0x72, 0x74, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28,
+ 0x09, 0x52, 0x10, 0x6c, 0x69, 0x63, 0x65, 0x6e, 0x73, 0x65, 0x53, 0x68, 0x6f, 0x72, 0x74, 0x4e,
+ 0x61, 0x6d, 0x65, 0x12, 0x21, 0x0a, 0x0c, 0x6c, 0x69, 0x63, 0x65, 0x6e, 0x73, 0x65, 0x5f, 0x6e,
+ 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x6c, 0x69, 0x63, 0x65, 0x6e,
+ 0x73, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x6c, 0x69, 0x63, 0x65, 0x6e, 0x73,
+ 0x65, 0x5f, 0x75, 0x72, 0x6c, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x6c, 0x69, 0x63,
+ 0x65, 0x6e, 0x73, 0x65, 0x55, 0x72, 0x6c, 0x12, 0x21, 0x0a, 0x0c, 0x6c, 0x69, 0x63, 0x65, 0x6e,
+ 0x73, 0x65, 0x5f, 0x70, 0x61, 0x74, 0x68, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x6c,
+ 0x69, 0x63, 0x65, 0x6e, 0x73, 0x65, 0x50, 0x61, 0x74, 0x68, 0x12, 0x29, 0x0a, 0x10, 0x6c, 0x69,
+ 0x63, 0x65, 0x6e, 0x73, 0x65, 0x5f, 0x6e, 0x69, 0x63, 0x6b, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x05,
+ 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x6c, 0x69, 0x63, 0x65, 0x6e, 0x73, 0x65, 0x4e, 0x69, 0x63,
+ 0x6b, 0x6e, 0x61, 0x6d, 0x65, 0x22, 0x54, 0x0a, 0x18, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x66, 0x6f,
+ 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, 0x1b, 0x0a, 0x09, 0x63,
- 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08,
- 0x63, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x49, 0x64, 0x12, 0x16, 0x0a, 0x06, 0x70, 0x72, 0x65, 0x66,
- 0x69, 0x78, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x70, 0x72, 0x65, 0x66, 0x69, 0x78,
- 0x12, 0x38, 0x0a, 0x06, 0x66, 0x6f, 0x72, 0x6d, 0x61, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e,
- 0x32, 0x20, 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, 0x2e, 0x46, 0x6f, 0x72, 0x6d,
- 0x61, 0x74, 0x52, 0x06, 0x66, 0x6f, 0x72, 0x6d, 0x61, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x70, 0x61,
- 0x74, 0x68, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x04, 0x70, 0x61, 0x74, 0x68, 0x12, 0x18,
- 0x0a, 0x07, 0x65, 0x78, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x18, 0x06, 0x20, 0x03, 0x28, 0x0c, 0x52,
- 0x07, 0x65, 0x78, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x65, 0x6c, 0x69, 0x64,
- 0x65, 0x5f, 0x70, 0x61, 0x74, 0x68, 0x18, 0x07, 0x20, 0x01, 0x28, 0x08, 0x52, 0x09, 0x65, 0x6c,
- 0x69, 0x64, 0x65, 0x50, 0x61, 0x74, 0x68, 0x12, 0x2a, 0x0a, 0x11, 0x69, 0x6e, 0x63, 0x6c, 0x75,
- 0x64, 0x65, 0x5f, 0x6c, 0x66, 0x73, 0x5f, 0x62, 0x6c, 0x6f, 0x62, 0x73, 0x18, 0x08, 0x20, 0x01,
- 0x28, 0x08, 0x52, 0x0f, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x4c, 0x66, 0x73, 0x42, 0x6c,
- 0x6f, 0x62, 0x73, 0x22, 0x33, 0x0a, 0x06, 0x46, 0x6f, 0x72, 0x6d, 0x61, 0x74, 0x12, 0x07, 0x0a,
- 0x03, 0x5a, 0x49, 0x50, 0x10, 0x00, 0x12, 0x07, 0x0a, 0x03, 0x54, 0x41, 0x52, 0x10, 0x01, 0x12,
- 0x0a, 0x0a, 0x06, 0x54, 0x41, 0x52, 0x5f, 0x47, 0x5a, 0x10, 0x02, 0x12, 0x0b, 0x0a, 0x07, 0x54,
- 0x41, 0x52, 0x5f, 0x42, 0x5a, 0x32, 0x10, 0x03, 0x22, 0x28, 0x0a, 0x12, 0x47, 0x65, 0x74, 0x41,
- 0x72, 0x63, 0x68, 0x69, 0x76, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x12,
- 0x0a, 0x04, 0x64, 0x61, 0x74, 0x61, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x04, 0x64, 0x61,
- 0x74, 0x61, 0x22, 0x53, 0x0a, 0x17, 0x48, 0x61, 0x73, 0x4c, 0x6f, 0x63, 0x61, 0x6c, 0x42, 0x72,
- 0x61, 0x6e, 0x63, 0x68, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x38, 0x0a,
+ 0x0a, 0x72, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x22, 0x3b, 0x0a, 0x19, 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, 0x12, 0x1e, 0x0a, 0x0a, 0x61, 0x74, 0x74, 0x72,
+ 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0a, 0x61, 0x74,
+ 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x22, 0x54, 0x0a, 0x18, 0x43, 0x61, 0x6c, 0x63,
+ 0x75, 0x6c, 0x61, 0x74, 0x65, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x73, 0x75, 0x6d, 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, 0x37,
+ 0x0a, 0x19, 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, 0x12, 0x1a, 0x0a, 0x08, 0x63,
+ 0x68, 0x65, 0x63, 0x6b, 0x73, 0x75, 0x6d, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x63,
+ 0x68, 0x65, 0x63, 0x6b, 0x73, 0x75, 0x6d, 0x22, 0x4e, 0x0a, 0x12, 0x47, 0x65, 0x74, 0x53, 0x6e,
+ 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 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, 0x30, 0x0a, 0x18, 0x48, 0x61, 0x73, 0x4c, 0x6f,
- 0x63, 0x61, 0x6c, 0x42, 0x72, 0x61, 0x6e, 0x63, 0x68, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f,
- 0x6e, 0x73, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x01, 0x20, 0x01,
- 0x28, 0x08, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x22, 0xd9, 0x01, 0x0a, 0x18, 0x46, 0x65,
- 0x74, 0x63, 0x68, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x42, 0x72, 0x61, 0x6e, 0x63, 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, 0x3f, 0x0a, 0x11, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 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, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72,
- 0x79, 0x12, 0x23, 0x0a, 0x0d, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x62, 0x72, 0x61, 0x6e,
- 0x63, 0x68, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0c, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65,
- 0x42, 0x72, 0x61, 0x6e, 0x63, 0x68, 0x12, 0x1d, 0x0a, 0x0a, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74,
- 0x5f, 0x72, 0x65, 0x66, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x09, 0x74, 0x61, 0x72, 0x67,
- 0x65, 0x74, 0x52, 0x65, 0x66, 0x22, 0x33, 0x0a, 0x19, 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, 0x12, 0x16, 0x0a, 0x06, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x18, 0x01, 0x20, 0x01,
- 0x28, 0x08, 0x52, 0x06, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x22, 0x47, 0x0a, 0x0b, 0x46, 0x73,
- 0x63, 0x6b, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x38, 0x0a, 0x0a, 0x72, 0x65, 0x70,
+ 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x22, 0x29, 0x0a, 0x13, 0x47, 0x65, 0x74, 0x53, 0x6e,
+ 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x12,
+ 0x0a, 0x04, 0x64, 0x61, 0x74, 0x61, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x04, 0x64, 0x61,
+ 0x74, 0x61, 0x22, 0xd3, 0x01, 0x0a, 0x23, 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, 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, 0x19, 0x0a, 0x08, 0x68, 0x74, 0x74, 0x70, 0x5f, 0x75, 0x72, 0x6c,
+ 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x68, 0x74, 0x74, 0x70, 0x55, 0x72, 0x6c, 0x12,
+ 0x1b, 0x0a, 0x09, 0x68, 0x74, 0x74, 0x70, 0x5f, 0x61, 0x75, 0x74, 0x68, 0x18, 0x03, 0x20, 0x01,
+ 0x28, 0x09, 0x52, 0x08, 0x68, 0x74, 0x74, 0x70, 0x41, 0x75, 0x74, 0x68, 0x12, 0x29, 0x0a, 0x10,
+ 0x72, 0x65, 0x73, 0x6f, 0x6c, 0x76, 0x65, 0x64, 0x5f, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73,
+ 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x72, 0x65, 0x73, 0x6f, 0x6c, 0x76, 0x65, 0x64,
+ 0x41, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x4a, 0x04, 0x08, 0x04, 0x10, 0x05, 0x52, 0x09, 0x68,
+ 0x74, 0x74, 0x70, 0x5f, 0x68, 0x6f, 0x73, 0x74, 0x22, 0x26, 0x0a, 0x24, 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, 0x96, 0x01, 0x0a, 0x14, 0x47, 0x65, 0x74, 0x52, 0x61, 0x77, 0x43, 0x68, 0x61, 0x6e, 0x67,
+ 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, 0x22, 0x24, 0x0a, 0x0c, 0x46, 0x73, 0x63, 0x6b, 0x52, 0x65, 0x73, 0x70, 0x6f,
- 0x6e, 0x73, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x01, 0x20, 0x01,
- 0x28, 0x0c, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x22, 0xb8, 0x01, 0x0a, 0x0f, 0x57, 0x72,
- 0x69, 0x74, 0x65, 0x52, 0x65, 0x66, 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, 0x10, 0x0a, 0x03, 0x72, 0x65, 0x66, 0x18, 0x02,
- 0x20, 0x01, 0x28, 0x0c, 0x52, 0x03, 0x72, 0x65, 0x66, 0x12, 0x1a, 0x0a, 0x08, 0x72, 0x65, 0x76,
- 0x69, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x08, 0x72, 0x65, 0x76,
- 0x69, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x21, 0x0a, 0x0c, 0x6f, 0x6c, 0x64, 0x5f, 0x72, 0x65, 0x76,
- 0x69, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0b, 0x6f, 0x6c, 0x64,
- 0x52, 0x65, 0x76, 0x69, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x14, 0x0a, 0x05, 0x66, 0x6f, 0x72, 0x63,
- 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, 0x05, 0x66, 0x6f, 0x72, 0x63, 0x65, 0x4a, 0x04,
- 0x08, 0x06, 0x10, 0x07, 0x22, 0x18, 0x0a, 0x10, 0x57, 0x72, 0x69, 0x74, 0x65, 0x52, 0x65, 0x66,
- 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x4a, 0x04, 0x08, 0x01, 0x10, 0x02, 0x22, 0x6e,
- 0x0a, 0x14, 0x46, 0x69, 0x6e, 0x64, 0x4d, 0x65, 0x72, 0x67, 0x65, 0x42, 0x61, 0x73, 0x65, 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, 0x1c, 0x0a, 0x09, 0x72, 0x65, 0x76, 0x69, 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x02, 0x20,
- 0x03, 0x28, 0x0c, 0x52, 0x09, 0x72, 0x65, 0x76, 0x69, 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x22, 0x2b,
- 0x0a, 0x15, 0x46, 0x69, 0x6e, 0x64, 0x4d, 0x65, 0x72, 0x67, 0x65, 0x42, 0x61, 0x73, 0x65, 0x52,
- 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x62, 0x61, 0x73, 0x65, 0x18,
- 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x62, 0x61, 0x73, 0x65, 0x22, 0xaa, 0x01, 0x0a, 0x11,
- 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x46, 0x6f, 0x72, 0x6b, 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, 0x73,
- 0x6f, 0x75, 0x72, 0x63, 0x65, 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, 0x73, 0x6f, 0x75, 0x72,
- 0x63, 0x65, 0x52, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x12, 0x1a, 0x0a, 0x08,
- 0x72, 0x65, 0x76, 0x69, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x08,
- 0x72, 0x65, 0x76, 0x69, 0x73, 0x69, 0x6f, 0x6e, 0x22, 0x14, 0x0a, 0x12, 0x43, 0x72, 0x65, 0x61,
- 0x74, 0x65, 0x46, 0x6f, 0x72, 0x6b, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xfc,
- 0x01, 0x0a, 0x1e, 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, 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, 0x10, 0x0a, 0x03, 0x75,
- 0x72, 0x6c, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x75, 0x72, 0x6c, 0x12, 0x3a, 0x0a,
- 0x19, 0x68, 0x74, 0x74, 0x70, 0x5f, 0x61, 0x75, 0x74, 0x68, 0x6f, 0x72, 0x69, 0x7a, 0x61, 0x74,
- 0x69, 0x6f, 0x6e, 0x5f, 0x68, 0x65, 0x61, 0x64, 0x65, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09,
- 0x52, 0x17, 0x68, 0x74, 0x74, 0x70, 0x41, 0x75, 0x74, 0x68, 0x6f, 0x72, 0x69, 0x7a, 0x61, 0x74,
- 0x69, 0x6f, 0x6e, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x16, 0x0a, 0x06, 0x6d, 0x69, 0x72,
- 0x72, 0x6f, 0x72, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, 0x06, 0x6d, 0x69, 0x72, 0x72, 0x6f,
- 0x72, 0x12, 0x29, 0x0a, 0x10, 0x72, 0x65, 0x73, 0x6f, 0x6c, 0x76, 0x65, 0x64, 0x5f, 0x61, 0x64,
- 0x64, 0x72, 0x65, 0x73, 0x73, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x72, 0x65, 0x73,
- 0x6f, 0x6c, 0x76, 0x65, 0x64, 0x41, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x4a, 0x04, 0x08, 0x03,
- 0x10, 0x04, 0x52, 0x09, 0x68, 0x74, 0x74, 0x70, 0x5f, 0x68, 0x6f, 0x73, 0x74, 0x22, 0x21, 0x0a,
- 0x1f, 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, 0x4f, 0x0a, 0x13, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65,
- 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, 0x2a, 0x0a, 0x14, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x42, 0x75, 0x6e, 0x64, 0x6c,
- 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x64, 0x61, 0x74,
- 0x61, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x04, 0x64, 0x61, 0x74, 0x61, 0x22, 0x76, 0x0a,
- 0x1e, 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, 0x12,
+ 0x6f, 0x72, 0x79, 0x12, 0x23, 0x0a, 0x0d, 0x66, 0x72, 0x6f, 0x6d, 0x5f, 0x72, 0x65, 0x76, 0x69,
+ 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x66, 0x72, 0x6f, 0x6d,
+ 0x52, 0x65, 0x76, 0x69, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x1f, 0x0a, 0x0b, 0x74, 0x6f, 0x5f, 0x72,
+ 0x65, 0x76, 0x69, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x74,
+ 0x6f, 0x52, 0x65, 0x76, 0x69, 0x73, 0x69, 0x6f, 0x6e, 0x22, 0x9f, 0x04, 0x0a, 0x15, 0x47, 0x65,
+ 0x74, 0x52, 0x61, 0x77, 0x43, 0x68, 0x61, 0x6e, 0x67, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f,
+ 0x6e, 0x73, 0x65, 0x12, 0x48, 0x0a, 0x0b, 0x72, 0x61, 0x77, 0x5f, 0x63, 0x68, 0x61, 0x6e, 0x67,
+ 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x27, 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, 0x2e, 0x52, 0x61, 0x77, 0x43, 0x68, 0x61, 0x6e, 0x67,
+ 0x65, 0x52, 0x0a, 0x72, 0x61, 0x77, 0x43, 0x68, 0x61, 0x6e, 0x67, 0x65, 0x73, 0x1a, 0xbb, 0x03,
+ 0x0a, 0x09, 0x52, 0x61, 0x77, 0x43, 0x68, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x17, 0x0a, 0x07, 0x62,
+ 0x6c, 0x6f, 0x62, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x62, 0x6c,
+ 0x6f, 0x62, 0x49, 0x64, 0x12, 0x12, 0x0a, 0x04, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x02, 0x20, 0x01,
+ 0x28, 0x03, 0x52, 0x04, 0x73, 0x69, 0x7a, 0x65, 0x12, 0x4f, 0x0a, 0x09, 0x6f, 0x70, 0x65, 0x72,
+ 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x31, 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, 0x2e, 0x52, 0x61, 0x77, 0x43, 0x68,
+ 0x61, 0x6e, 0x67, 0x65, 0x2e, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x09,
+ 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x23, 0x0a, 0x0d, 0x72, 0x61, 0x77,
+ 0x5f, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09,
+ 0x52, 0x0c, 0x72, 0x61, 0x77, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x19,
+ 0x0a, 0x08, 0x6f, 0x6c, 0x64, 0x5f, 0x6d, 0x6f, 0x64, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x05,
+ 0x52, 0x07, 0x6f, 0x6c, 0x64, 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x19, 0x0a, 0x08, 0x6e, 0x65, 0x77,
+ 0x5f, 0x6d, 0x6f, 0x64, 0x65, 0x18, 0x08, 0x20, 0x01, 0x28, 0x05, 0x52, 0x07, 0x6e, 0x65, 0x77,
+ 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x24, 0x0a, 0x0e, 0x6e, 0x65, 0x77, 0x5f, 0x70, 0x61, 0x74, 0x68,
+ 0x5f, 0x62, 0x79, 0x74, 0x65, 0x73, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0c, 0x6e, 0x65,
+ 0x77, 0x50, 0x61, 0x74, 0x68, 0x42, 0x79, 0x74, 0x65, 0x73, 0x12, 0x24, 0x0a, 0x0e, 0x6f, 0x6c,
+ 0x64, 0x5f, 0x70, 0x61, 0x74, 0x68, 0x5f, 0x62, 0x79, 0x74, 0x65, 0x73, 0x18, 0x0a, 0x20, 0x01,
+ 0x28, 0x0c, 0x52, 0x0c, 0x6f, 0x6c, 0x64, 0x50, 0x61, 0x74, 0x68, 0x42, 0x79, 0x74, 0x65, 0x73,
+ 0x22, 0x69, 0x0a, 0x09, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x0b, 0x0a,
+ 0x07, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e, 0x10, 0x00, 0x12, 0x09, 0x0a, 0x05, 0x41, 0x44,
+ 0x44, 0x45, 0x44, 0x10, 0x01, 0x12, 0x0a, 0x0a, 0x06, 0x43, 0x4f, 0x50, 0x49, 0x45, 0x44, 0x10,
+ 0x02, 0x12, 0x0b, 0x0a, 0x07, 0x44, 0x45, 0x4c, 0x45, 0x54, 0x45, 0x44, 0x10, 0x03, 0x12, 0x0c,
+ 0x0a, 0x08, 0x4d, 0x4f, 0x44, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x04, 0x12, 0x0b, 0x0a, 0x07,
+ 0x52, 0x45, 0x4e, 0x41, 0x4d, 0x45, 0x44, 0x10, 0x05, 0x12, 0x10, 0x0a, 0x0c, 0x54, 0x59, 0x50,
+ 0x45, 0x5f, 0x43, 0x48, 0x41, 0x4e, 0x47, 0x45, 0x44, 0x10, 0x06, 0x4a, 0x04, 0x08, 0x03, 0x10,
+ 0x04, 0x4a, 0x04, 0x08, 0x04, 0x10, 0x05, 0x52, 0x08, 0x6e, 0x65, 0x77, 0x5f, 0x70, 0x61, 0x74,
+ 0x68, 0x52, 0x08, 0x6f, 0x6c, 0x64, 0x5f, 0x70, 0x61, 0x74, 0x68, 0x22, 0xc2, 0x01, 0x0a, 0x18,
+ 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, 0x46, 0x69, 0x6c, 0x65, 0x73, 0x42, 0x79, 0x4e, 0x61, 0x6d,
+ 0x65, 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, 0x14, 0x0a, 0x05, 0x71, 0x75, 0x65, 0x72, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28,
+ 0x09, 0x52, 0x05, 0x71, 0x75, 0x65, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x72, 0x65, 0x66, 0x18,
+ 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x03, 0x72, 0x65, 0x66, 0x12, 0x16, 0x0a, 0x06, 0x66, 0x69,
+ 0x6c, 0x74, 0x65, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x66, 0x69, 0x6c, 0x74,
+ 0x65, 0x72, 0x12, 0x14, 0x0a, 0x05, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28,
+ 0x0d, 0x52, 0x05, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x12, 0x16, 0x0a, 0x06, 0x6f, 0x66, 0x66, 0x73,
+ 0x65, 0x74, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x06, 0x6f, 0x66, 0x66, 0x73, 0x65, 0x74,
+ 0x22, 0x31, 0x0a, 0x19, 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, 0x12, 0x14, 0x0a,
+ 0x05, 0x66, 0x69, 0x6c, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0c, 0x52, 0x05, 0x66, 0x69,
+ 0x6c, 0x65, 0x73, 0x22, 0xaa, 0x01, 0x0a, 0x1b, 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, 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, 0x14, 0x0a,
+ 0x05, 0x71, 0x75, 0x65, 0x72, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x71, 0x75,
+ 0x65, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x72, 0x65, 0x66, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c,
+ 0x52, 0x03, 0x72, 0x65, 0x66, 0x12, 0x29, 0x0a, 0x10, 0x63, 0x68, 0x75, 0x6e, 0x6b, 0x65, 0x64,
+ 0x5f, 0x72, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52,
+ 0x0f, 0x63, 0x68, 0x75, 0x6e, 0x6b, 0x65, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65,
+ 0x22, 0x79, 0x0a, 0x1c, 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,
+ 0x12, 0x18, 0x0a, 0x07, 0x6d, 0x61, 0x74, 0x63, 0x68, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28,
+ 0x0c, 0x52, 0x07, 0x6d, 0x61, 0x74, 0x63, 0x68, 0x65, 0x73, 0x12, 0x1d, 0x0a, 0x0a, 0x6d, 0x61,
+ 0x74, 0x63, 0x68, 0x5f, 0x64, 0x61, 0x74, 0x61, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x09,
+ 0x6d, 0x61, 0x74, 0x63, 0x68, 0x44, 0x61, 0x74, 0x61, 0x12, 0x20, 0x0a, 0x0c, 0x65, 0x6e, 0x64,
+ 0x5f, 0x6f, 0x66, 0x5f, 0x6d, 0x61, 0x74, 0x63, 0x68, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52,
+ 0x0a, 0x65, 0x6e, 0x64, 0x4f, 0x66, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x22, 0xc5, 0x01, 0x0a, 0x06,
+ 0x52, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x12, 0x10, 0x0a, 0x03, 0x75, 0x72, 0x6c, 0x18, 0x01, 0x20,
+ 0x01, 0x28, 0x09, 0x52, 0x03, 0x75, 0x72, 0x6c, 0x12, 0x3a, 0x0a, 0x19, 0x68, 0x74, 0x74, 0x70,
+ 0x5f, 0x61, 0x75, 0x74, 0x68, 0x6f, 0x72, 0x69, 0x7a, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x68,
+ 0x65, 0x61, 0x64, 0x65, 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x17, 0x68, 0x74, 0x74,
+ 0x70, 0x41, 0x75, 0x74, 0x68, 0x6f, 0x72, 0x69, 0x7a, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x48, 0x65,
+ 0x61, 0x64, 0x65, 0x72, 0x12, 0x25, 0x0a, 0x0e, 0x6d, 0x69, 0x72, 0x72, 0x6f, 0x72, 0x5f, 0x72,
+ 0x65, 0x66, 0x6d, 0x61, 0x70, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0d, 0x6d, 0x69,
+ 0x72, 0x72, 0x6f, 0x72, 0x52, 0x65, 0x66, 0x6d, 0x61, 0x70, 0x73, 0x12, 0x29, 0x0a, 0x10, 0x72,
+ 0x65, 0x73, 0x6f, 0x6c, 0x76, 0x65, 0x64, 0x5f, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x18,
+ 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x72, 0x65, 0x73, 0x6f, 0x6c, 0x76, 0x65, 0x64, 0x41,
+ 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x4a, 0x04, 0x08, 0x02, 0x10, 0x03, 0x4a, 0x04, 0x08, 0x05,
+ 0x10, 0x06, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x52, 0x09, 0x68, 0x74, 0x74, 0x70, 0x5f, 0x68,
+ 0x6f, 0x73, 0x74, 0x22, 0x59, 0x0a, 0x1d, 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, 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, 0x34,
+ 0x0a, 0x1e, 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,
+ 0x12, 0x12, 0x0a, 0x04, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x04,
+ 0x73, 0x69, 0x7a, 0x65, 0x22, 0x53, 0x0a, 0x17, 0x52, 0x65, 0x6d, 0x6f, 0x76, 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, 0x1a, 0x0a, 0x08, 0x70, 0x61, 0x74,
- 0x74, 0x65, 0x72, 0x6e, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0c, 0x52, 0x08, 0x70, 0x61, 0x74,
- 0x74, 0x65, 0x72, 0x6e, 0x73, 0x22, 0x35, 0x0a, 0x1f, 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, 0x12, 0x12, 0x0a, 0x04, 0x64, 0x61, 0x74, 0x61,
- 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x04, 0x64, 0x61, 0x74, 0x61, 0x22, 0x4c, 0x0a, 0x10,
- 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 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, 0x27, 0x0a, 0x11, 0x47, 0x65,
- 0x74, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12,
- 0x12, 0x0a, 0x04, 0x64, 0x61, 0x74, 0x61, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x04, 0x64,
- 0x61, 0x74, 0x61, 0x22, 0x69, 0x0a, 0x19, 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,
- 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, 0x64, 0x61,
- 0x74, 0x61, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x04, 0x64, 0x61, 0x74, 0x61, 0x22, 0x65,
- 0x0a, 0x15, 0x53, 0x65, 0x74, 0x43, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x48, 0x6f, 0x6f, 0x6b, 0x73,
+ 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, 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, 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, 0x12, 0x0a, 0x04, 0x64, 0x61, 0x74, 0x61, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52,
- 0x04, 0x64, 0x61, 0x74, 0x61, 0x22, 0x1c, 0x0a, 0x1a, 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, 0x18, 0x0a, 0x16, 0x53, 0x65, 0x74, 0x43, 0x75, 0x73, 0x74, 0x6f, 0x6d,
- 0x48, 0x6f, 0x6f, 0x6b, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x54, 0x0a,
- 0x18, 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x43, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x48, 0x6f, 0x6f,
- 0x6b, 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, 0x51, 0x0a, 0x15, 0x47, 0x65, 0x74, 0x43, 0x75, 0x73, 0x74, 0x6f, 0x6d,
- 0x48, 0x6f, 0x6f, 0x6b, 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, 0x2f, 0x0a, 0x19, 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, 0x12, 0x12, 0x0a, 0x04, 0x64, 0x61, 0x74, 0x61, 0x18, 0x01, 0x20, 0x01, 0x28,
- 0x0c, 0x52, 0x04, 0x64, 0x61, 0x74, 0x61, 0x22, 0x2c, 0x0a, 0x16, 0x47, 0x65, 0x74, 0x43, 0x75,
- 0x73, 0x74, 0x6f, 0x6d, 0x48, 0x6f, 0x6f, 0x6b, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73,
- 0x65, 0x12, 0x12, 0x0a, 0x04, 0x64, 0x61, 0x74, 0x61, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52,
- 0x04, 0x64, 0x61, 0x74, 0x61, 0x22, 0x71, 0x0a, 0x21, 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, 0x12, 0x38, 0x0a, 0x0a, 0x72, 0x65,
+ 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, 0x64, 0x61, 0x74, 0x61, 0x18, 0x02, 0x20, 0x01,
- 0x28, 0x0c, 0x52, 0x04, 0x64, 0x61, 0x74, 0x61, 0x22, 0x24, 0x0a, 0x22, 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, 0x4e,
- 0x0a, 0x12, 0x46, 0x69, 0x6e, 0x64, 0x4c, 0x69, 0x63, 0x65, 0x6e, 0x73, 0x65, 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, 0xd5,
- 0x01, 0x0a, 0x13, 0x46, 0x69, 0x6e, 0x64, 0x4c, 0x69, 0x63, 0x65, 0x6e, 0x73, 0x65, 0x52, 0x65,
- 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2c, 0x0a, 0x12, 0x6c, 0x69, 0x63, 0x65, 0x6e, 0x73,
- 0x65, 0x5f, 0x73, 0x68, 0x6f, 0x72, 0x74, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01,
- 0x28, 0x09, 0x52, 0x10, 0x6c, 0x69, 0x63, 0x65, 0x6e, 0x73, 0x65, 0x53, 0x68, 0x6f, 0x72, 0x74,
- 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x21, 0x0a, 0x0c, 0x6c, 0x69, 0x63, 0x65, 0x6e, 0x73, 0x65, 0x5f,
- 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x6c, 0x69, 0x63, 0x65,
- 0x6e, 0x73, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x6c, 0x69, 0x63, 0x65, 0x6e,
- 0x73, 0x65, 0x5f, 0x75, 0x72, 0x6c, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x6c, 0x69,
- 0x63, 0x65, 0x6e, 0x73, 0x65, 0x55, 0x72, 0x6c, 0x12, 0x21, 0x0a, 0x0c, 0x6c, 0x69, 0x63, 0x65,
- 0x6e, 0x73, 0x65, 0x5f, 0x70, 0x61, 0x74, 0x68, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b,
- 0x6c, 0x69, 0x63, 0x65, 0x6e, 0x73, 0x65, 0x50, 0x61, 0x74, 0x68, 0x12, 0x29, 0x0a, 0x10, 0x6c,
- 0x69, 0x63, 0x65, 0x6e, 0x73, 0x65, 0x5f, 0x6e, 0x69, 0x63, 0x6b, 0x6e, 0x61, 0x6d, 0x65, 0x18,
- 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x6c, 0x69, 0x63, 0x65, 0x6e, 0x73, 0x65, 0x4e, 0x69,
- 0x63, 0x6b, 0x6e, 0x61, 0x6d, 0x65, 0x22, 0x54, 0x0a, 0x18, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x66,
- 0x6f, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65,
+ 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, 0x3b, 0x0a, 0x19,
- 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, 0x12, 0x1e, 0x0a, 0x0a, 0x61, 0x74, 0x74,
- 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0a, 0x61,
- 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x22, 0x54, 0x0a, 0x18, 0x43, 0x61, 0x6c,
- 0x63, 0x75, 0x6c, 0x61, 0x74, 0x65, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x73, 0x75, 0x6d, 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,
- 0x37, 0x0a, 0x19, 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, 0x12, 0x1a, 0x0a, 0x08,
- 0x63, 0x68, 0x65, 0x63, 0x6b, 0x73, 0x75, 0x6d, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08,
- 0x63, 0x68, 0x65, 0x63, 0x6b, 0x73, 0x75, 0x6d, 0x22, 0x4e, 0x0a, 0x12, 0x47, 0x65, 0x74, 0x53,
- 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 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, 0x29, 0x0a, 0x13, 0x47, 0x65, 0x74, 0x53,
- 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12,
- 0x12, 0x0a, 0x04, 0x64, 0x61, 0x74, 0x61, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x04, 0x64,
- 0x61, 0x74, 0x61, 0x22, 0xd3, 0x01, 0x0a, 0x23, 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, 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, 0x19, 0x0a, 0x08, 0x68, 0x74, 0x74, 0x70, 0x5f, 0x75, 0x72,
- 0x6c, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x68, 0x74, 0x74, 0x70, 0x55, 0x72, 0x6c,
- 0x12, 0x1b, 0x0a, 0x09, 0x68, 0x74, 0x74, 0x70, 0x5f, 0x61, 0x75, 0x74, 0x68, 0x18, 0x03, 0x20,
- 0x01, 0x28, 0x09, 0x52, 0x08, 0x68, 0x74, 0x74, 0x70, 0x41, 0x75, 0x74, 0x68, 0x12, 0x29, 0x0a,
- 0x10, 0x72, 0x65, 0x73, 0x6f, 0x6c, 0x76, 0x65, 0x64, 0x5f, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73,
- 0x73, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x72, 0x65, 0x73, 0x6f, 0x6c, 0x76, 0x65,
- 0x64, 0x41, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x4a, 0x04, 0x08, 0x04, 0x10, 0x05, 0x52, 0x09,
- 0x68, 0x74, 0x74, 0x70, 0x5f, 0x68, 0x6f, 0x73, 0x74, 0x22, 0x26, 0x0a, 0x24, 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, 0x96, 0x01, 0x0a, 0x14, 0x47, 0x65, 0x74, 0x52, 0x61, 0x77, 0x43, 0x68, 0x61, 0x6e,
- 0x67, 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, 0x23, 0x0a, 0x0d, 0x66, 0x72, 0x6f, 0x6d, 0x5f, 0x72, 0x65, 0x76,
- 0x69, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x66, 0x72, 0x6f,
- 0x6d, 0x52, 0x65, 0x76, 0x69, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x1f, 0x0a, 0x0b, 0x74, 0x6f, 0x5f,
- 0x72, 0x65, 0x76, 0x69, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a,
- 0x74, 0x6f, 0x52, 0x65, 0x76, 0x69, 0x73, 0x69, 0x6f, 0x6e, 0x22, 0x9f, 0x04, 0x0a, 0x15, 0x47,
- 0x65, 0x74, 0x52, 0x61, 0x77, 0x43, 0x68, 0x61, 0x6e, 0x67, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70,
- 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x48, 0x0a, 0x0b, 0x72, 0x61, 0x77, 0x5f, 0x63, 0x68, 0x61, 0x6e,
- 0x67, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x27, 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, 0x2e, 0x52, 0x61, 0x77, 0x43, 0x68, 0x61, 0x6e,
- 0x67, 0x65, 0x52, 0x0a, 0x72, 0x61, 0x77, 0x43, 0x68, 0x61, 0x6e, 0x67, 0x65, 0x73, 0x1a, 0xbb,
- 0x03, 0x0a, 0x09, 0x52, 0x61, 0x77, 0x43, 0x68, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x17, 0x0a, 0x07,
- 0x62, 0x6c, 0x6f, 0x62, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x62,
- 0x6c, 0x6f, 0x62, 0x49, 0x64, 0x12, 0x12, 0x0a, 0x04, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x02, 0x20,
- 0x01, 0x28, 0x03, 0x52, 0x04, 0x73, 0x69, 0x7a, 0x65, 0x12, 0x4f, 0x0a, 0x09, 0x6f, 0x70, 0x65,
- 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x31, 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, 0x2e, 0x52, 0x61, 0x77, 0x43,
- 0x68, 0x61, 0x6e, 0x67, 0x65, 0x2e, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52,
- 0x09, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x23, 0x0a, 0x0d, 0x72, 0x61,
- 0x77, 0x5f, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x06, 0x20, 0x01, 0x28,
- 0x09, 0x52, 0x0c, 0x72, 0x61, 0x77, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12,
- 0x19, 0x0a, 0x08, 0x6f, 0x6c, 0x64, 0x5f, 0x6d, 0x6f, 0x64, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28,
- 0x05, 0x52, 0x07, 0x6f, 0x6c, 0x64, 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x19, 0x0a, 0x08, 0x6e, 0x65,
- 0x77, 0x5f, 0x6d, 0x6f, 0x64, 0x65, 0x18, 0x08, 0x20, 0x01, 0x28, 0x05, 0x52, 0x07, 0x6e, 0x65,
- 0x77, 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x24, 0x0a, 0x0e, 0x6e, 0x65, 0x77, 0x5f, 0x70, 0x61, 0x74,
- 0x68, 0x5f, 0x62, 0x79, 0x74, 0x65, 0x73, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0c, 0x6e,
- 0x65, 0x77, 0x50, 0x61, 0x74, 0x68, 0x42, 0x79, 0x74, 0x65, 0x73, 0x12, 0x24, 0x0a, 0x0e, 0x6f,
- 0x6c, 0x64, 0x5f, 0x70, 0x61, 0x74, 0x68, 0x5f, 0x62, 0x79, 0x74, 0x65, 0x73, 0x18, 0x0a, 0x20,
- 0x01, 0x28, 0x0c, 0x52, 0x0c, 0x6f, 0x6c, 0x64, 0x50, 0x61, 0x74, 0x68, 0x42, 0x79, 0x74, 0x65,
- 0x73, 0x22, 0x69, 0x0a, 0x09, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x0b,
- 0x0a, 0x07, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e, 0x10, 0x00, 0x12, 0x09, 0x0a, 0x05, 0x41,
- 0x44, 0x44, 0x45, 0x44, 0x10, 0x01, 0x12, 0x0a, 0x0a, 0x06, 0x43, 0x4f, 0x50, 0x49, 0x45, 0x44,
- 0x10, 0x02, 0x12, 0x0b, 0x0a, 0x07, 0x44, 0x45, 0x4c, 0x45, 0x54, 0x45, 0x44, 0x10, 0x03, 0x12,
- 0x0c, 0x0a, 0x08, 0x4d, 0x4f, 0x44, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x04, 0x12, 0x0b, 0x0a,
- 0x07, 0x52, 0x45, 0x4e, 0x41, 0x4d, 0x45, 0x44, 0x10, 0x05, 0x12, 0x10, 0x0a, 0x0c, 0x54, 0x59,
- 0x50, 0x45, 0x5f, 0x43, 0x48, 0x41, 0x4e, 0x47, 0x45, 0x44, 0x10, 0x06, 0x4a, 0x04, 0x08, 0x03,
- 0x10, 0x04, 0x4a, 0x04, 0x08, 0x04, 0x10, 0x05, 0x52, 0x08, 0x6e, 0x65, 0x77, 0x5f, 0x70, 0x61,
- 0x74, 0x68, 0x52, 0x08, 0x6f, 0x6c, 0x64, 0x5f, 0x70, 0x61, 0x74, 0x68, 0x22, 0xc2, 0x01, 0x0a,
- 0x18, 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, 0x46, 0x69, 0x6c, 0x65, 0x73, 0x42, 0x79, 0x4e, 0x61,
- 0x6d, 0x65, 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, 0x14, 0x0a, 0x05, 0x71, 0x75, 0x65, 0x72, 0x79, 0x18, 0x02, 0x20, 0x01,
- 0x28, 0x09, 0x52, 0x05, 0x71, 0x75, 0x65, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x72, 0x65, 0x66,
- 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x03, 0x72, 0x65, 0x66, 0x12, 0x16, 0x0a, 0x06, 0x66,
- 0x69, 0x6c, 0x74, 0x65, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x66, 0x69, 0x6c,
- 0x74, 0x65, 0x72, 0x12, 0x14, 0x0a, 0x05, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x18, 0x05, 0x20, 0x01,
- 0x28, 0x0d, 0x52, 0x05, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x12, 0x16, 0x0a, 0x06, 0x6f, 0x66, 0x66,
- 0x73, 0x65, 0x74, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x06, 0x6f, 0x66, 0x66, 0x73, 0x65,
- 0x74, 0x22, 0x31, 0x0a, 0x19, 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, 0x12, 0x14,
- 0x0a, 0x05, 0x66, 0x69, 0x6c, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0c, 0x52, 0x05, 0x66,
- 0x69, 0x6c, 0x65, 0x73, 0x22, 0xaa, 0x01, 0x0a, 0x1b, 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, 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, 0x14,
- 0x0a, 0x05, 0x71, 0x75, 0x65, 0x72, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x71,
- 0x75, 0x65, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x72, 0x65, 0x66, 0x18, 0x03, 0x20, 0x01, 0x28,
- 0x0c, 0x52, 0x03, 0x72, 0x65, 0x66, 0x12, 0x29, 0x0a, 0x10, 0x63, 0x68, 0x75, 0x6e, 0x6b, 0x65,
- 0x64, 0x5f, 0x72, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08,
- 0x52, 0x0f, 0x63, 0x68, 0x75, 0x6e, 0x6b, 0x65, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73,
- 0x65, 0x22, 0x79, 0x0a, 0x1c, 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, 0x12, 0x18, 0x0a, 0x07, 0x6d, 0x61, 0x74, 0x63, 0x68, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03,
- 0x28, 0x0c, 0x52, 0x07, 0x6d, 0x61, 0x74, 0x63, 0x68, 0x65, 0x73, 0x12, 0x1d, 0x0a, 0x0a, 0x6d,
- 0x61, 0x74, 0x63, 0x68, 0x5f, 0x64, 0x61, 0x74, 0x61, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52,
- 0x09, 0x6d, 0x61, 0x74, 0x63, 0x68, 0x44, 0x61, 0x74, 0x61, 0x12, 0x20, 0x0a, 0x0c, 0x65, 0x6e,
- 0x64, 0x5f, 0x6f, 0x66, 0x5f, 0x6d, 0x61, 0x74, 0x63, 0x68, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08,
- 0x52, 0x0a, 0x65, 0x6e, 0x64, 0x4f, 0x66, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x22, 0xc5, 0x01, 0x0a,
- 0x06, 0x52, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x12, 0x10, 0x0a, 0x03, 0x75, 0x72, 0x6c, 0x18, 0x01,
- 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x75, 0x72, 0x6c, 0x12, 0x3a, 0x0a, 0x19, 0x68, 0x74, 0x74,
- 0x70, 0x5f, 0x61, 0x75, 0x74, 0x68, 0x6f, 0x72, 0x69, 0x7a, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f,
- 0x68, 0x65, 0x61, 0x64, 0x65, 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x17, 0x68, 0x74,
- 0x74, 0x70, 0x41, 0x75, 0x74, 0x68, 0x6f, 0x72, 0x69, 0x7a, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x48,
- 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x25, 0x0a, 0x0e, 0x6d, 0x69, 0x72, 0x72, 0x6f, 0x72, 0x5f,
- 0x72, 0x65, 0x66, 0x6d, 0x61, 0x70, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0d, 0x6d,
- 0x69, 0x72, 0x72, 0x6f, 0x72, 0x52, 0x65, 0x66, 0x6d, 0x61, 0x70, 0x73, 0x12, 0x29, 0x0a, 0x10,
- 0x72, 0x65, 0x73, 0x6f, 0x6c, 0x76, 0x65, 0x64, 0x5f, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73,
- 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x72, 0x65, 0x73, 0x6f, 0x6c, 0x76, 0x65, 0x64,
- 0x41, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x4a, 0x04, 0x08, 0x02, 0x10, 0x03, 0x4a, 0x04, 0x08,
- 0x05, 0x10, 0x06, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x52, 0x09, 0x68, 0x74, 0x74, 0x70, 0x5f,
- 0x68, 0x6f, 0x73, 0x74, 0x22, 0x59, 0x0a, 0x1d, 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, 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,
- 0x34, 0x0a, 0x1e, 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, 0x12, 0x12, 0x0a, 0x04, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52,
- 0x04, 0x73, 0x69, 0x7a, 0x65, 0x22, 0x53, 0x0a, 0x17, 0x52, 0x65, 0x6d, 0x6f, 0x76, 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, 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, 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,
+ 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,
- 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, 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, 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,
+ 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, 0xc7, 0x1f, 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,
- 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,
+ 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, 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,
+ 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, 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, 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, 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, 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,
+ 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, 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, 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,
+ 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,
- 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,
+ 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, 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,
+ 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, 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, 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,
+ 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,
- 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,
+ 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,
- 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,
- 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,
+ 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, 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, 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,
+ 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 (
@@ -6460,7 +6348,7 @@ func file_repository_proto_rawDescGZIP() []byte {
}
var file_repository_proto_enumTypes = make([]protoimpl.EnumInfo, 3)
-var file_repository_proto_msgTypes = make([]protoimpl.MessageInfo, 95)
+var file_repository_proto_msgTypes = make([]protoimpl.MessageInfo, 93)
var file_repository_proto_goTypes = []interface{}{
(GetArchiveRequest_Format)(0), // 0: gitaly.GetArchiveRequest.Format
(GetRawChangesResponse_RawChange_Operation)(0), // 1: gitaly.GetRawChangesResponse.RawChange.Operation
@@ -6475,246 +6363,241 @@ var file_repository_proto_goTypes = []interface{}{
(*ObjectsSizeResponse)(nil), // 10: gitaly.ObjectsSizeResponse
(*ObjectFormatRequest)(nil), // 11: gitaly.ObjectFormatRequest
(*ObjectFormatResponse)(nil), // 12: gitaly.ObjectFormatResponse
- (*ApplyGitattributesRequest)(nil), // 13: gitaly.ApplyGitattributesRequest
- (*ApplyGitattributesResponse)(nil), // 14: gitaly.ApplyGitattributesResponse
- (*FetchBundleRequest)(nil), // 15: gitaly.FetchBundleRequest
- (*FetchBundleResponse)(nil), // 16: gitaly.FetchBundleResponse
- (*FetchRemoteRequest)(nil), // 17: gitaly.FetchRemoteRequest
- (*FetchRemoteResponse)(nil), // 18: gitaly.FetchRemoteResponse
- (*CreateRepositoryRequest)(nil), // 19: gitaly.CreateRepositoryRequest
- (*CreateRepositoryResponse)(nil), // 20: gitaly.CreateRepositoryResponse
- (*GetArchiveRequest)(nil), // 21: gitaly.GetArchiveRequest
- (*GetArchiveResponse)(nil), // 22: gitaly.GetArchiveResponse
- (*HasLocalBranchesRequest)(nil), // 23: gitaly.HasLocalBranchesRequest
- (*HasLocalBranchesResponse)(nil), // 24: gitaly.HasLocalBranchesResponse
- (*FetchSourceBranchRequest)(nil), // 25: gitaly.FetchSourceBranchRequest
- (*FetchSourceBranchResponse)(nil), // 26: gitaly.FetchSourceBranchResponse
- (*FsckRequest)(nil), // 27: gitaly.FsckRequest
- (*FsckResponse)(nil), // 28: gitaly.FsckResponse
- (*WriteRefRequest)(nil), // 29: gitaly.WriteRefRequest
- (*WriteRefResponse)(nil), // 30: gitaly.WriteRefResponse
- (*FindMergeBaseRequest)(nil), // 31: gitaly.FindMergeBaseRequest
- (*FindMergeBaseResponse)(nil), // 32: gitaly.FindMergeBaseResponse
- (*CreateForkRequest)(nil), // 33: gitaly.CreateForkRequest
- (*CreateForkResponse)(nil), // 34: gitaly.CreateForkResponse
- (*CreateRepositoryFromURLRequest)(nil), // 35: gitaly.CreateRepositoryFromURLRequest
- (*CreateRepositoryFromURLResponse)(nil), // 36: gitaly.CreateRepositoryFromURLResponse
- (*CreateBundleRequest)(nil), // 37: gitaly.CreateBundleRequest
- (*CreateBundleResponse)(nil), // 38: gitaly.CreateBundleResponse
- (*CreateBundleFromRefListRequest)(nil), // 39: gitaly.CreateBundleFromRefListRequest
- (*CreateBundleFromRefListResponse)(nil), // 40: gitaly.CreateBundleFromRefListResponse
- (*GetConfigRequest)(nil), // 41: gitaly.GetConfigRequest
- (*GetConfigResponse)(nil), // 42: gitaly.GetConfigResponse
- (*RestoreCustomHooksRequest)(nil), // 43: gitaly.RestoreCustomHooksRequest
- (*SetCustomHooksRequest)(nil), // 44: gitaly.SetCustomHooksRequest
- (*RestoreCustomHooksResponse)(nil), // 45: gitaly.RestoreCustomHooksResponse
- (*SetCustomHooksResponse)(nil), // 46: gitaly.SetCustomHooksResponse
- (*BackupCustomHooksRequest)(nil), // 47: gitaly.BackupCustomHooksRequest
- (*GetCustomHooksRequest)(nil), // 48: gitaly.GetCustomHooksRequest
- (*BackupCustomHooksResponse)(nil), // 49: gitaly.BackupCustomHooksResponse
- (*GetCustomHooksResponse)(nil), // 50: gitaly.GetCustomHooksResponse
- (*CreateRepositoryFromBundleRequest)(nil), // 51: gitaly.CreateRepositoryFromBundleRequest
- (*CreateRepositoryFromBundleResponse)(nil), // 52: gitaly.CreateRepositoryFromBundleResponse
- (*FindLicenseRequest)(nil), // 53: gitaly.FindLicenseRequest
- (*FindLicenseResponse)(nil), // 54: gitaly.FindLicenseResponse
- (*GetInfoAttributesRequest)(nil), // 55: gitaly.GetInfoAttributesRequest
- (*GetInfoAttributesResponse)(nil), // 56: gitaly.GetInfoAttributesResponse
- (*CalculateChecksumRequest)(nil), // 57: gitaly.CalculateChecksumRequest
- (*CalculateChecksumResponse)(nil), // 58: gitaly.CalculateChecksumResponse
- (*GetSnapshotRequest)(nil), // 59: gitaly.GetSnapshotRequest
- (*GetSnapshotResponse)(nil), // 60: gitaly.GetSnapshotResponse
- (*CreateRepositoryFromSnapshotRequest)(nil), // 61: gitaly.CreateRepositoryFromSnapshotRequest
- (*CreateRepositoryFromSnapshotResponse)(nil), // 62: gitaly.CreateRepositoryFromSnapshotResponse
- (*GetRawChangesRequest)(nil), // 63: gitaly.GetRawChangesRequest
- (*GetRawChangesResponse)(nil), // 64: gitaly.GetRawChangesResponse
- (*SearchFilesByNameRequest)(nil), // 65: gitaly.SearchFilesByNameRequest
- (*SearchFilesByNameResponse)(nil), // 66: gitaly.SearchFilesByNameResponse
- (*SearchFilesByContentRequest)(nil), // 67: gitaly.SearchFilesByContentRequest
- (*SearchFilesByContentResponse)(nil), // 68: gitaly.SearchFilesByContentResponse
- (*Remote)(nil), // 69: gitaly.Remote
- (*GetObjectDirectorySizeRequest)(nil), // 70: gitaly.GetObjectDirectorySizeRequest
- (*GetObjectDirectorySizeResponse)(nil), // 71: gitaly.GetObjectDirectorySizeResponse
- (*RemoveRepositoryRequest)(nil), // 72: gitaly.RemoveRepositoryRequest
- (*RemoveRepositoryResponse)(nil), // 73: gitaly.RemoveRepositoryResponse
- (*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
+ (*FetchBundleRequest)(nil), // 13: gitaly.FetchBundleRequest
+ (*FetchBundleResponse)(nil), // 14: gitaly.FetchBundleResponse
+ (*FetchRemoteRequest)(nil), // 15: gitaly.FetchRemoteRequest
+ (*FetchRemoteResponse)(nil), // 16: gitaly.FetchRemoteResponse
+ (*CreateRepositoryRequest)(nil), // 17: gitaly.CreateRepositoryRequest
+ (*CreateRepositoryResponse)(nil), // 18: gitaly.CreateRepositoryResponse
+ (*GetArchiveRequest)(nil), // 19: gitaly.GetArchiveRequest
+ (*GetArchiveResponse)(nil), // 20: gitaly.GetArchiveResponse
+ (*HasLocalBranchesRequest)(nil), // 21: gitaly.HasLocalBranchesRequest
+ (*HasLocalBranchesResponse)(nil), // 22: gitaly.HasLocalBranchesResponse
+ (*FetchSourceBranchRequest)(nil), // 23: gitaly.FetchSourceBranchRequest
+ (*FetchSourceBranchResponse)(nil), // 24: gitaly.FetchSourceBranchResponse
+ (*FsckRequest)(nil), // 25: gitaly.FsckRequest
+ (*FsckResponse)(nil), // 26: gitaly.FsckResponse
+ (*WriteRefRequest)(nil), // 27: gitaly.WriteRefRequest
+ (*WriteRefResponse)(nil), // 28: gitaly.WriteRefResponse
+ (*FindMergeBaseRequest)(nil), // 29: gitaly.FindMergeBaseRequest
+ (*FindMergeBaseResponse)(nil), // 30: gitaly.FindMergeBaseResponse
+ (*CreateForkRequest)(nil), // 31: gitaly.CreateForkRequest
+ (*CreateForkResponse)(nil), // 32: gitaly.CreateForkResponse
+ (*CreateRepositoryFromURLRequest)(nil), // 33: gitaly.CreateRepositoryFromURLRequest
+ (*CreateRepositoryFromURLResponse)(nil), // 34: gitaly.CreateRepositoryFromURLResponse
+ (*CreateBundleRequest)(nil), // 35: gitaly.CreateBundleRequest
+ (*CreateBundleResponse)(nil), // 36: gitaly.CreateBundleResponse
+ (*CreateBundleFromRefListRequest)(nil), // 37: gitaly.CreateBundleFromRefListRequest
+ (*CreateBundleFromRefListResponse)(nil), // 38: gitaly.CreateBundleFromRefListResponse
+ (*GetConfigRequest)(nil), // 39: gitaly.GetConfigRequest
+ (*GetConfigResponse)(nil), // 40: gitaly.GetConfigResponse
+ (*RestoreCustomHooksRequest)(nil), // 41: gitaly.RestoreCustomHooksRequest
+ (*SetCustomHooksRequest)(nil), // 42: gitaly.SetCustomHooksRequest
+ (*RestoreCustomHooksResponse)(nil), // 43: gitaly.RestoreCustomHooksResponse
+ (*SetCustomHooksResponse)(nil), // 44: gitaly.SetCustomHooksResponse
+ (*BackupCustomHooksRequest)(nil), // 45: gitaly.BackupCustomHooksRequest
+ (*GetCustomHooksRequest)(nil), // 46: gitaly.GetCustomHooksRequest
+ (*BackupCustomHooksResponse)(nil), // 47: gitaly.BackupCustomHooksResponse
+ (*GetCustomHooksResponse)(nil), // 48: gitaly.GetCustomHooksResponse
+ (*CreateRepositoryFromBundleRequest)(nil), // 49: gitaly.CreateRepositoryFromBundleRequest
+ (*CreateRepositoryFromBundleResponse)(nil), // 50: gitaly.CreateRepositoryFromBundleResponse
+ (*FindLicenseRequest)(nil), // 51: gitaly.FindLicenseRequest
+ (*FindLicenseResponse)(nil), // 52: gitaly.FindLicenseResponse
+ (*GetInfoAttributesRequest)(nil), // 53: gitaly.GetInfoAttributesRequest
+ (*GetInfoAttributesResponse)(nil), // 54: gitaly.GetInfoAttributesResponse
+ (*CalculateChecksumRequest)(nil), // 55: gitaly.CalculateChecksumRequest
+ (*CalculateChecksumResponse)(nil), // 56: gitaly.CalculateChecksumResponse
+ (*GetSnapshotRequest)(nil), // 57: gitaly.GetSnapshotRequest
+ (*GetSnapshotResponse)(nil), // 58: gitaly.GetSnapshotResponse
+ (*CreateRepositoryFromSnapshotRequest)(nil), // 59: gitaly.CreateRepositoryFromSnapshotRequest
+ (*CreateRepositoryFromSnapshotResponse)(nil), // 60: gitaly.CreateRepositoryFromSnapshotResponse
+ (*GetRawChangesRequest)(nil), // 61: gitaly.GetRawChangesRequest
+ (*GetRawChangesResponse)(nil), // 62: gitaly.GetRawChangesResponse
+ (*SearchFilesByNameRequest)(nil), // 63: gitaly.SearchFilesByNameRequest
+ (*SearchFilesByNameResponse)(nil), // 64: gitaly.SearchFilesByNameResponse
+ (*SearchFilesByContentRequest)(nil), // 65: gitaly.SearchFilesByContentRequest
+ (*SearchFilesByContentResponse)(nil), // 66: gitaly.SearchFilesByContentResponse
+ (*Remote)(nil), // 67: gitaly.Remote
+ (*GetObjectDirectorySizeRequest)(nil), // 68: gitaly.GetObjectDirectorySizeRequest
+ (*GetObjectDirectorySizeResponse)(nil), // 69: gitaly.GetObjectDirectorySizeResponse
+ (*RemoveRepositoryRequest)(nil), // 70: gitaly.RemoveRepositoryRequest
+ (*RemoveRepositoryResponse)(nil), // 71: gitaly.RemoveRepositoryResponse
+ (*ReplicateRepositoryRequest)(nil), // 72: gitaly.ReplicateRepositoryRequest
+ (*ReplicateRepositoryResponse)(nil), // 73: gitaly.ReplicateRepositoryResponse
+ (*OptimizeRepositoryRequest)(nil), // 74: gitaly.OptimizeRepositoryRequest
+ (*OptimizeRepositoryResponse)(nil), // 75: gitaly.OptimizeRepositoryResponse
+ (*PruneUnreachableObjectsRequest)(nil), // 76: gitaly.PruneUnreachableObjectsRequest
+ (*PruneUnreachableObjectsResponse)(nil), // 77: gitaly.PruneUnreachableObjectsResponse
+ (*SetFullPathRequest)(nil), // 78: gitaly.SetFullPathRequest
+ (*SetFullPathResponse)(nil), // 79: gitaly.SetFullPathResponse
+ (*FullPathRequest)(nil), // 80: gitaly.FullPathRequest
+ (*FullPathResponse)(nil), // 81: gitaly.FullPathResponse
+ (*RemoveAllRequest)(nil), // 82: gitaly.RemoveAllRequest
+ (*RemoveAllResponse)(nil), // 83: gitaly.RemoveAllResponse
+ (*BackupRepositoryRequest)(nil), // 84: gitaly.BackupRepositoryRequest
+ (*BackupRepositoryResponse)(nil), // 85: gitaly.BackupRepositoryResponse
+ (*RestoreRepositoryRequest)(nil), // 86: gitaly.RestoreRepositoryRequest
+ (*RestoreRepositoryResponse)(nil), // 87: gitaly.RestoreRepositoryResponse
+ (*GetFileAttributesRequest)(nil), // 88: gitaly.GetFileAttributesRequest
+ (*GetFileAttributesResponse)(nil), // 89: gitaly.GetFileAttributesResponse
+ (*RepositoryInfoResponse_ReferencesInfo)(nil), // 90: gitaly.RepositoryInfoResponse.ReferencesInfo
+ (*RepositoryInfoResponse_ObjectsInfo)(nil), // 91: gitaly.RepositoryInfoResponse.ObjectsInfo
+ (*GetRawChangesResponse_RawChange)(nil), // 92: gitaly.GetRawChangesResponse.RawChange
+ (*BackupRepositoryResponse_SkippedError)(nil), // 93: gitaly.BackupRepositoryResponse.SkippedError
+ (*RestoreRepositoryResponse_SkippedError)(nil), // 94: gitaly.RestoreRepositoryResponse.SkippedError
+ (*GetFileAttributesResponse_AttributeInfo)(nil), // 95: gitaly.GetFileAttributesResponse.AttributeInfo
+ (*Repository)(nil), // 96: gitaly.Repository
+ (ObjectFormat)(0), // 97: gitaly.ObjectFormat
}
var file_repository_proto_depIdxs = []int32{
- 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
- 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
- 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
+ 96, // 0: gitaly.RepositoryExistsRequest.repository:type_name -> gitaly.Repository
+ 96, // 1: gitaly.RepositorySizeRequest.repository:type_name -> gitaly.Repository
+ 96, // 2: gitaly.RepositoryInfoRequest.repository:type_name -> gitaly.Repository
+ 90, // 3: gitaly.RepositoryInfoResponse.references:type_name -> gitaly.RepositoryInfoResponse.ReferencesInfo
+ 91, // 4: gitaly.RepositoryInfoResponse.objects:type_name -> gitaly.RepositoryInfoResponse.ObjectsInfo
+ 96, // 5: gitaly.ObjectsSizeRequest.repository:type_name -> gitaly.Repository
+ 96, // 6: gitaly.ObjectFormatRequest.repository:type_name -> gitaly.Repository
+ 97, // 7: gitaly.ObjectFormatResponse.format:type_name -> gitaly.ObjectFormat
+ 96, // 8: gitaly.FetchBundleRequest.repository:type_name -> gitaly.Repository
+ 96, // 9: gitaly.FetchRemoteRequest.repository:type_name -> gitaly.Repository
+ 67, // 10: gitaly.FetchRemoteRequest.remote_params:type_name -> gitaly.Remote
+ 96, // 11: gitaly.CreateRepositoryRequest.repository:type_name -> gitaly.Repository
+ 97, // 12: gitaly.CreateRepositoryRequest.object_format:type_name -> gitaly.ObjectFormat
+ 96, // 13: gitaly.GetArchiveRequest.repository:type_name -> gitaly.Repository
+ 0, // 14: gitaly.GetArchiveRequest.format:type_name -> gitaly.GetArchiveRequest.Format
+ 96, // 15: gitaly.HasLocalBranchesRequest.repository:type_name -> gitaly.Repository
+ 96, // 16: gitaly.FetchSourceBranchRequest.repository:type_name -> gitaly.Repository
+ 96, // 17: gitaly.FetchSourceBranchRequest.source_repository:type_name -> gitaly.Repository
+ 96, // 18: gitaly.FsckRequest.repository:type_name -> gitaly.Repository
+ 96, // 19: gitaly.WriteRefRequest.repository:type_name -> gitaly.Repository
+ 96, // 20: gitaly.FindMergeBaseRequest.repository:type_name -> gitaly.Repository
+ 96, // 21: gitaly.CreateForkRequest.repository:type_name -> gitaly.Repository
+ 96, // 22: gitaly.CreateForkRequest.source_repository:type_name -> gitaly.Repository
+ 96, // 23: gitaly.CreateRepositoryFromURLRequest.repository:type_name -> gitaly.Repository
+ 96, // 24: gitaly.CreateBundleRequest.repository:type_name -> gitaly.Repository
+ 96, // 25: gitaly.CreateBundleFromRefListRequest.repository:type_name -> gitaly.Repository
+ 96, // 26: gitaly.GetConfigRequest.repository:type_name -> gitaly.Repository
+ 96, // 27: gitaly.RestoreCustomHooksRequest.repository:type_name -> gitaly.Repository
+ 96, // 28: gitaly.SetCustomHooksRequest.repository:type_name -> gitaly.Repository
+ 96, // 29: gitaly.BackupCustomHooksRequest.repository:type_name -> gitaly.Repository
+ 96, // 30: gitaly.GetCustomHooksRequest.repository:type_name -> gitaly.Repository
+ 96, // 31: gitaly.CreateRepositoryFromBundleRequest.repository:type_name -> gitaly.Repository
+ 96, // 32: gitaly.FindLicenseRequest.repository:type_name -> gitaly.Repository
+ 96, // 33: gitaly.GetInfoAttributesRequest.repository:type_name -> gitaly.Repository
+ 96, // 34: gitaly.CalculateChecksumRequest.repository:type_name -> gitaly.Repository
+ 96, // 35: gitaly.GetSnapshotRequest.repository:type_name -> gitaly.Repository
+ 96, // 36: gitaly.CreateRepositoryFromSnapshotRequest.repository:type_name -> gitaly.Repository
+ 96, // 37: gitaly.GetRawChangesRequest.repository:type_name -> gitaly.Repository
+ 92, // 38: gitaly.GetRawChangesResponse.raw_changes:type_name -> gitaly.GetRawChangesResponse.RawChange
+ 96, // 39: gitaly.SearchFilesByNameRequest.repository:type_name -> gitaly.Repository
+ 96, // 40: gitaly.SearchFilesByContentRequest.repository:type_name -> gitaly.Repository
+ 96, // 41: gitaly.GetObjectDirectorySizeRequest.repository:type_name -> gitaly.Repository
+ 96, // 42: gitaly.RemoveRepositoryRequest.repository:type_name -> gitaly.Repository
+ 96, // 43: gitaly.ReplicateRepositoryRequest.repository:type_name -> gitaly.Repository
+ 96, // 44: gitaly.ReplicateRepositoryRequest.source:type_name -> gitaly.Repository
+ 96, // 45: gitaly.OptimizeRepositoryRequest.repository:type_name -> gitaly.Repository
+ 2, // 46: gitaly.OptimizeRepositoryRequest.strategy:type_name -> gitaly.OptimizeRepositoryRequest.Strategy
+ 96, // 47: gitaly.PruneUnreachableObjectsRequest.repository:type_name -> gitaly.Repository
+ 96, // 48: gitaly.SetFullPathRequest.repository:type_name -> gitaly.Repository
+ 96, // 49: gitaly.FullPathRequest.repository:type_name -> gitaly.Repository
+ 96, // 50: gitaly.BackupRepositoryRequest.repository:type_name -> gitaly.Repository
+ 96, // 51: gitaly.BackupRepositoryRequest.vanity_repository:type_name -> gitaly.Repository
+ 96, // 52: gitaly.RestoreRepositoryRequest.repository:type_name -> gitaly.Repository
+ 96, // 53: gitaly.RestoreRepositoryRequest.vanity_repository:type_name -> gitaly.Repository
+ 96, // 54: gitaly.GetFileAttributesRequest.repository:type_name -> gitaly.Repository
+ 95, // 55: gitaly.GetFileAttributesResponse.attribute_infos:type_name -> gitaly.GetFileAttributesResponse.AttributeInfo
+ 1, // 56: gitaly.GetRawChangesResponse.RawChange.operation:type_name -> gitaly.GetRawChangesResponse.RawChange.Operation
+ 3, // 57: gitaly.RepositoryService.RepositoryExists:input_type -> gitaly.RepositoryExistsRequest
+ 5, // 58: gitaly.RepositoryService.RepositorySize:input_type -> gitaly.RepositorySizeRequest
+ 7, // 59: gitaly.RepositoryService.RepositoryInfo:input_type -> gitaly.RepositoryInfoRequest
+ 9, // 60: gitaly.RepositoryService.ObjectsSize:input_type -> gitaly.ObjectsSizeRequest
+ 11, // 61: gitaly.RepositoryService.ObjectFormat:input_type -> gitaly.ObjectFormatRequest
+ 15, // 62: gitaly.RepositoryService.FetchRemote:input_type -> gitaly.FetchRemoteRequest
+ 17, // 63: gitaly.RepositoryService.CreateRepository:input_type -> gitaly.CreateRepositoryRequest
+ 19, // 64: gitaly.RepositoryService.GetArchive:input_type -> gitaly.GetArchiveRequest
+ 21, // 65: gitaly.RepositoryService.HasLocalBranches:input_type -> gitaly.HasLocalBranchesRequest
+ 23, // 66: gitaly.RepositoryService.FetchSourceBranch:input_type -> gitaly.FetchSourceBranchRequest
+ 25, // 67: gitaly.RepositoryService.Fsck:input_type -> gitaly.FsckRequest
+ 27, // 68: gitaly.RepositoryService.WriteRef:input_type -> gitaly.WriteRefRequest
+ 29, // 69: gitaly.RepositoryService.FindMergeBase:input_type -> gitaly.FindMergeBaseRequest
+ 31, // 70: gitaly.RepositoryService.CreateFork:input_type -> gitaly.CreateForkRequest
+ 33, // 71: gitaly.RepositoryService.CreateRepositoryFromURL:input_type -> gitaly.CreateRepositoryFromURLRequest
+ 35, // 72: gitaly.RepositoryService.CreateBundle:input_type -> gitaly.CreateBundleRequest
+ 37, // 73: gitaly.RepositoryService.CreateBundleFromRefList:input_type -> gitaly.CreateBundleFromRefListRequest
+ 13, // 74: gitaly.RepositoryService.FetchBundle:input_type -> gitaly.FetchBundleRequest
+ 49, // 75: gitaly.RepositoryService.CreateRepositoryFromBundle:input_type -> gitaly.CreateRepositoryFromBundleRequest
+ 39, // 76: gitaly.RepositoryService.GetConfig:input_type -> gitaly.GetConfigRequest
+ 51, // 77: gitaly.RepositoryService.FindLicense:input_type -> gitaly.FindLicenseRequest
+ 53, // 78: gitaly.RepositoryService.GetInfoAttributes:input_type -> gitaly.GetInfoAttributesRequest
+ 55, // 79: gitaly.RepositoryService.CalculateChecksum:input_type -> gitaly.CalculateChecksumRequest
+ 57, // 80: gitaly.RepositoryService.GetSnapshot:input_type -> gitaly.GetSnapshotRequest
+ 59, // 81: gitaly.RepositoryService.CreateRepositoryFromSnapshot:input_type -> gitaly.CreateRepositoryFromSnapshotRequest
+ 61, // 82: gitaly.RepositoryService.GetRawChanges:input_type -> gitaly.GetRawChangesRequest
+ 65, // 83: gitaly.RepositoryService.SearchFilesByContent:input_type -> gitaly.SearchFilesByContentRequest
+ 63, // 84: gitaly.RepositoryService.SearchFilesByName:input_type -> gitaly.SearchFilesByNameRequest
+ 41, // 85: gitaly.RepositoryService.RestoreCustomHooks:input_type -> gitaly.RestoreCustomHooksRequest
+ 42, // 86: gitaly.RepositoryService.SetCustomHooks:input_type -> gitaly.SetCustomHooksRequest
+ 45, // 87: gitaly.RepositoryService.BackupCustomHooks:input_type -> gitaly.BackupCustomHooksRequest
+ 46, // 88: gitaly.RepositoryService.GetCustomHooks:input_type -> gitaly.GetCustomHooksRequest
+ 68, // 89: gitaly.RepositoryService.GetObjectDirectorySize:input_type -> gitaly.GetObjectDirectorySizeRequest
+ 70, // 90: gitaly.RepositoryService.RemoveRepository:input_type -> gitaly.RemoveRepositoryRequest
+ 72, // 91: gitaly.RepositoryService.ReplicateRepository:input_type -> gitaly.ReplicateRepositoryRequest
+ 74, // 92: gitaly.RepositoryService.OptimizeRepository:input_type -> gitaly.OptimizeRepositoryRequest
+ 76, // 93: gitaly.RepositoryService.PruneUnreachableObjects:input_type -> gitaly.PruneUnreachableObjectsRequest
+ 78, // 94: gitaly.RepositoryService.SetFullPath:input_type -> gitaly.SetFullPathRequest
+ 80, // 95: gitaly.RepositoryService.FullPath:input_type -> gitaly.FullPathRequest
+ 82, // 96: gitaly.RepositoryService.RemoveAll:input_type -> gitaly.RemoveAllRequest
+ 84, // 97: gitaly.RepositoryService.BackupRepository:input_type -> gitaly.BackupRepositoryRequest
+ 86, // 98: gitaly.RepositoryService.RestoreRepository:input_type -> gitaly.RestoreRepositoryRequest
+ 88, // 99: gitaly.RepositoryService.GetFileAttributes:input_type -> gitaly.GetFileAttributesRequest
+ 4, // 100: gitaly.RepositoryService.RepositoryExists:output_type -> gitaly.RepositoryExistsResponse
+ 6, // 101: gitaly.RepositoryService.RepositorySize:output_type -> gitaly.RepositorySizeResponse
+ 8, // 102: gitaly.RepositoryService.RepositoryInfo:output_type -> gitaly.RepositoryInfoResponse
+ 10, // 103: gitaly.RepositoryService.ObjectsSize:output_type -> gitaly.ObjectsSizeResponse
+ 12, // 104: gitaly.RepositoryService.ObjectFormat:output_type -> gitaly.ObjectFormatResponse
+ 16, // 105: gitaly.RepositoryService.FetchRemote:output_type -> gitaly.FetchRemoteResponse
+ 18, // 106: gitaly.RepositoryService.CreateRepository:output_type -> gitaly.CreateRepositoryResponse
+ 20, // 107: gitaly.RepositoryService.GetArchive:output_type -> gitaly.GetArchiveResponse
+ 22, // 108: gitaly.RepositoryService.HasLocalBranches:output_type -> gitaly.HasLocalBranchesResponse
+ 24, // 109: gitaly.RepositoryService.FetchSourceBranch:output_type -> gitaly.FetchSourceBranchResponse
+ 26, // 110: gitaly.RepositoryService.Fsck:output_type -> gitaly.FsckResponse
+ 28, // 111: gitaly.RepositoryService.WriteRef:output_type -> gitaly.WriteRefResponse
+ 30, // 112: gitaly.RepositoryService.FindMergeBase:output_type -> gitaly.FindMergeBaseResponse
+ 32, // 113: gitaly.RepositoryService.CreateFork:output_type -> gitaly.CreateForkResponse
+ 34, // 114: gitaly.RepositoryService.CreateRepositoryFromURL:output_type -> gitaly.CreateRepositoryFromURLResponse
+ 36, // 115: gitaly.RepositoryService.CreateBundle:output_type -> gitaly.CreateBundleResponse
+ 38, // 116: gitaly.RepositoryService.CreateBundleFromRefList:output_type -> gitaly.CreateBundleFromRefListResponse
+ 14, // 117: gitaly.RepositoryService.FetchBundle:output_type -> gitaly.FetchBundleResponse
+ 50, // 118: gitaly.RepositoryService.CreateRepositoryFromBundle:output_type -> gitaly.CreateRepositoryFromBundleResponse
+ 40, // 119: gitaly.RepositoryService.GetConfig:output_type -> gitaly.GetConfigResponse
+ 52, // 120: gitaly.RepositoryService.FindLicense:output_type -> gitaly.FindLicenseResponse
+ 54, // 121: gitaly.RepositoryService.GetInfoAttributes:output_type -> gitaly.GetInfoAttributesResponse
+ 56, // 122: gitaly.RepositoryService.CalculateChecksum:output_type -> gitaly.CalculateChecksumResponse
+ 58, // 123: gitaly.RepositoryService.GetSnapshot:output_type -> gitaly.GetSnapshotResponse
+ 60, // 124: gitaly.RepositoryService.CreateRepositoryFromSnapshot:output_type -> gitaly.CreateRepositoryFromSnapshotResponse
+ 62, // 125: gitaly.RepositoryService.GetRawChanges:output_type -> gitaly.GetRawChangesResponse
+ 66, // 126: gitaly.RepositoryService.SearchFilesByContent:output_type -> gitaly.SearchFilesByContentResponse
+ 64, // 127: gitaly.RepositoryService.SearchFilesByName:output_type -> gitaly.SearchFilesByNameResponse
+ 43, // 128: gitaly.RepositoryService.RestoreCustomHooks:output_type -> gitaly.RestoreCustomHooksResponse
+ 44, // 129: gitaly.RepositoryService.SetCustomHooks:output_type -> gitaly.SetCustomHooksResponse
+ 47, // 130: gitaly.RepositoryService.BackupCustomHooks:output_type -> gitaly.BackupCustomHooksResponse
+ 48, // 131: gitaly.RepositoryService.GetCustomHooks:output_type -> gitaly.GetCustomHooksResponse
+ 69, // 132: gitaly.RepositoryService.GetObjectDirectorySize:output_type -> gitaly.GetObjectDirectorySizeResponse
+ 71, // 133: gitaly.RepositoryService.RemoveRepository:output_type -> gitaly.RemoveRepositoryResponse
+ 73, // 134: gitaly.RepositoryService.ReplicateRepository:output_type -> gitaly.ReplicateRepositoryResponse
+ 75, // 135: gitaly.RepositoryService.OptimizeRepository:output_type -> gitaly.OptimizeRepositoryResponse
+ 77, // 136: gitaly.RepositoryService.PruneUnreachableObjects:output_type -> gitaly.PruneUnreachableObjectsResponse
+ 79, // 137: gitaly.RepositoryService.SetFullPath:output_type -> gitaly.SetFullPathResponse
+ 81, // 138: gitaly.RepositoryService.FullPath:output_type -> gitaly.FullPathResponse
+ 83, // 139: gitaly.RepositoryService.RemoveAll:output_type -> gitaly.RemoveAllResponse
+ 85, // 140: gitaly.RepositoryService.BackupRepository:output_type -> gitaly.BackupRepositoryResponse
+ 87, // 141: gitaly.RepositoryService.RestoreRepository:output_type -> gitaly.RestoreRepositoryResponse
+ 89, // 142: gitaly.RepositoryService.GetFileAttributes:output_type -> gitaly.GetFileAttributesResponse
+ 100, // [100:143] is the sub-list for method output_type
+ 57, // [57:100] is the sub-list for method input_type
+ 57, // [57:57] is the sub-list for extension type_name
+ 57, // [57:57] is the sub-list for extension extendee
+ 0, // [0:57] is the sub-list for field type_name
}
func init() { file_repository_proto_init() }
@@ -6846,30 +6729,6 @@ func file_repository_proto_init() {
}
}
file_repository_proto_msgTypes[10].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*ApplyGitattributesRequest); i {
- case 0:
- return &v.state
- case 1:
- return &v.sizeCache
- case 2:
- return &v.unknownFields
- default:
- return nil
- }
- }
- file_repository_proto_msgTypes[11].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*ApplyGitattributesResponse); i {
- case 0:
- return &v.state
- case 1:
- return &v.sizeCache
- case 2:
- return &v.unknownFields
- default:
- return nil
- }
- }
- file_repository_proto_msgTypes[12].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*FetchBundleRequest); i {
case 0:
return &v.state
@@ -6881,7 +6740,7 @@ func file_repository_proto_init() {
return nil
}
}
- file_repository_proto_msgTypes[13].Exporter = func(v interface{}, i int) interface{} {
+ file_repository_proto_msgTypes[11].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*FetchBundleResponse); i {
case 0:
return &v.state
@@ -6893,7 +6752,7 @@ func file_repository_proto_init() {
return nil
}
}
- file_repository_proto_msgTypes[14].Exporter = func(v interface{}, i int) interface{} {
+ file_repository_proto_msgTypes[12].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*FetchRemoteRequest); i {
case 0:
return &v.state
@@ -6905,7 +6764,7 @@ func file_repository_proto_init() {
return nil
}
}
- file_repository_proto_msgTypes[15].Exporter = func(v interface{}, i int) interface{} {
+ file_repository_proto_msgTypes[13].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*FetchRemoteResponse); i {
case 0:
return &v.state
@@ -6917,7 +6776,7 @@ func file_repository_proto_init() {
return nil
}
}
- file_repository_proto_msgTypes[16].Exporter = func(v interface{}, i int) interface{} {
+ file_repository_proto_msgTypes[14].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*CreateRepositoryRequest); i {
case 0:
return &v.state
@@ -6929,7 +6788,7 @@ func file_repository_proto_init() {
return nil
}
}
- file_repository_proto_msgTypes[17].Exporter = func(v interface{}, i int) interface{} {
+ file_repository_proto_msgTypes[15].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*CreateRepositoryResponse); i {
case 0:
return &v.state
@@ -6941,7 +6800,7 @@ func file_repository_proto_init() {
return nil
}
}
- file_repository_proto_msgTypes[18].Exporter = func(v interface{}, i int) interface{} {
+ file_repository_proto_msgTypes[16].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*GetArchiveRequest); i {
case 0:
return &v.state
@@ -6953,7 +6812,7 @@ func file_repository_proto_init() {
return nil
}
}
- file_repository_proto_msgTypes[19].Exporter = func(v interface{}, i int) interface{} {
+ file_repository_proto_msgTypes[17].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*GetArchiveResponse); i {
case 0:
return &v.state
@@ -6965,7 +6824,7 @@ func file_repository_proto_init() {
return nil
}
}
- file_repository_proto_msgTypes[20].Exporter = func(v interface{}, i int) interface{} {
+ file_repository_proto_msgTypes[18].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*HasLocalBranchesRequest); i {
case 0:
return &v.state
@@ -6977,7 +6836,7 @@ func file_repository_proto_init() {
return nil
}
}
- file_repository_proto_msgTypes[21].Exporter = func(v interface{}, i int) interface{} {
+ file_repository_proto_msgTypes[19].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*HasLocalBranchesResponse); i {
case 0:
return &v.state
@@ -6989,7 +6848,7 @@ func file_repository_proto_init() {
return nil
}
}
- file_repository_proto_msgTypes[22].Exporter = func(v interface{}, i int) interface{} {
+ file_repository_proto_msgTypes[20].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*FetchSourceBranchRequest); i {
case 0:
return &v.state
@@ -7001,7 +6860,7 @@ func file_repository_proto_init() {
return nil
}
}
- file_repository_proto_msgTypes[23].Exporter = func(v interface{}, i int) interface{} {
+ file_repository_proto_msgTypes[21].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*FetchSourceBranchResponse); i {
case 0:
return &v.state
@@ -7013,7 +6872,7 @@ func file_repository_proto_init() {
return nil
}
}
- file_repository_proto_msgTypes[24].Exporter = func(v interface{}, i int) interface{} {
+ file_repository_proto_msgTypes[22].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*FsckRequest); i {
case 0:
return &v.state
@@ -7025,7 +6884,7 @@ func file_repository_proto_init() {
return nil
}
}
- file_repository_proto_msgTypes[25].Exporter = func(v interface{}, i int) interface{} {
+ file_repository_proto_msgTypes[23].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*FsckResponse); i {
case 0:
return &v.state
@@ -7037,7 +6896,7 @@ func file_repository_proto_init() {
return nil
}
}
- file_repository_proto_msgTypes[26].Exporter = func(v interface{}, i int) interface{} {
+ file_repository_proto_msgTypes[24].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*WriteRefRequest); i {
case 0:
return &v.state
@@ -7049,7 +6908,7 @@ func file_repository_proto_init() {
return nil
}
}
- file_repository_proto_msgTypes[27].Exporter = func(v interface{}, i int) interface{} {
+ file_repository_proto_msgTypes[25].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*WriteRefResponse); i {
case 0:
return &v.state
@@ -7061,7 +6920,7 @@ func file_repository_proto_init() {
return nil
}
}
- file_repository_proto_msgTypes[28].Exporter = func(v interface{}, i int) interface{} {
+ file_repository_proto_msgTypes[26].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*FindMergeBaseRequest); i {
case 0:
return &v.state
@@ -7073,7 +6932,7 @@ func file_repository_proto_init() {
return nil
}
}
- file_repository_proto_msgTypes[29].Exporter = func(v interface{}, i int) interface{} {
+ file_repository_proto_msgTypes[27].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*FindMergeBaseResponse); i {
case 0:
return &v.state
@@ -7085,7 +6944,7 @@ func file_repository_proto_init() {
return nil
}
}
- file_repository_proto_msgTypes[30].Exporter = func(v interface{}, i int) interface{} {
+ file_repository_proto_msgTypes[28].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*CreateForkRequest); i {
case 0:
return &v.state
@@ -7097,7 +6956,7 @@ func file_repository_proto_init() {
return nil
}
}
- file_repository_proto_msgTypes[31].Exporter = func(v interface{}, i int) interface{} {
+ file_repository_proto_msgTypes[29].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*CreateForkResponse); i {
case 0:
return &v.state
@@ -7109,7 +6968,7 @@ func file_repository_proto_init() {
return nil
}
}
- file_repository_proto_msgTypes[32].Exporter = func(v interface{}, i int) interface{} {
+ file_repository_proto_msgTypes[30].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*CreateRepositoryFromURLRequest); i {
case 0:
return &v.state
@@ -7121,7 +6980,7 @@ func file_repository_proto_init() {
return nil
}
}
- file_repository_proto_msgTypes[33].Exporter = func(v interface{}, i int) interface{} {
+ file_repository_proto_msgTypes[31].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*CreateRepositoryFromURLResponse); i {
case 0:
return &v.state
@@ -7133,7 +6992,7 @@ func file_repository_proto_init() {
return nil
}
}
- file_repository_proto_msgTypes[34].Exporter = func(v interface{}, i int) interface{} {
+ file_repository_proto_msgTypes[32].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*CreateBundleRequest); i {
case 0:
return &v.state
@@ -7145,7 +7004,7 @@ func file_repository_proto_init() {
return nil
}
}
- file_repository_proto_msgTypes[35].Exporter = func(v interface{}, i int) interface{} {
+ file_repository_proto_msgTypes[33].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*CreateBundleResponse); i {
case 0:
return &v.state
@@ -7157,7 +7016,7 @@ func file_repository_proto_init() {
return nil
}
}
- file_repository_proto_msgTypes[36].Exporter = func(v interface{}, i int) interface{} {
+ file_repository_proto_msgTypes[34].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*CreateBundleFromRefListRequest); i {
case 0:
return &v.state
@@ -7169,7 +7028,7 @@ func file_repository_proto_init() {
return nil
}
}
- file_repository_proto_msgTypes[37].Exporter = func(v interface{}, i int) interface{} {
+ file_repository_proto_msgTypes[35].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*CreateBundleFromRefListResponse); i {
case 0:
return &v.state
@@ -7181,7 +7040,7 @@ func file_repository_proto_init() {
return nil
}
}
- file_repository_proto_msgTypes[38].Exporter = func(v interface{}, i int) interface{} {
+ file_repository_proto_msgTypes[36].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*GetConfigRequest); i {
case 0:
return &v.state
@@ -7193,7 +7052,7 @@ func file_repository_proto_init() {
return nil
}
}
- file_repository_proto_msgTypes[39].Exporter = func(v interface{}, i int) interface{} {
+ file_repository_proto_msgTypes[37].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*GetConfigResponse); i {
case 0:
return &v.state
@@ -7205,7 +7064,7 @@ func file_repository_proto_init() {
return nil
}
}
- file_repository_proto_msgTypes[40].Exporter = func(v interface{}, i int) interface{} {
+ file_repository_proto_msgTypes[38].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*RestoreCustomHooksRequest); i {
case 0:
return &v.state
@@ -7217,7 +7076,7 @@ func file_repository_proto_init() {
return nil
}
}
- file_repository_proto_msgTypes[41].Exporter = func(v interface{}, i int) interface{} {
+ file_repository_proto_msgTypes[39].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*SetCustomHooksRequest); i {
case 0:
return &v.state
@@ -7229,7 +7088,7 @@ func file_repository_proto_init() {
return nil
}
}
- file_repository_proto_msgTypes[42].Exporter = func(v interface{}, i int) interface{} {
+ file_repository_proto_msgTypes[40].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*RestoreCustomHooksResponse); i {
case 0:
return &v.state
@@ -7241,7 +7100,7 @@ func file_repository_proto_init() {
return nil
}
}
- file_repository_proto_msgTypes[43].Exporter = func(v interface{}, i int) interface{} {
+ file_repository_proto_msgTypes[41].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*SetCustomHooksResponse); i {
case 0:
return &v.state
@@ -7253,7 +7112,7 @@ func file_repository_proto_init() {
return nil
}
}
- file_repository_proto_msgTypes[44].Exporter = func(v interface{}, i int) interface{} {
+ file_repository_proto_msgTypes[42].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*BackupCustomHooksRequest); i {
case 0:
return &v.state
@@ -7265,7 +7124,7 @@ func file_repository_proto_init() {
return nil
}
}
- file_repository_proto_msgTypes[45].Exporter = func(v interface{}, i int) interface{} {
+ file_repository_proto_msgTypes[43].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*GetCustomHooksRequest); i {
case 0:
return &v.state
@@ -7277,7 +7136,7 @@ func file_repository_proto_init() {
return nil
}
}
- file_repository_proto_msgTypes[46].Exporter = func(v interface{}, i int) interface{} {
+ file_repository_proto_msgTypes[44].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*BackupCustomHooksResponse); i {
case 0:
return &v.state
@@ -7289,7 +7148,7 @@ func file_repository_proto_init() {
return nil
}
}
- file_repository_proto_msgTypes[47].Exporter = func(v interface{}, i int) interface{} {
+ file_repository_proto_msgTypes[45].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*GetCustomHooksResponse); i {
case 0:
return &v.state
@@ -7301,7 +7160,7 @@ func file_repository_proto_init() {
return nil
}
}
- file_repository_proto_msgTypes[48].Exporter = func(v interface{}, i int) interface{} {
+ file_repository_proto_msgTypes[46].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*CreateRepositoryFromBundleRequest); i {
case 0:
return &v.state
@@ -7313,7 +7172,7 @@ func file_repository_proto_init() {
return nil
}
}
- file_repository_proto_msgTypes[49].Exporter = func(v interface{}, i int) interface{} {
+ file_repository_proto_msgTypes[47].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*CreateRepositoryFromBundleResponse); i {
case 0:
return &v.state
@@ -7325,7 +7184,7 @@ func file_repository_proto_init() {
return nil
}
}
- file_repository_proto_msgTypes[50].Exporter = func(v interface{}, i int) interface{} {
+ file_repository_proto_msgTypes[48].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*FindLicenseRequest); i {
case 0:
return &v.state
@@ -7337,7 +7196,7 @@ func file_repository_proto_init() {
return nil
}
}
- file_repository_proto_msgTypes[51].Exporter = func(v interface{}, i int) interface{} {
+ file_repository_proto_msgTypes[49].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*FindLicenseResponse); i {
case 0:
return &v.state
@@ -7349,7 +7208,7 @@ func file_repository_proto_init() {
return nil
}
}
- file_repository_proto_msgTypes[52].Exporter = func(v interface{}, i int) interface{} {
+ file_repository_proto_msgTypes[50].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*GetInfoAttributesRequest); i {
case 0:
return &v.state
@@ -7361,7 +7220,7 @@ func file_repository_proto_init() {
return nil
}
}
- file_repository_proto_msgTypes[53].Exporter = func(v interface{}, i int) interface{} {
+ file_repository_proto_msgTypes[51].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*GetInfoAttributesResponse); i {
case 0:
return &v.state
@@ -7373,7 +7232,7 @@ func file_repository_proto_init() {
return nil
}
}
- file_repository_proto_msgTypes[54].Exporter = func(v interface{}, i int) interface{} {
+ file_repository_proto_msgTypes[52].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*CalculateChecksumRequest); i {
case 0:
return &v.state
@@ -7385,7 +7244,7 @@ func file_repository_proto_init() {
return nil
}
}
- file_repository_proto_msgTypes[55].Exporter = func(v interface{}, i int) interface{} {
+ file_repository_proto_msgTypes[53].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*CalculateChecksumResponse); i {
case 0:
return &v.state
@@ -7397,7 +7256,7 @@ func file_repository_proto_init() {
return nil
}
}
- file_repository_proto_msgTypes[56].Exporter = func(v interface{}, i int) interface{} {
+ file_repository_proto_msgTypes[54].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*GetSnapshotRequest); i {
case 0:
return &v.state
@@ -7409,7 +7268,7 @@ func file_repository_proto_init() {
return nil
}
}
- file_repository_proto_msgTypes[57].Exporter = func(v interface{}, i int) interface{} {
+ file_repository_proto_msgTypes[55].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*GetSnapshotResponse); i {
case 0:
return &v.state
@@ -7421,7 +7280,7 @@ func file_repository_proto_init() {
return nil
}
}
- file_repository_proto_msgTypes[58].Exporter = func(v interface{}, i int) interface{} {
+ file_repository_proto_msgTypes[56].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*CreateRepositoryFromSnapshotRequest); i {
case 0:
return &v.state
@@ -7433,7 +7292,7 @@ func file_repository_proto_init() {
return nil
}
}
- file_repository_proto_msgTypes[59].Exporter = func(v interface{}, i int) interface{} {
+ file_repository_proto_msgTypes[57].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*CreateRepositoryFromSnapshotResponse); i {
case 0:
return &v.state
@@ -7445,7 +7304,7 @@ func file_repository_proto_init() {
return nil
}
}
- file_repository_proto_msgTypes[60].Exporter = func(v interface{}, i int) interface{} {
+ file_repository_proto_msgTypes[58].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*GetRawChangesRequest); i {
case 0:
return &v.state
@@ -7457,7 +7316,7 @@ func file_repository_proto_init() {
return nil
}
}
- file_repository_proto_msgTypes[61].Exporter = func(v interface{}, i int) interface{} {
+ file_repository_proto_msgTypes[59].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*GetRawChangesResponse); i {
case 0:
return &v.state
@@ -7469,7 +7328,7 @@ func file_repository_proto_init() {
return nil
}
}
- file_repository_proto_msgTypes[62].Exporter = func(v interface{}, i int) interface{} {
+ file_repository_proto_msgTypes[60].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*SearchFilesByNameRequest); i {
case 0:
return &v.state
@@ -7481,7 +7340,7 @@ func file_repository_proto_init() {
return nil
}
}
- file_repository_proto_msgTypes[63].Exporter = func(v interface{}, i int) interface{} {
+ file_repository_proto_msgTypes[61].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*SearchFilesByNameResponse); i {
case 0:
return &v.state
@@ -7493,7 +7352,7 @@ func file_repository_proto_init() {
return nil
}
}
- file_repository_proto_msgTypes[64].Exporter = func(v interface{}, i int) interface{} {
+ file_repository_proto_msgTypes[62].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*SearchFilesByContentRequest); i {
case 0:
return &v.state
@@ -7505,7 +7364,7 @@ func file_repository_proto_init() {
return nil
}
}
- file_repository_proto_msgTypes[65].Exporter = func(v interface{}, i int) interface{} {
+ file_repository_proto_msgTypes[63].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*SearchFilesByContentResponse); i {
case 0:
return &v.state
@@ -7517,7 +7376,7 @@ func file_repository_proto_init() {
return nil
}
}
- file_repository_proto_msgTypes[66].Exporter = func(v interface{}, i int) interface{} {
+ file_repository_proto_msgTypes[64].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*Remote); i {
case 0:
return &v.state
@@ -7529,7 +7388,7 @@ func file_repository_proto_init() {
return nil
}
}
- file_repository_proto_msgTypes[67].Exporter = func(v interface{}, i int) interface{} {
+ file_repository_proto_msgTypes[65].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*GetObjectDirectorySizeRequest); i {
case 0:
return &v.state
@@ -7541,7 +7400,7 @@ func file_repository_proto_init() {
return nil
}
}
- file_repository_proto_msgTypes[68].Exporter = func(v interface{}, i int) interface{} {
+ file_repository_proto_msgTypes[66].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*GetObjectDirectorySizeResponse); i {
case 0:
return &v.state
@@ -7553,7 +7412,7 @@ func file_repository_proto_init() {
return nil
}
}
- file_repository_proto_msgTypes[69].Exporter = func(v interface{}, i int) interface{} {
+ file_repository_proto_msgTypes[67].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*RemoveRepositoryRequest); i {
case 0:
return &v.state
@@ -7565,7 +7424,7 @@ func file_repository_proto_init() {
return nil
}
}
- file_repository_proto_msgTypes[70].Exporter = func(v interface{}, i int) interface{} {
+ file_repository_proto_msgTypes[68].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*RemoveRepositoryResponse); i {
case 0:
return &v.state
@@ -7577,7 +7436,7 @@ func file_repository_proto_init() {
return nil
}
}
- file_repository_proto_msgTypes[71].Exporter = func(v interface{}, i int) interface{} {
+ file_repository_proto_msgTypes[69].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*ReplicateRepositoryRequest); i {
case 0:
return &v.state
@@ -7589,7 +7448,7 @@ func file_repository_proto_init() {
return nil
}
}
- file_repository_proto_msgTypes[72].Exporter = func(v interface{}, i int) interface{} {
+ file_repository_proto_msgTypes[70].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*ReplicateRepositoryResponse); i {
case 0:
return &v.state
@@ -7601,7 +7460,7 @@ func file_repository_proto_init() {
return nil
}
}
- file_repository_proto_msgTypes[73].Exporter = func(v interface{}, i int) interface{} {
+ file_repository_proto_msgTypes[71].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*OptimizeRepositoryRequest); i {
case 0:
return &v.state
@@ -7613,7 +7472,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.(*OptimizeRepositoryResponse); i {
case 0:
return &v.state
@@ -7625,7 +7484,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.(*PruneUnreachableObjectsRequest); i {
case 0:
return &v.state
@@ -7637,7 +7496,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.(*PruneUnreachableObjectsResponse); i {
case 0:
return &v.state
@@ -7649,7 +7508,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.(*SetFullPathRequest); i {
case 0:
return &v.state
@@ -7661,7 +7520,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.(*SetFullPathResponse); i {
case 0:
return &v.state
@@ -7673,7 +7532,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.(*FullPathRequest); i {
case 0:
return &v.state
@@ -7685,7 +7544,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.(*FullPathResponse); i {
case 0:
return &v.state
@@ -7697,7 +7556,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.(*RemoveAllRequest); i {
case 0:
return &v.state
@@ -7709,7 +7568,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.(*RemoveAllResponse); i {
case 0:
return &v.state
@@ -7721,7 +7580,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.(*BackupRepositoryRequest); i {
case 0:
return &v.state
@@ -7733,7 +7592,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.(*BackupRepositoryResponse); i {
case 0:
return &v.state
@@ -7745,7 +7604,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.(*RestoreRepositoryRequest); i {
case 0:
return &v.state
@@ -7757,7 +7616,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.(*RestoreRepositoryResponse); i {
case 0:
return &v.state
@@ -7769,7 +7628,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.(*GetFileAttributesRequest); i {
case 0:
return &v.state
@@ -7781,7 +7640,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.(*GetFileAttributesResponse); i {
case 0:
return &v.state
@@ -7793,7 +7652,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.(*RepositoryInfoResponse_ReferencesInfo); i {
case 0:
return &v.state
@@ -7805,7 +7664,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.(*RepositoryInfoResponse_ObjectsInfo); i {
case 0:
return &v.state
@@ -7817,7 +7676,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.(*GetRawChangesResponse_RawChange); i {
case 0:
return &v.state
@@ -7829,7 +7688,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.(*BackupRepositoryResponse_SkippedError); i {
case 0:
return &v.state
@@ -7841,7 +7700,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.(*RestoreRepositoryResponse_SkippedError); i {
case 0:
return &v.state
@@ -7853,7 +7712,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.(*GetFileAttributesResponse_AttributeInfo); i {
case 0:
return &v.state
@@ -7872,7 +7731,7 @@ func file_repository_proto_init() {
GoPackagePath: reflect.TypeOf(x{}).PkgPath(),
RawDescriptor: file_repository_proto_rawDesc,
NumEnums: 3,
- NumMessages: 95,
+ NumMessages: 93,
NumExtensions: 0,
NumServices: 1,
},
diff --git a/proto/go/gitalypb/repository_grpc.pb.go b/proto/go/gitalypb/repository_grpc.pb.go
index b46e4ab3a..123f9dc79 100644
--- a/proto/go/gitalypb/repository_grpc.pb.go
+++ b/proto/go/gitalypb/repository_grpc.pb.go
@@ -55,10 +55,6 @@ type RepositoryServiceClient interface {
ObjectsSize(ctx context.Context, opts ...grpc.CallOption) (RepositoryService_ObjectsSizeClient, error)
// ObjectFormat determines the object format that is being used by the repository.
ObjectFormat(ctx context.Context, in *ObjectFormatRequest, opts ...grpc.CallOption) (*ObjectFormatResponse, error)
- // Deprecated: Do not use.
- // ApplyGitattributes writes the attributes from the given revision to info/attributes.
- // This RPC will be removed in 17.0.
- ApplyGitattributes(ctx context.Context, in *ApplyGitattributesRequest, opts ...grpc.CallOption) (*ApplyGitattributesResponse, error)
// FetchRemote fetches references from a remote repository into the local
// repository.
FetchRemote(ctx context.Context, in *FetchRemoteRequest, opts ...grpc.CallOption) (*FetchRemoteResponse, error)
@@ -276,16 +272,6 @@ func (c *repositoryServiceClient) ObjectFormat(ctx context.Context, in *ObjectFo
return out, nil
}
-// Deprecated: Do not use.
-func (c *repositoryServiceClient) ApplyGitattributes(ctx context.Context, in *ApplyGitattributesRequest, opts ...grpc.CallOption) (*ApplyGitattributesResponse, error) {
- out := new(ApplyGitattributesResponse)
- err := c.cc.Invoke(ctx, "/gitaly.RepositoryService/ApplyGitattributes", in, out, opts...)
- if err != nil {
- return nil, err
- }
- return out, nil
-}
-
func (c *repositoryServiceClient) FetchRemote(ctx context.Context, in *FetchRemoteRequest, opts ...grpc.CallOption) (*FetchRemoteResponse, error) {
out := new(FetchRemoteResponse)
err := c.cc.Invoke(ctx, "/gitaly.RepositoryService/FetchRemote", in, out, opts...)
@@ -1022,10 +1008,6 @@ type RepositoryServiceServer interface {
ObjectsSize(RepositoryService_ObjectsSizeServer) error
// ObjectFormat determines the object format that is being used by the repository.
ObjectFormat(context.Context, *ObjectFormatRequest) (*ObjectFormatResponse, error)
- // Deprecated: Do not use.
- // ApplyGitattributes writes the attributes from the given revision to info/attributes.
- // This RPC will be removed in 17.0.
- ApplyGitattributes(context.Context, *ApplyGitattributesRequest) (*ApplyGitattributesResponse, error)
// FetchRemote fetches references from a remote repository into the local
// repository.
FetchRemote(context.Context, *FetchRemoteRequest) (*FetchRemoteResponse, error)
@@ -1185,9 +1167,6 @@ func (UnimplementedRepositoryServiceServer) ObjectsSize(RepositoryService_Object
func (UnimplementedRepositoryServiceServer) ObjectFormat(context.Context, *ObjectFormatRequest) (*ObjectFormatResponse, error) {
return nil, status.Errorf(codes.Unimplemented, "method ObjectFormat not implemented")
}
-func (UnimplementedRepositoryServiceServer) ApplyGitattributes(context.Context, *ApplyGitattributesRequest) (*ApplyGitattributesResponse, error) {
- return nil, status.Errorf(codes.Unimplemented, "method ApplyGitattributes not implemented")
-}
func (UnimplementedRepositoryServiceServer) FetchRemote(context.Context, *FetchRemoteRequest) (*FetchRemoteResponse, error) {
return nil, status.Errorf(codes.Unimplemented, "method FetchRemote not implemented")
}
@@ -1413,24 +1392,6 @@ func _RepositoryService_ObjectFormat_Handler(srv interface{}, ctx context.Contex
return interceptor(ctx, in, info, handler)
}
-func _RepositoryService_ApplyGitattributes_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
- in := new(ApplyGitattributesRequest)
- if err := dec(in); err != nil {
- return nil, err
- }
- if interceptor == nil {
- return srv.(RepositoryServiceServer).ApplyGitattributes(ctx, in)
- }
- info := &grpc.UnaryServerInfo{
- Server: srv,
- FullMethod: "/gitaly.RepositoryService/ApplyGitattributes",
- }
- handler := func(ctx context.Context, req interface{}) (interface{}, error) {
- return srv.(RepositoryServiceServer).ApplyGitattributes(ctx, req.(*ApplyGitattributesRequest))
- }
- return interceptor(ctx, in, info, handler)
-}
-
func _RepositoryService_FetchRemote_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
in := new(FetchRemoteRequest)
if err := dec(in); err != nil {
@@ -2209,10 +2170,6 @@ var RepositoryService_ServiceDesc = grpc.ServiceDesc{
Handler: _RepositoryService_ObjectFormat_Handler,
},
{
- MethodName: "ApplyGitattributes",
- Handler: _RepositoryService_ApplyGitattributes_Handler,
- },
- {
MethodName: "FetchRemote",
Handler: _RepositoryService_FetchRemote_Handler,
},
diff --git a/proto/repository.proto b/proto/repository.proto
index 27c5ce666..44abaa2ae 100644
--- a/proto/repository.proto
+++ b/proto/repository.proto
@@ -68,15 +68,6 @@ service RepositoryService {
};
}
- // ApplyGitattributes writes the attributes from the given revision to info/attributes.
- // This RPC will be removed in 17.0.
- rpc ApplyGitattributes(ApplyGitattributesRequest) returns (ApplyGitattributesResponse) {
- option (op_type) = {
- op: MUTATOR
- };
- option deprecated = true;
- }
-
// FetchRemote fetches references from a remote repository into the local
// repository.
rpc FetchRemote(FetchRemoteRequest) returns (FetchRemoteResponse) {
@@ -529,18 +520,6 @@ message ObjectFormatResponse {
ObjectFormat format = 1;
}
-// ApplyGitattributesRequest ...
-message ApplyGitattributesRequest {
- // repository ...
- Repository repository = 1 [(target_repository)=true];
- // revision ...
- bytes revision = 2;
-}
-
-// ApplyGitattributesResponse ...
-message ApplyGitattributesResponse {
-}
-
// FetchBundleRequest ...
message FetchBundleRequest {
// repository into which the reference shall be fetched.