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:
authorJohn Cai <jcai@gitlab.com>2020-02-26 03:42:43 +0300
committerJohn Cai <jcai@gitlab.com>2020-02-26 04:55:19 +0300
commitf4824e3c191a5d529d4f266bfd349ca8def4a272 (patch)
treeda15d28697bfa625e0c8dd6f31481fd1c4acbe95
parent834272761e734d0e2d391ffb5d1a4a0685ae84f1 (diff)
with locking
-rw-r--r--internal/git/repository/repository.go72
-rw-r--r--internal/middleware/repositoryhandler/transactions.go142
-rw-r--r--internal/praefect/coordinator.go58
-rw-r--r--internal/praefect/grpc-proxy/proxy/handler.go28
-rw-r--r--internal/praefect/grpc-proxy/proxy/peeker.go2
-rw-r--r--internal/repotx/lock.go10
-rw-r--r--internal/server/server.go15
-rw-r--r--internal/service/register.go5
-rw-r--r--internal/service/repository/server.go4
-rw-r--r--internal/service/repository/write_ref.go31
-rw-r--r--proto/go/gitalypb/repository-service.pb.go407
-rw-r--r--proto/go/gitalypb/shared.pb.go214
-rw-r--r--proto/go/internal/linter/method.go53
-rw-r--r--proto/repository-service.proto4
-rw-r--r--proto/shared.proto7
-rw-r--r--ruby/proto/gitaly/repository-service_pb.rb4
-rw-r--r--ruby/proto/gitaly/shared_pb.rb11
17 files changed, 604 insertions, 463 deletions
diff --git a/internal/git/repository/repository.go b/internal/git/repository/repository.go
index 58b9d4ab9..c086c1b3e 100644
--- a/internal/git/repository/repository.go
+++ b/internal/git/repository/repository.go
@@ -8,7 +8,6 @@ import (
"github.com/sirupsen/logrus"
- "github.com/google/uuid"
"gitlab.com/gitlab-org/gitaly/proto/go/gitalypb"
"gitlab.com/gitlab-org/gitaly/internal/command"
@@ -37,14 +36,11 @@ type Transactions struct {
log *logrus.Entry
}
-func (t *Transactions) StartTransaction(repo *gitalypb.Repository, rollback command.Cmd) string {
+func (t *Transactions) NewTransaction(transactionID string, repo *gitalypb.Repository) {
tx := Transaction{
- Repo: repo,
- Rollback: rollback,
+ Repo: repo,
}
- transactionID := uuid.New().String()
-
t.txMutex.Lock()
defer t.txMutex.Unlock()
t.transactions[transactionID] = &tx
@@ -58,27 +54,60 @@ func (t *Transactions) StartTransaction(repo *gitalypb.Repository, rollback comm
t.repositories[repo.RelativePath].Lock()
t.repoMutex.Unlock()
- t.log.WithField("relative_path", repo.RelativePath)
+ t.log.WithField("relative_path", repo.RelativePath).Info("transaction created")
+}
+
+func (t *Transactions) Start(transactionID string, rollback command.Cmd) {
+ t.txMutex.Lock()
+ defer t.txMutex.Unlock()
- t.transactions[transactionID].Commit = t.repositories[repo.RelativePath].Unlock
+ _, ok := t.transactions[transactionID]
+ if !ok {
+ return
+ }
- return transactionID
+ t.transactions[transactionID].inProgress = true
+ t.transactions[transactionID].Rollback = rollback
}
func (t *Transactions) Commit(transactionID string) error {
t.txMutex.Lock()
defer t.txMutex.Unlock()
+ _, ok := t.transactions[transactionID]
+ if !ok {
+ return errors.New("request_id not found")
+ }
+
+ t.transactions[transactionID].inProgress = false
+
+ t.log.WithField("transaction_id", transactionID).Info("commited")
+ return nil
+}
+
+func (t *Transactions) Unlock(transactionID string) error {
+ t.txMutex.Lock()
+ defer t.txMutex.Unlock()
+
tx, ok := t.transactions[transactionID]
if !ok {
return errors.New("request_id not found")
}
- t.repoMutex.Lock()
- tx.Commit()
- t.repoMutex.Unlock()
+ if !tx.inProgress {
+ t.repoMutex.Lock()
+ defer t.repoMutex.Unlock()
+
+ repoLock, ok := t.repositories[tx.Repo.GetRelativePath()]
+ if !ok {
+ return nil
+ }
+ repoLock.Unlock()
+
+ delete(t.transactions, transactionID)
+ t.log.WithField("transaction_id", transactionID).Info("unlocked")
+ }
- delete(t.transactions, transactionID)
return nil
}
@@ -98,13 +127,24 @@ func (t *Transactions) Rollback(transactionID string) error {
return nil
}
+func (t *Transactions) TransactionStarted(transactionID string) bool {
+ t.txMutex.Lock()
+ defer t.txMutex.Unlock()
+
+ _, ok := t.transactions[transactionID]
+ if !ok {
+ return false
+ }
+ return true
+}
+
type Transaction struct {
// serviceName string
// methodName string
// transactionID string
- Repo GitRepo
- Rollback command.Cmd
- Commit func()
+ Repo GitRepo
+ Rollback command.Cmd
+ inProgress bool
}
type RepoLock interface {
diff --git a/internal/middleware/repositoryhandler/transactions.go b/internal/middleware/repositoryhandler/transactions.go
new file mode 100644
index 000000000..45c802fab
--- /dev/null
+++ b/internal/middleware/repositoryhandler/transactions.go
@@ -0,0 +1,142 @@
+package repositoryhandler
+
+import (
+ "context"
+ "errors"
+ "fmt"
+ "reflect"
+
+ "gitlab.com/gitlab-org/gitaly/proto/go/gitalypb"
+
+ "gitlab.com/gitlab-org/gitaly/internal/praefect/protoregistry"
+
+ "github.com/google/uuid"
+ "google.golang.org/grpc/metadata"
+
+ "gitlab.com/gitlab-org/gitaly/internal/git/repository"
+
+ "google.golang.org/grpc"
+)
+
+type RepositoryRequest interface {
+ GetRepository() *gitalypb.Repository
+}
+
+func RepositoryTransactionUnaryInterceptor(transactions *repository.Transactions, registry *protoregistry.Registry) grpc.UnaryServerInterceptor {
+ return func(ctx context.Context, req interface{}, info *grpc.UnaryServerInfo, handler grpc.UnaryHandler) (interface{}, error) {
+ mi, err := registry.LookupMethod(info.FullMethod)
+ if err != nil {
+ return nil, err
+ }
+
+ if mi.Scope == protoregistry.ScopeRepository && mi.Operation == protoregistry.OpMutator {
+ var transactionID string
+
+ md, ok := metadata.FromIncomingContext(ctx)
+ if ok && len(md.Get("transaction_id")) > 0 {
+ transactionID = md.Get("transaction_id")[0]
+ } else {
+ transactionID = uuid.New().String()
+
+ }
+
+ if !transactions.TransactionStarted(transactionID) {
+ repoReq, ok := req.(RepositoryRequest)
+ if !ok {
+ return nil, errors.New("not a repository request")
+ }
+
+ transactions.NewTransaction(transactionID, repoReq.GetRepository())
+ }
+
+ defer transactions.Unlock(transactionID)
+
+ ctx = metadata.NewIncomingContext(ctx, metadata.New(map[string]string{"transaction_id": transactionID}))
+
+ grpc.SetTrailer(ctx, metadata.New(map[string]string{"transaction_id": transactionID}))
+ }
+
+ res, err := handler(ctx, req)
+
+ return res, err
+ }
+}
+
+func NewTransactionServerStream(ss grpc.ServerStream, methodInfo protoregistry.MethodInfo, transactions *repository.Transactions, transactionID string) TransactionServerStream {
+ return TransactionServerStream{
+ transactionID: transactionID,
+ ss: ss,
+ mi: methodInfo,
+ transactions: transactions,
+ }
+}
+
+type TransactionServerStream struct {
+ transactionID string
+ ss grpc.ServerStream
+ mi protoregistry.MethodInfo
+ transactions *repository.Transactions
+}
+
+func (t TransactionServerStream) SetHeader(m metadata.MD) error {
+ return t.ss.SetHeader(m)
+}
+
+func (t TransactionServerStream) SendHeader(m metadata.MD) error {
+ return t.ss.SendHeader(m)
+}
+
+func (t TransactionServerStream) SetTrailer(m metadata.MD) {
+ t.ss.SetTrailer(m)
+}
+
+func (t TransactionServerStream) Context() context.Context {
+ return t.ss.Context()
+}
+
+func (t TransactionServerStream) SendMsg(m interface{}) error {
+ return t.ss.SendMsg(m)
+}
+
+func (t TransactionServerStream) RecvMsg(m interface{}) error {
+ if !t.transactions.TransactionStarted(t.transactionID) {
+ fmt.Printf("Receiving Message: type=%s\n", reflect.TypeOf(m).String())
+ repoReq, ok := m.(RepositoryRequest)
+ if ok {
+ t.transactions.NewTransaction(t.transactionID, repoReq.GetRepository())
+ }
+ }
+
+ t.ss.SetTrailer(metadata.Pairs("transaction_id", t.transactionID))
+
+ return t.ss.RecvMsg(m)
+}
+
+// StreamInterceptor returns a Stream Interceptor
+func RepositoryTransactionServerInterceptor(transactions *repository.Transactions, registry *protoregistry.Registry) grpc.StreamServerInterceptor {
+ return func(srv interface{}, stream grpc.ServerStream, info *grpc.StreamServerInfo, handler grpc.StreamHandler) error {
+ ctx := stream.Context()
+
+ mi, err := registry.LookupMethod(info.FullMethod)
+ if err != nil {
+ return err
+ }
+
+ if mi.Scope == protoregistry.ScopeRepository && mi.Operation == protoregistry.OpMutator {
+ var transactionID string
+
+ md, ok := metadata.FromIncomingContext(ctx)
+ if ok && len(md.Get("transaction_id")) > 0 {
+ transactionID = md.Get("transaction_id")[0]
+ } else {
+ transactionID = uuid.New().String()
+ }
+
+ defer transactions.Unlock(transactionID)
+
+ return handler(srv, NewTransactionServerStream(stream, mi, transactions, transactionID))
+ }
+
+ return handler(srv, stream)
+ }
+}
diff --git a/internal/praefect/coordinator.go b/internal/praefect/coordinator.go
index 6b674e438..fa7a24aba 100644
--- a/internal/praefect/coordinator.go
+++ b/internal/praefect/coordinator.go
@@ -8,6 +8,8 @@ import (
"sync"
"syscall"
+ "google.golang.org/grpc/metadata"
+
"google.golang.org/grpc"
"github.com/golang/protobuf/proto"
@@ -148,27 +150,12 @@ func (c *Coordinator) streamDirector(ctx context.Context, fullMethodName string,
}
func (s *Coordinator) HandleWriteRef(srv interface{}, serverStream grpc.ServerStream) error {
- s.log.Error("I'm here!!!!")
-
- peeker := proxy.NewPeeker(serverStream)
+ var writeRefReq gitalypb.WriteRefRequest
- mi, err := s.registry.LookupMethod("/gitaly.RepositoryService/WriteRef")
- if err != nil {
+ if err := serverStream.RecvMsg(&writeRefReq); err != nil {
return err
}
- frame, err := peeker.Peek()
-
- m, err := mi.UnmarshalRequestProto(frame)
- if err != nil {
- return err
- }
-
- writeRefReq, ok := m.(*gitalypb.WriteRefRequest)
- if !ok {
- return errors.New("sholud have been write ref request!")
- }
-
shard, err := s.nodeMgr.GetShard(writeRefReq.GetRepository().GetStorageName())
if err != nil {
return err
@@ -190,35 +177,42 @@ func (s *Coordinator) HandleWriteRef(srv interface{}, serverStream grpc.ServerSt
var errs []error
transactionIDs := make(map[string]nodes.Node)
+ // PreCommit
for _, node := range append(secondaries, primary) {
client := gitalypb.NewRepositoryServiceClient(node.GetConnection())
targetRepo := &gitalypb.Repository{
StorageName: node.GetStorage(),
RelativePath: writeRefReq.GetRepository().GetRelativePath(),
}
- resp, err := client.WriteRef(ctx, &gitalypb.WriteRefRequest{
+ var trailer metadata.MD
+
+ if _, err := client.WriteRef(ctx, &gitalypb.WriteRefRequest{
Repository: targetRepo,
Ref: writeRefReq.Ref,
Revision: writeRefReq.Revision,
OldRevision: writeRefReq.OldRevision,
Force: writeRefReq.Force,
- TransactionStep: &gitalypb.TransactionStep{
- Id: "",
- Step: gitalypb.TransactionStep_PRECOMMIT,
- }})
- if err != nil {
+ Transaction: &gitalypb.Transaction{
+ Step: gitalypb.Transaction_PRECOMMIT,
+ }}, grpc.Trailer(&trailer)); err != nil {
errs = append(errs, err)
}
- transactionIDs[resp.TransactionStep.Id] = node
+
+ if len(trailer.Get("transaction_id")) == 0 {
+ return errors.New("transaction id not found")
+ }
+
+ transactionID := trailer.Get("transaction_id")[0]
+ transactionIDs[transactionID] = node
}
+ // Commit
if len(errs) == 0 {
for transactionID, node := range transactionIDs {
client := gitalypb.NewRepositoryServiceClient(node.GetConnection())
- if _, err = client.WriteRef(ctx, &gitalypb.WriteRefRequest{
- TransactionStep: &gitalypb.TransactionStep{
- Id: transactionID,
- Step: gitalypb.TransactionStep_COMMIT,
+ if _, err = client.WriteRef(metadata.AppendToOutgoingContext(ctx, "transaction_id", transactionID), &gitalypb.WriteRefRequest{
+ Transaction: &gitalypb.Transaction{
+ Step: gitalypb.Transaction_COMMIT,
}}); err != nil {
return err
}
@@ -227,12 +221,12 @@ func (s *Coordinator) HandleWriteRef(srv interface{}, serverStream grpc.ServerSt
return nil
}
+ // Rollback
for transactionID, node := range transactionIDs {
client := gitalypb.NewRepositoryServiceClient(node.GetConnection())
- if _, err = client.WriteRef(ctx, &gitalypb.WriteRefRequest{
- TransactionStep: &gitalypb.TransactionStep{
- Id: transactionID,
- Step: gitalypb.TransactionStep_ROLLBACK,
+ if _, err = client.WriteRef(metadata.AppendToOutgoingContext(ctx, "transaction_id", transactionID), &gitalypb.WriteRefRequest{
+ Transaction: &gitalypb.Transaction{
+ Step: gitalypb.Transaction_ROLLBACK,
}}); err != nil {
return err
}
diff --git a/internal/praefect/grpc-proxy/proxy/handler.go b/internal/praefect/grpc-proxy/proxy/handler.go
index 2e960c3ee..3c3676d34 100644
--- a/internal/praefect/grpc-proxy/proxy/handler.go
+++ b/internal/praefect/grpc-proxy/proxy/handler.go
@@ -27,27 +27,6 @@ var (
// The behaviour is the same as if you were registering a handler method, e.g. from a codegenerated pb.go file.
//
// This can *only* be used if the `server` also uses grpcproxy.CodecForServer() ServerOption.
-/*
-func RegisterService(server *grpc.Server, director StreamDirector, serviceName string, methodNames ...string) {
- streamer := &handler{director: director}
- fakeDesc := &grpc.ServiceDesc{
- ServiceName: serviceName,
- HandlerType: (*interface{})(nil),
- }
- for _, m := range methodNames {
- streamDesc := grpc.StreamDesc{
- StreamName: m,
- Handler: streamer.handler,
- ServerStreams: true,
- ClientStreams: true,
- }
- fakeDesc.Streams = append(fakeDesc.Streams, streamDesc)
- }
- server.RegisterService(fakeDesc, streamer)
-}
-
-*/
-
func RegisterService(server *grpc.Server, streamer grpc.StreamHandler, serviceName string, methodNames ...string) {
fakeDesc := &grpc.ServiceDesc{
ServiceName: serviceName,
@@ -89,13 +68,6 @@ func (s *handler) handler(srv interface{}, serverStream grpc.ServerStream) error
return status.Errorf(codes.Internal, "lowLevelServerStream not exists in context")
}
- /*
- if fullMethodName == "/gitaly.RepositoryService/WriteRef" {
- return s.mutatorHandler(srv, serverStream)
- }
-
- */
-
peeker := newPeeker(serverStream)
// We require that the director's returned context inherits from the serverStream.Context().
diff --git a/internal/praefect/grpc-proxy/proxy/peeker.go b/internal/praefect/grpc-proxy/proxy/peeker.go
index df7fa4cb4..1d1e02df5 100644
--- a/internal/praefect/grpc-proxy/proxy/peeker.go
+++ b/internal/praefect/grpc-proxy/proxy/peeker.go
@@ -31,8 +31,6 @@ type peeker struct {
consumedStream *partialStream
}
-var NewPeeker = newPeeker
-
func newPeeker(stream grpc.ServerStream) *peeker {
return &peeker{
srcStream: stream,
diff --git a/internal/repotx/lock.go b/internal/repotx/lock.go
deleted file mode 100644
index 6ded8ce28..000000000
--- a/internal/repotx/lock.go
+++ /dev/null
@@ -1,10 +0,0 @@
-package repotx
-
-import "gitlab.com/gitlab-org/gitaly/internal/git/repository"
-
-type Transaction struct {
- serviceName string
- methodName string
- repo repository.GitRepo
- repository.RepoLock
-}
diff --git a/internal/server/server.go b/internal/server/server.go
index 250749248..32749eb9a 100644
--- a/internal/server/server.go
+++ b/internal/server/server.go
@@ -5,6 +5,12 @@ import (
"crypto/tls"
"os"
+ "gitlab.com/gitlab-org/gitaly/internal/praefect/grpc-proxy/proxy"
+
+ "gitlab.com/gitlab-org/gitaly/internal/middleware/repositoryhandler"
+
+ "gitlab.com/gitlab-org/gitaly/internal/git/repository"
+
grpc_middleware "github.com/grpc-ecosystem/go-grpc-middleware"
grpc_logrus "github.com/grpc-ecosystem/go-grpc-middleware/logging/logrus"
grpc_ctxtags "github.com/grpc-ecosystem/go-grpc-middleware/tags"
@@ -76,7 +82,12 @@ func createNewServer(rubyServer *rubyserver.Server, secure bool) *grpc.Server {
lh := limithandler.New(concurrencyKeyFn)
+ transactions := repository.NewTransactions()
+ registry := protoregistry.New()
+ registry.RegisterFiles(protoregistry.GitalyProtoFileDescriptors...)
+
opts := []grpc.ServerOption{
+ grpc.CustomCodec(proxy.Codec()),
grpc.StreamInterceptor(grpc_middleware.ChainStreamServer(
grpc_ctxtags.StreamServerInterceptor(ctxTagOpts...),
grpccorrelation.StreamServerCorrelationInterceptor(), // Must be above the metadata handler
@@ -92,6 +103,7 @@ func createNewServer(rubyServer *rubyserver.Server, secure bool) *grpc.Server {
// Panic handler should remain last so that application panics will be
// converted to errors and logged
panichandler.StreamPanicHandler,
+ repositoryhandler.RepositoryTransactionServerInterceptor(transactions, registry),
)),
grpc.UnaryInterceptor(grpc_middleware.ChainUnaryServer(
grpc_ctxtags.UnaryServerInterceptor(ctxTagOpts...),
@@ -108,6 +120,7 @@ func createNewServer(rubyServer *rubyserver.Server, secure bool) *grpc.Server {
// Panic handler should remain last so that application panics will be
// converted to errors and logged
panichandler.UnaryPanicHandler,
+ repositoryhandler.RepositoryTransactionUnaryInterceptor(transactions, registry),
)),
}
@@ -123,7 +136,7 @@ func createNewServer(rubyServer *rubyserver.Server, secure bool) *grpc.Server {
server := grpc.NewServer(opts...)
- service.RegisterAll(server, rubyServer)
+ service.RegisterAll(server, rubyServer, transactions)
reflection.Register(server)
grpc_prometheus.Register(server)
diff --git a/internal/service/register.go b/internal/service/register.go
index 119b01fa1..2db53735f 100644
--- a/internal/service/register.go
+++ b/internal/service/register.go
@@ -1,6 +1,7 @@
package service
import (
+ repo "gitlab.com/gitlab-org/gitaly/internal/git/repository"
"gitlab.com/gitlab-org/gitaly/internal/rubyserver"
"gitlab.com/gitlab-org/gitaly/internal/service/blob"
"gitlab.com/gitlab-org/gitaly/internal/service/cleanup"
@@ -26,7 +27,7 @@ import (
// RegisterAll will register all the known grpc services with
// the specified grpc service instance
-func RegisterAll(grpcServer *grpc.Server, rubyServer *rubyserver.Server) {
+func RegisterAll(grpcServer *grpc.Server, rubyServer *rubyserver.Server, transactions *repo.Transactions) {
gitalypb.RegisterBlobServiceServer(grpcServer, blob.NewServer(rubyServer))
gitalypb.RegisterCleanupServiceServer(grpcServer, cleanup.NewServer())
gitalypb.RegisterCommitServiceServer(grpcServer, commit.NewServer())
@@ -34,7 +35,7 @@ func RegisterAll(grpcServer *grpc.Server, rubyServer *rubyserver.Server) {
gitalypb.RegisterNamespaceServiceServer(grpcServer, namespace.NewServer())
gitalypb.RegisterOperationServiceServer(grpcServer, operations.NewServer(rubyServer))
gitalypb.RegisterRefServiceServer(grpcServer, ref.NewServer())
- gitalypb.RegisterRepositoryServiceServer(grpcServer, repository.NewServer(rubyServer))
+ gitalypb.RegisterRepositoryServiceServer(grpcServer, repository.NewServer(transactions, rubyServer))
gitalypb.RegisterSSHServiceServer(grpcServer, ssh.NewServer())
gitalypb.RegisterSmartHTTPServiceServer(grpcServer, smarthttp.NewServer())
gitalypb.RegisterWikiServiceServer(grpcServer, wiki.NewServer(rubyServer))
diff --git a/internal/service/repository/server.go b/internal/service/repository/server.go
index ba28188aa..9dc817d67 100644
--- a/internal/service/repository/server.go
+++ b/internal/service/repository/server.go
@@ -20,8 +20,8 @@ type server struct {
}
// NewServer creates a new instance of a gRPC repo server
-func NewServer(rs *rubyserver.Server) gitalypb.RepositoryServiceServer {
- return &server{ruby: rs, connsByAddress: make(map[string]*grpc.ClientConn), transactions: repository.NewTransactions()}
+func NewServer(transactions *repository.Transactions, rs *rubyserver.Server) gitalypb.RepositoryServiceServer {
+ return &server{ruby: rs, connsByAddress: make(map[string]*grpc.ClientConn), transactions: transactions}
}
func (*server) FetchHTTPRemote(context.Context, *gitalypb.FetchHTTPRemoteRequest) (*gitalypb.FetchHTTPRemoteResponse, error) {
diff --git a/internal/service/repository/write_ref.go b/internal/service/repository/write_ref.go
index afff0bfa0..bac47a5cb 100644
--- a/internal/service/repository/write_ref.go
+++ b/internal/service/repository/write_ref.go
@@ -3,8 +3,11 @@ package repository
import (
"bytes"
"context"
+ "errors"
"fmt"
+ "google.golang.org/grpc/metadata"
+
"gitlab.com/gitlab-org/gitaly/internal/command"
"gitlab.com/gitlab-org/gitaly/internal/git"
@@ -24,8 +27,18 @@ func nonTransactionalWriteRef(ctx context.Context, req *gitalypb.WriteRefRequest
func (s *server) transactionalWriteRef(ctx context.Context, req *gitalypb.WriteRefRequest) (*gitalypb.WriteRefResponse, error) {
var resp gitalypb.WriteRefResponse
- switch req.TransactionStep.Step {
- case gitalypb.TransactionStep_PRECOMMIT:
+ md, ok := metadata.FromIncomingContext(ctx)
+ if !ok {
+ return nil, helper.ErrInternal(errors.New("couldn't get metadata"))
+ }
+ if len(md.Get("transaction_id")) == 0 {
+ return nil, helper.ErrInternal(errors.New("expected request_id"))
+ }
+
+ transactionID := md.Get("transaction_id")[0]
+
+ switch req.Transaction.Step {
+ case gitalypb.Transaction_PRECOMMIT:
if err := writeRef(ctx, req); err != nil {
return nil, helper.ErrInternal(err)
}
@@ -34,14 +47,14 @@ func (s *server) transactionalWriteRef(ctx context.Context, req *gitalypb.WriteR
if err != nil {
return nil, err
}
- transactionID := s.transactions.StartTransaction(req.GetRepository(), rollback)
- resp.TransactionStep.Id = transactionID
- case gitalypb.TransactionStep_COMMIT:
- if err := s.transactions.Commit(req.TransactionStep.Id); err != nil {
+ s.transactions.Start(transactionID, rollback)
+
+ case gitalypb.Transaction_COMMIT:
+ if err := s.transactions.Commit(transactionID); err != nil {
return nil, err
}
- case gitalypb.TransactionStep_ROLLBACK:
- if err := s.transactions.Rollback(req.TransactionStep.Id); err != nil {
+ case gitalypb.Transaction_ROLLBACK:
+ if err := s.transactions.Rollback(transactionID); err != nil {
return nil, err
}
}
@@ -54,7 +67,7 @@ func (s *server) WriteRef(ctx context.Context, req *gitalypb.WriteRefRequest) (*
return nil, helper.ErrInvalidArgument(err)
}
- if req.TransactionStep == nil {
+ if req.Transaction == nil {
return nonTransactionalWriteRef(ctx, req)
}
diff --git a/proto/go/gitalypb/repository-service.pb.go b/proto/go/gitalypb/repository-service.pb.go
index 59983cc6d..a86203ee5 100644
--- a/proto/go/gitalypb/repository-service.pb.go
+++ b/proto/go/gitalypb/repository-service.pb.go
@@ -1199,15 +1199,15 @@ func (m *FsckResponse) GetError() []byte {
}
type WriteRefRequest struct {
- Repository *Repository `protobuf:"bytes,1,opt,name=repository,proto3" json:"repository,omitempty"`
- Ref []byte `protobuf:"bytes,2,opt,name=ref,proto3" json:"ref,omitempty"`
- Revision []byte `protobuf:"bytes,3,opt,name=revision,proto3" json:"revision,omitempty"`
- OldRevision []byte `protobuf:"bytes,4,opt,name=old_revision,json=oldRevision,proto3" json:"old_revision,omitempty"`
- Force bool `protobuf:"varint,5,opt,name=force,proto3" json:"force,omitempty"`
- TransactionStep *TransactionStep `protobuf:"bytes,7,opt,name=transaction_step,json=transactionStep,proto3" json:"transaction_step,omitempty"`
- XXX_NoUnkeyedLiteral struct{} `json:"-"`
- XXX_unrecognized []byte `json:"-"`
- XXX_sizecache int32 `json:"-"`
+ Repository *Repository `protobuf:"bytes,1,opt,name=repository,proto3" json:"repository,omitempty"`
+ Ref []byte `protobuf:"bytes,2,opt,name=ref,proto3" json:"ref,omitempty"`
+ Revision []byte `protobuf:"bytes,3,opt,name=revision,proto3" json:"revision,omitempty"`
+ OldRevision []byte `protobuf:"bytes,4,opt,name=old_revision,json=oldRevision,proto3" json:"old_revision,omitempty"`
+ Force bool `protobuf:"varint,5,opt,name=force,proto3" json:"force,omitempty"`
+ Transaction *Transaction `protobuf:"bytes,7,opt,name=transaction,proto3" json:"transaction,omitempty"`
+ XXX_NoUnkeyedLiteral struct{} `json:"-"`
+ XXX_unrecognized []byte `json:"-"`
+ XXX_sizecache int32 `json:"-"`
}
func (m *WriteRefRequest) Reset() { *m = WriteRefRequest{} }
@@ -1270,18 +1270,18 @@ func (m *WriteRefRequest) GetForce() bool {
return false
}
-func (m *WriteRefRequest) GetTransactionStep() *TransactionStep {
+func (m *WriteRefRequest) GetTransaction() *Transaction {
if m != nil {
- return m.TransactionStep
+ return m.Transaction
}
return nil
}
type WriteRefResponse struct {
- TransactionStep *TransactionStep `protobuf:"bytes,2,opt,name=transaction_step,json=transactionStep,proto3" json:"transaction_step,omitempty"`
- XXX_NoUnkeyedLiteral struct{} `json:"-"`
- XXX_unrecognized []byte `json:"-"`
- XXX_sizecache int32 `json:"-"`
+ Transaction *Transaction `protobuf:"bytes,2,opt,name=transaction,proto3" json:"transaction,omitempty"`
+ XXX_NoUnkeyedLiteral struct{} `json:"-"`
+ XXX_unrecognized []byte `json:"-"`
+ XXX_sizecache int32 `json:"-"`
}
func (m *WriteRefResponse) Reset() { *m = WriteRefResponse{} }
@@ -1309,9 +1309,9 @@ func (m *WriteRefResponse) XXX_DiscardUnknown() {
var xxx_messageInfo_WriteRefResponse proto.InternalMessageInfo
-func (m *WriteRefResponse) GetTransactionStep() *TransactionStep {
+func (m *WriteRefResponse) GetTransaction() *Transaction {
if m != nil {
- return m.TransactionStep
+ return m.Transaction
}
return nil
}
@@ -3841,193 +3841,192 @@ func init() {
func init() { proto.RegisterFile("repository-service.proto", fileDescriptor_e9b1768cf174c79b) }
var fileDescriptor_e9b1768cf174c79b = []byte{
- // 2962 bytes of a gzipped FileDescriptorProto
- 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x5a, 0x5b, 0x6f, 0x1b, 0xc7,
- 0xf5, 0x37, 0x75, 0x23, 0x79, 0x44, 0xdb, 0xd4, 0x48, 0xb6, 0xa8, 0xb5, 0x65, 0xd9, 0x6b, 0xc7,
- 0x71, 0x12, 0x47, 0x76, 0xe4, 0x3f, 0xf0, 0x4f, 0x5b, 0x14, 0x85, 0xa8, 0x7b, 0x6c, 0x4b, 0xca,
- 0x4a, 0x69, 0x10, 0xa3, 0xc1, 0x66, 0xb9, 0x1c, 0x89, 0x5b, 0x2d, 0x77, 0xe8, 0xd9, 0xa1, 0x14,
- 0x05, 0xed, 0x43, 0x0b, 0xe4, 0x35, 0x40, 0x81, 0xa2, 0xe9, 0x63, 0x9f, 0xfa, 0xd0, 0xef, 0x50,
- 0x14, 0x7d, 0xe9, 0x77, 0xc8, 0x57, 0x29, 0xfa, 0x50, 0xcc, 0x85, 0x3b, 0x7b, 0x65, 0x1c, 0x90,
- 0x48, 0xdf, 0x76, 0xce, 0x9c, 0x39, 0xe7, 0xcc, 0x99, 0x33, 0x97, 0xf3, 0x3b, 0x0b, 0x0d, 0x8a,
- 0x7b, 0x24, 0xf4, 0x18, 0xa1, 0x97, 0xef, 0x87, 0x98, 0x9e, 0x7b, 0x2e, 0x5e, 0xed, 0x51, 0xc2,
- 0x08, 0x9a, 0x39, 0xf5, 0x98, 0xe3, 0x5f, 0x1a, 0xb5, 0xb0, 0xe3, 0x50, 0xdc, 0x96, 0x54, 0xf3,
- 0x08, 0x16, 0xad, 0x68, 0xc4, 0xd6, 0x97, 0x5e, 0xc8, 0x42, 0x0b, 0xbf, 0xee, 0xe3, 0x90, 0xa1,
- 0x0f, 0x01, 0xb4, 0xb0, 0x46, 0xe9, 0x6e, 0xe9, 0xd1, 0xec, 0x1a, 0x5a, 0x95, 0x52, 0x56, 0xf5,
- 0xa0, 0xe6, 0xd4, 0x9f, 0xff, 0xf5, 0xb8, 0x64, 0xc5, 0x78, 0xcd, 0x35, 0x68, 0x64, 0x85, 0x86,
- 0x3d, 0x12, 0x84, 0x18, 0xdd, 0x84, 0x19, 0x2c, 0x28, 0x42, 0x62, 0xc5, 0x52, 0x2d, 0xf3, 0x58,
- 0x8c, 0x71, 0xdc, 0xb3, 0xbd, 0xc0, 0xa5, 0xb8, 0x8b, 0x03, 0xe6, 0xf8, 0xa3, 0x5b, 0x72, 0x0b,
- 0x96, 0x72, 0xa4, 0x4a, 0x53, 0x4c, 0x0a, 0x73, 0xb2, 0x73, 0xbb, 0xef, 0x8f, 0xae, 0x0b, 0xdd,
- 0x87, 0xab, 0x2e, 0xc5, 0x0e, 0xc3, 0x76, 0xcb, 0x63, 0x5d, 0xa7, 0xd7, 0x98, 0x10, 0x13, 0xac,
- 0x49, 0x62, 0x53, 0xd0, 0xcc, 0x05, 0x40, 0x71, 0x9d, 0xca, 0x92, 0x73, 0xb8, 0xb1, 0xe3, 0xd0,
- 0x96, 0x73, 0x8a, 0x37, 0x88, 0xef, 0x63, 0x97, 0xfd, 0x48, 0xd6, 0x34, 0xe0, 0x66, 0x5a, 0xaf,
- 0xb2, 0xe8, 0x23, 0xb8, 0xb6, 0xe1, 0x63, 0x27, 0xe8, 0xf7, 0x46, 0x5f, 0x84, 0x39, 0xb8, 0x1e,
- 0xc9, 0x52, 0xe2, 0x3f, 0x86, 0x1b, 0x7a, 0xc8, 0x91, 0xf7, 0x15, 0x1e, 0x5d, 0xcb, 0x63, 0xb8,
- 0x99, 0x16, 0xa9, 0x42, 0x0e, 0xc1, 0x54, 0xe8, 0x7d, 0x85, 0x85, 0xb4, 0x49, 0x4b, 0x7c, 0x9b,
- 0xaf, 0x61, 0x69, 0xbd, 0xd7, 0xf3, 0x2f, 0x77, 0x3c, 0xe6, 0x30, 0x46, 0xbd, 0x56, 0x9f, 0xe1,
- 0xd1, 0x23, 0x1f, 0x19, 0x50, 0xa1, 0xf8, 0xdc, 0x0b, 0x3d, 0x12, 0x08, 0x87, 0xd7, 0xac, 0xa8,
- 0x6d, 0xde, 0x06, 0x23, 0x4f, 0xa5, 0xf2, 0xc8, 0x3f, 0x26, 0x00, 0x6d, 0x63, 0xe6, 0x76, 0x2c,
- 0xdc, 0x25, 0x6c, 0x74, 0x7f, 0xf0, 0x8d, 0x46, 0x85, 0x28, 0x61, 0x48, 0xd5, 0x52, 0x2d, 0xb4,
- 0x00, 0xd3, 0x27, 0x84, 0xba, 0xb8, 0x31, 0x29, 0x02, 0x42, 0x36, 0xd0, 0x22, 0x94, 0x03, 0x62,
- 0x33, 0xe7, 0x34, 0x6c, 0x4c, 0xc9, 0x7d, 0x19, 0x90, 0x63, 0xe7, 0x34, 0x44, 0x0d, 0x28, 0x33,
- 0xaf, 0x8b, 0x49, 0x9f, 0x35, 0xa6, 0xef, 0x96, 0x1e, 0x4d, 0x5b, 0x83, 0x26, 0x1f, 0x12, 0x86,
- 0x1d, 0xfb, 0x0c, 0x5f, 0x36, 0x66, 0xa4, 0x86, 0x30, 0xec, 0x3c, 0xc7, 0x97, 0x68, 0x05, 0x66,
- 0xcf, 0x02, 0x72, 0x11, 0xd8, 0x1d, 0xc2, 0xf7, 0x79, 0x59, 0x74, 0x82, 0x20, 0xed, 0x72, 0x0a,
- 0x5a, 0x82, 0x4a, 0x40, 0xec, 0x1e, 0xed, 0x07, 0xb8, 0x51, 0x15, 0xda, 0xca, 0x01, 0x39, 0xe4,
- 0x4d, 0xf4, 0x0c, 0xae, 0x4a, 0x3b, 0xed, 0x9e, 0x43, 0x9d, 0x6e, 0xd8, 0x00, 0x31, 0xe5, 0x6b,
- 0x7a, 0xca, 0xc2, 0x3b, 0x35, 0xc9, 0x74, 0x28, 0x78, 0x3e, 0x9a, 0xaa, 0x54, 0xea, 0x55, 0xf3,
- 0x06, 0xcc, 0x27, 0x1c, 0xa8, 0x1c, 0x7b, 0x04, 0x8b, 0x1b, 0x22, 0xe6, 0xb5, 0xb7, 0x46, 0x0f,
- 0x36, 0x03, 0x1a, 0x59, 0xa1, 0x4a, 0xe1, 0xd7, 0x13, 0x30, 0xb7, 0x83, 0xd9, 0x3a, 0x75, 0x3b,
- 0xde, 0xf9, 0x18, 0x16, 0xf2, 0x16, 0x54, 0x5d, 0xd2, 0xed, 0x7a, 0xcc, 0xf6, 0xda, 0x6a, 0x2d,
- 0x2b, 0x92, 0xb0, 0xd7, 0xe6, 0xab, 0xdc, 0xa3, 0xf8, 0xc4, 0xfb, 0x52, 0x2c, 0x67, 0xd5, 0x52,
- 0x2d, 0xf4, 0x21, 0xcc, 0x9c, 0x10, 0xda, 0x75, 0x98, 0x58, 0xce, 0x6b, 0x6b, 0x77, 0x07, 0xaa,
- 0x32, 0x96, 0xad, 0x6e, 0x0b, 0x3e, 0x4b, 0xf1, 0xf3, 0xdd, 0xd2, 0x73, 0x58, 0x47, 0xac, 0x76,
- 0xcd, 0x12, 0xdf, 0xe6, 0x33, 0x98, 0x91, 0x5c, 0xa8, 0x0c, 0x93, 0xaf, 0xf6, 0x0e, 0xeb, 0x57,
- 0xf8, 0xc7, 0xf1, 0xba, 0x55, 0x2f, 0x21, 0x80, 0x99, 0xe3, 0x75, 0xcb, 0xde, 0x79, 0x55, 0x9f,
- 0x40, 0xb3, 0x50, 0xe6, 0xdf, 0xcd, 0x57, 0x6b, 0xf5, 0x49, 0xf3, 0x11, 0xa0, 0xb8, 0x32, 0xbd,
- 0x19, 0xdb, 0x0e, 0x73, 0x84, 0x07, 0x6a, 0x96, 0xf8, 0xe6, 0x4b, 0xb4, 0xeb, 0x84, 0x2f, 0x88,
- 0xeb, 0xf8, 0x4d, 0xea, 0x04, 0x6e, 0x67, 0x0c, 0x5b, 0xd1, 0x7c, 0x0a, 0x8d, 0xac, 0x50, 0x65,
- 0xc4, 0x02, 0x4c, 0x9f, 0x3b, 0x7e, 0x1f, 0xab, 0x3b, 0x48, 0x36, 0xcc, 0xef, 0x4a, 0xd0, 0x10,
- 0x11, 0x74, 0x44, 0xfa, 0xd4, 0xc5, 0x72, 0xd4, 0xe8, 0xeb, 0xf7, 0x0b, 0x98, 0x0b, 0x85, 0x40,
- 0x3b, 0x26, 0x60, 0xa2, 0x48, 0x80, 0x55, 0x97, 0xcc, 0x56, 0xe2, 0x28, 0x57, 0x02, 0x5a, 0xc2,
- 0x24, 0xb1, 0xd4, 0x35, 0xab, 0x16, 0xc6, 0xcc, 0x44, 0xcb, 0x00, 0xcc, 0xa1, 0xa7, 0x98, 0xd9,
- 0x14, 0x9f, 0x88, 0x45, 0xaf, 0x59, 0x55, 0x49, 0xb1, 0xf0, 0x89, 0xf9, 0x0c, 0x96, 0x72, 0xa6,
- 0xa6, 0xef, 0x64, 0x8a, 0xc3, 0xbe, 0xcf, 0x06, 0x77, 0xb2, 0x6c, 0x99, 0x3b, 0x30, 0xbb, 0x1d,
- 0xba, 0x67, 0xa3, 0xaf, 0xc5, 0x03, 0xa8, 0x49, 0x41, 0xda, 0xff, 0x98, 0x52, 0x42, 0x55, 0x14,
- 0xc8, 0x86, 0xf9, 0x9f, 0x12, 0x5c, 0xff, 0x94, 0x7a, 0x7c, 0x53, 0x9d, 0x8c, 0xee, 0xf6, 0x3a,
- 0x4c, 0x72, 0x4f, 0xc8, 0x53, 0x98, 0x7f, 0x26, 0x0e, 0xe7, 0xc9, 0xe4, 0xe1, 0x8c, 0xee, 0x41,
- 0x8d, 0xf8, 0x6d, 0x3b, 0xea, 0x97, 0x0e, 0x9c, 0x25, 0x7e, 0xdb, 0x1a, 0xb0, 0x44, 0x07, 0xe7,
- 0x74, 0xfc, 0xe0, 0x6c, 0x42, 0x9d, 0x51, 0x27, 0x08, 0x1d, 0x97, 0x79, 0x24, 0xb0, 0x43, 0x86,
- 0x7b, 0xe2, 0xc4, 0x9b, 0x5d, 0x5b, 0x1c, 0x98, 0x79, 0xac, 0xfb, 0x8f, 0x18, 0xee, 0x59, 0xd7,
- 0x59, 0x92, 0xf0, 0xd1, 0x54, 0x65, 0xa6, 0x5e, 0x36, 0x7f, 0x05, 0x75, 0x3d, 0x7b, 0xe5, 0xa8,
- 0x3c, 0xe9, 0x13, 0x3f, 0x58, 0x7a, 0xa9, 0x3e, 0x61, 0x06, 0xb0, 0xb0, 0xed, 0x05, 0xed, 0x97,
- 0x98, 0x9e, 0xe2, 0xa6, 0x13, 0x8e, 0xe1, 0x5c, 0xba, 0x0d, 0xd5, 0x81, 0xbb, 0xc2, 0xc6, 0xc4,
- 0xdd, 0x49, 0x1e, 0x70, 0x11, 0xc1, 0x7c, 0x0f, 0x6e, 0xa4, 0xf4, 0xe9, 0x03, 0xa0, 0xe5, 0x84,
- 0x72, 0xeb, 0x55, 0x2d, 0xf1, 0x6d, 0x7e, 0x53, 0x82, 0x39, 0x79, 0x9e, 0x6e, 0x13, 0x7a, 0xf6,
- 0xbf, 0xdf, 0x72, 0xfc, 0x99, 0x16, 0xb7, 0x27, 0x7a, 0x30, 0x2e, 0xed, 0x85, 0x16, 0xe6, 0x26,
- 0xef, 0x05, 0x87, 0x94, 0x9c, 0x52, 0x1c, 0x86, 0x63, 0x39, 0xe0, 0xa9, 0x10, 0x1a, 0x3b, 0xe0,
- 0x25, 0x61, 0xaf, 0x6d, 0xfe, 0x1c, 0x8c, 0x3c, 0x9d, 0xca, 0x99, 0x2b, 0x30, 0xeb, 0x05, 0x76,
- 0x4f, 0x91, 0xd5, 0xf6, 0x05, 0x2f, 0x62, 0x94, 0x26, 0x1f, 0xbd, 0xee, 0x3b, 0x61, 0x67, 0xcc,
- 0x26, 0x87, 0x42, 0x68, 0xcc, 0x64, 0x49, 0x18, 0x98, 0x9c, 0xd5, 0xf9, 0xa6, 0x26, 0xfb, 0x70,
- 0x27, 0x7d, 0xb7, 0x6e, 0x53, 0xd2, 0xfd, 0xc4, 0x7a, 0x31, 0x96, 0x43, 0xa1, 0x4f, 0x7d, 0x65,
- 0x31, 0xff, 0x34, 0xef, 0xc1, 0x4a, 0xa1, 0x36, 0xb5, 0xec, 0x07, 0x30, 0x2f, 0x59, 0x9a, 0xfd,
- 0xa0, 0xed, 0x8f, 0xe1, 0xa9, 0xfa, 0x2e, 0x2c, 0x24, 0x05, 0x0e, 0xb9, 0x1b, 0xbf, 0x99, 0x80,
- 0xfa, 0x11, 0x66, 0x1b, 0x24, 0x38, 0xf1, 0x4e, 0x47, 0x77, 0xc0, 0x87, 0x50, 0xc6, 0x01, 0xa3,
- 0x1e, 0x96, 0x5b, 0x76, 0x76, 0xed, 0xce, 0x60, 0x58, 0x5a, 0xc9, 0xea, 0x56, 0xc0, 0xe8, 0xa5,
- 0x35, 0x60, 0x37, 0xbe, 0x2e, 0xc1, 0xb4, 0x20, 0x71, 0x27, 0xf2, 0x47, 0x9f, 0xdc, 0xc0, 0xfc,
- 0x13, 0x2d, 0x43, 0x55, 0x5c, 0xa1, 0x76, 0xc8, 0xa8, 0x74, 0xee, 0xee, 0x15, 0xab, 0x22, 0x48,
- 0x47, 0x8c, 0xa2, 0x7b, 0x30, 0x2b, 0xbb, 0xbd, 0x80, 0x3d, 0x5b, 0x13, 0x67, 0xef, 0xf4, 0xee,
- 0x15, 0x0b, 0x04, 0x71, 0x8f, 0xd3, 0xd0, 0x0a, 0xc8, 0x96, 0xdd, 0x22, 0xc4, 0x97, 0x4f, 0xd0,
- 0xdd, 0x2b, 0x96, 0x94, 0xda, 0x24, 0xc4, 0x6f, 0x96, 0xd5, 0x95, 0x6d, 0xce, 0xc3, 0x5c, 0xcc,
- 0x54, 0xb5, 0x44, 0x2e, 0xcc, 0x6f, 0x62, 0x1f, 0x33, 0x3c, 0x2e, 0x3f, 0x21, 0x98, 0x3a, 0xc3,
- 0x97, 0xd2, 0x49, 0x55, 0x4b, 0x7c, 0x9b, 0x37, 0x61, 0x21, 0xa9, 0x44, 0x29, 0xf7, 0x78, 0x92,
- 0x19, 0x32, 0x42, 0xf1, 0x46, 0x3f, 0x64, 0xa4, 0xbb, 0x4b, 0xc8, 0x59, 0x38, 0x16, 0x13, 0x44,
- 0x34, 0x4c, 0xc4, 0xa2, 0xe1, 0x36, 0x18, 0x79, 0xaa, 0x94, 0x21, 0xc7, 0xd0, 0x68, 0x3a, 0xee,
- 0x59, 0xbf, 0x37, 0x4e, 0x3b, 0xcc, 0x27, 0xb0, 0x94, 0x23, 0x75, 0x48, 0xc8, 0xbe, 0x86, 0x7b,
- 0x79, 0x5b, 0x6a, 0x4c, 0xbb, 0x27, 0xd7, 0x2f, 0x0f, 0xc0, 0x1c, 0xa6, 0x52, 0xf9, 0x67, 0x1f,
- 0x10, 0xbf, 0x93, 0x5e, 0x78, 0x2e, 0x0e, 0xc6, 0x70, 0x03, 0x9a, 0x1b, 0x30, 0x9f, 0x90, 0xa7,
- 0x7c, 0xf2, 0x18, 0x90, 0x2f, 0x49, 0x76, 0xd8, 0x21, 0x94, 0xd9, 0x81, 0xd3, 0x1d, 0xdc, 0x77,
- 0x75, 0xd5, 0x73, 0xc4, 0x3b, 0xf6, 0x9d, 0xae, 0x58, 0xb4, 0x1d, 0xcc, 0xf6, 0x82, 0x13, 0xb2,
- 0x3e, 0xbe, 0x44, 0xd4, 0xfc, 0x19, 0x2c, 0xe5, 0x48, 0x55, 0x06, 0xde, 0x01, 0xd0, 0x19, 0xa8,
- 0x5a, 0xba, 0x18, 0x85, 0x9b, 0xb4, 0xe1, 0xf8, 0x6e, 0xdf, 0x77, 0x18, 0xde, 0xe8, 0x60, 0xf7,
- 0x2c, 0xec, 0x77, 0x47, 0x37, 0xe9, 0xff, 0x61, 0x29, 0x47, 0xaa, 0x32, 0xc9, 0x80, 0x8a, 0xab,
- 0x68, 0xca, 0x53, 0x51, 0x9b, 0x2f, 0xdb, 0x0e, 0x66, 0x47, 0x81, 0xd3, 0x0b, 0x3b, 0x64, 0x74,
- 0x68, 0xc4, 0x7c, 0x07, 0xe6, 0x13, 0xf2, 0x86, 0x84, 0xf2, 0xb7, 0x25, 0xb8, 0x9f, 0x17, 0x58,
- 0x63, 0x33, 0x86, 0xe7, 0xc2, 0x1d, 0xc6, 0x7a, 0xb6, 0xbe, 0x96, 0xca, 0xbc, 0xfd, 0x09, 0xf5,
- 0xf9, 0x25, 0x2b, 0xba, 0x9c, 0x3e, 0xeb, 0xa8, 0xf4, 0x4e, 0xf0, 0xae, 0xf7, 0x59, 0xc7, 0x7c,
- 0x08, 0x0f, 0x86, 0x1b, 0xa6, 0x62, 0xfe, 0x4f, 0x25, 0x58, 0xd8, 0xc1, 0xcc, 0x72, 0x2e, 0x36,
- 0x3a, 0x4e, 0x70, 0x3a, 0x0e, 0x90, 0xe3, 0x3e, 0x5c, 0x3d, 0xa1, 0xa4, 0x6b, 0x27, 0x90, 0x8e,
- 0xaa, 0x55, 0xe3, 0xc4, 0xe8, 0xb5, 0xbc, 0x02, 0xb3, 0x8c, 0xd8, 0x89, 0xf7, 0x76, 0xd5, 0x02,
- 0x46, 0x06, 0x0c, 0xe6, 0xdf, 0xa7, 0xe0, 0x46, 0xca, 0x30, 0xb5, 0x10, 0xbb, 0x30, 0x4b, 0x9d,
- 0x0b, 0xdb, 0x95, 0xe4, 0x46, 0x49, 0xdc, 0x53, 0x6f, 0xc7, 0x12, 0xd8, 0xec, 0x98, 0xd5, 0x88,
- 0x64, 0x01, 0x8d, 0x7a, 0x8d, 0xef, 0x26, 0xa1, 0x1a, 0xf5, 0xa0, 0x45, 0x28, 0xb7, 0x7c, 0xd2,
- 0xe2, 0x4f, 0x16, 0x19, 0x62, 0x33, 0xbc, 0xb9, 0xd7, 0x8e, 0x00, 0xa2, 0x09, 0x0d, 0x10, 0xa1,
- 0x65, 0xa8, 0x04, 0xf8, 0xc2, 0x16, 0xa9, 0xb0, 0x30, 0xbe, 0x39, 0xd1, 0x28, 0x59, 0xe5, 0x00,
- 0x5f, 0x1c, 0x3a, 0x8c, 0xa7, 0x5b, 0x15, 0x9e, 0x2f, 0x88, 0xee, 0x29, 0xdd, 0x4d, 0xfc, 0xb6,
- 0xe8, 0x3e, 0x80, 0x2a, 0xe9, 0x61, 0xea, 0xf0, 0x47, 0xb8, 0xc8, 0x17, 0xae, 0xad, 0x7d, 0xf0,
- 0x86, 0x13, 0x58, 0x3d, 0x18, 0x0c, 0xb4, 0xb4, 0x0c, 0xee, 0x73, 0xee, 0x13, 0x2d, 0x54, 0x42,
- 0x2e, 0x35, 0xea, 0x5c, 0x44, 0xfc, 0x3c, 0x96, 0xb8, 0x51, 0x5d, 0xd2, 0xc6, 0x22, 0x07, 0x99,
- 0x16, 0x06, 0xbd, 0x24, 0x6d, 0x2c, 0x20, 0x17, 0x7c, 0x21, 0xbb, 0x2a, 0xb2, 0x2b, 0xc0, 0x17,
- 0xa2, 0xeb, 0x01, 0x5c, 0x1b, 0xcc, 0xd4, 0x6e, 0x5d, 0xf2, 0x13, 0xa1, 0x2a, 0xf3, 0x4b, 0x35,
- 0xd7, 0x26, 0xa7, 0x71, 0xae, 0xc1, 0x84, 0x15, 0x17, 0x48, 0x2e, 0x35, 0x65, 0xc1, 0x65, 0x7a,
- 0x50, 0xd5, 0xe6, 0xcc, 0x42, 0xf9, 0x93, 0xfd, 0xe7, 0xfb, 0x07, 0x9f, 0xee, 0xd7, 0xaf, 0xa0,
- 0x2a, 0x4c, 0xaf, 0x6f, 0x6e, 0x6e, 0x6d, 0x4a, 0xc4, 0x60, 0xe3, 0xe0, 0x70, 0x6f, 0x6b, 0x53,
- 0x22, 0x06, 0x9b, 0x5b, 0x2f, 0xb6, 0x8e, 0xb7, 0x36, 0xeb, 0x93, 0xa8, 0x06, 0x95, 0x97, 0x07,
- 0x9b, 0x7b, 0xdb, 0xbc, 0x6b, 0x8a, 0x77, 0x59, 0x5b, 0xfb, 0xeb, 0x2f, 0xb7, 0x36, 0xeb, 0xd3,
- 0xa8, 0x0e, 0xb5, 0xe3, 0xcf, 0x0e, 0xb7, 0xec, 0x8d, 0xdd, 0xf5, 0xfd, 0x9d, 0xad, 0xcd, 0xfa,
- 0x8c, 0xf9, 0x1b, 0x68, 0x1c, 0x61, 0x87, 0xba, 0x9d, 0x6d, 0xcf, 0xc7, 0x61, 0xf3, 0x92, 0x1f,
- 0xa6, 0xa3, 0xc7, 0xf6, 0x02, 0x4c, 0xbf, 0xee, 0x63, 0x95, 0x2d, 0x54, 0x2d, 0xd9, 0x18, 0xe4,
- 0x92, 0x93, 0x51, 0x2e, 0x69, 0x7e, 0x00, 0x4b, 0x39, 0xda, 0x75, 0x7a, 0x7b, 0xc2, 0xc9, 0x22,
- 0x74, 0x6b, 0x96, 0x6c, 0x98, 0x7f, 0x2b, 0xc1, 0xad, 0xc4, 0x98, 0x0d, 0x12, 0x30, 0x1c, 0xb0,
- 0x1f, 0xcd, 0x68, 0xf4, 0x0e, 0xd4, 0xdd, 0x4e, 0x3f, 0x38, 0xc3, 0x3c, 0xd1, 0x95, 0xb6, 0x2a,
- 0xb4, 0xef, 0xba, 0xa2, 0x47, 0xc7, 0xc6, 0x25, 0xdc, 0xce, 0xb7, 0x55, 0x4d, 0xb1, 0x01, 0xe5,
- 0xae, 0xc3, 0xdc, 0x4e, 0x34, 0xc9, 0x41, 0x13, 0x2d, 0x03, 0x88, 0x4f, 0x3b, 0x76, 0x49, 0x57,
- 0x05, 0x65, 0xd3, 0x61, 0x0e, 0xba, 0x0b, 0x35, 0x1c, 0xb4, 0x6d, 0x72, 0x62, 0x0b, 0x9a, 0x42,
- 0x21, 0x01, 0x07, 0xed, 0x83, 0x93, 0x97, 0x9c, 0x62, 0xfe, 0xa1, 0x04, 0x33, 0x12, 0xc3, 0x1b,
- 0x3c, 0xd7, 0x4b, 0xd1, 0x73, 0x9d, 0x6f, 0x55, 0x71, 0x9b, 0xca, 0x99, 0x8a, 0x6f, 0xf4, 0x53,
- 0x58, 0x8a, 0xce, 0x49, 0x42, 0xbd, 0xaf, 0x44, 0xf4, 0xd9, 0x1d, 0xec, 0xb4, 0x31, 0x55, 0x07,
- 0xcf, 0xe2, 0xe0, 0xdc, 0x8c, 0xfa, 0x77, 0x45, 0x37, 0x7a, 0x0b, 0xae, 0x75, 0x3d, 0x4a, 0x09,
- 0xb5, 0x29, 0x3e, 0xe9, 0x3a, 0xbd, 0xb0, 0x31, 0x25, 0x5e, 0x7c, 0x57, 0x25, 0xd5, 0x92, 0x44,
- 0xf3, 0x8f, 0x25, 0xb8, 0x29, 0xf0, 0x93, 0xdd, 0xe3, 0xe3, 0xc3, 0x71, 0x21, 0xb4, 0x0f, 0x13,
- 0x08, 0x6d, 0x16, 0xe4, 0x1c, 0x20, 0xb6, 0x31, 0x08, 0x76, 0x32, 0x01, 0xc1, 0x9a, 0x4b, 0xb0,
- 0x98, 0xb1, 0x4a, 0x2d, 0xe0, 0x67, 0xb0, 0xbc, 0x83, 0xd9, 0x41, 0xeb, 0xd7, 0xd8, 0x65, 0x9b,
- 0x1e, 0xc5, 0xee, 0xf8, 0x90, 0xf6, 0xff, 0x83, 0x3b, 0x45, 0xa2, 0x87, 0x20, 0xee, 0x7f, 0x29,
- 0xc1, 0xc2, 0x86, 0x4f, 0x02, 0xcc, 0xaf, 0xa9, 0x43, 0x42, 0xfc, 0x71, 0x38, 0x70, 0xaa, 0xc7,
- 0xd3, 0x85, 0x54, 0x66, 0x2f, 0x2d, 0x13, 0x2a, 0x44, 0x7f, 0xcc, 0xd1, 0x93, 0xc3, 0x1c, 0x6d,
- 0x2e, 0xc2, 0x8d, 0x94, 0x85, 0xca, 0x99, 0xff, 0x2c, 0xc1, 0xed, 0x44, 0xcf, 0x5e, 0xc0, 0x30,
- 0x0d, 0x9c, 0x1f, 0x71, 0x0e, 0xb9, 0x90, 0xc6, 0xe4, 0x0f, 0x80, 0x34, 0x56, 0x60, 0xb9, 0x60,
- 0x0a, 0x1a, 0x28, 0xe7, 0xfe, 0x38, 0x1f, 0x37, 0x50, 0x9e, 0x15, 0xaa, 0x14, 0x7e, 0xc9, 0x15,
- 0x06, 0xe2, 0xe0, 0x1c, 0x9b, 0x42, 0x71, 0x51, 0x62, 0xdf, 0x61, 0xde, 0x39, 0x96, 0xb7, 0xb3,
- 0x7a, 0x9c, 0x0c, 0x88, 0xfc, 0xae, 0x92, 0x56, 0xa5, 0x35, 0x2b, 0xab, 0x7e, 0x5f, 0xe2, 0x39,
- 0x56, 0xcf, 0xf7, 0xdc, 0xf1, 0xd6, 0x0c, 0xd0, 0xbb, 0x30, 0x23, 0x17, 0x65, 0x08, 0x12, 0xa5,
- 0x38, 0xcc, 0x65, 0xb8, 0x95, 0x6b, 0x83, 0xb4, 0x71, 0xed, 0xaf, 0xcb, 0xa2, 0x74, 0x39, 0x28,
- 0x76, 0xc9, 0x3a, 0x2f, 0xfa, 0x1c, 0xea, 0xe9, 0xb2, 0x2b, 0x5a, 0xc9, 0x2a, 0x49, 0x54, 0x79,
- 0x8d, 0xbb, 0xc5, 0x0c, 0xca, 0x21, 0x33, 0xff, 0xfe, 0xf6, 0xd1, 0x44, 0x65, 0x02, 0x7d, 0x31,
- 0x28, 0x97, 0xc6, 0x6a, 0xa9, 0x28, 0x3e, 0x3c, 0xb7, 0x78, 0x6b, 0xdc, 0x1b, 0xc2, 0x91, 0xd0,
- 0x50, 0x42, 0xcf, 0x01, 0x74, 0x71, 0x14, 0x2d, 0x25, 0x07, 0xc6, 0x8a, 0xb4, 0x86, 0x91, 0xd7,
- 0x95, 0x12, 0xf6, 0x29, 0x5c, 0x4b, 0xd6, 0x36, 0xd1, 0x72, 0xf4, 0x02, 0xcb, 0xab, 0xb5, 0x1a,
- 0x77, 0x8a, 0xba, 0xb3, 0x82, 0x93, 0x85, 0x46, 0x2d, 0x38, 0xb7, 0xa6, 0xa9, 0x05, 0xe7, 0xd7,
- 0x27, 0x23, 0xc1, 0x2e, 0xa0, 0x6c, 0x81, 0x10, 0x45, 0xfe, 0x2b, 0xac, 0x57, 0x1a, 0xe6, 0x30,
- 0x96, 0x94, 0x92, 0x7d, 0x98, 0x8d, 0x55, 0xc9, 0x50, 0xe4, 0xc9, 0x6c, 0xed, 0xd1, 0xb8, 0x95,
- 0xdb, 0x97, 0x92, 0xf7, 0x39, 0xd4, 0xd3, 0x79, 0x88, 0x0e, 0xba, 0x82, 0xc2, 0x9b, 0x0e, 0xba,
- 0xc2, 0x22, 0xda, 0x40, 0xfc, 0x4b, 0x00, 0x5d, 0x44, 0xd2, 0x21, 0x91, 0xa9, 0x62, 0xe9, 0x90,
- 0xc8, 0xd6, 0x9c, 0x06, 0xc2, 0x9e, 0x0a, 0x6b, 0xd3, 0x45, 0x21, 0x6d, 0x6d, 0x41, 0x0d, 0x4a,
- 0x5b, 0x5b, 0x54, 0x4f, 0x8a, 0x6f, 0x91, 0x4c, 0x95, 0x45, 0x6f, 0x91, 0xa2, 0xda, 0x92, 0xde,
- 0x22, 0x85, 0x25, 0x9a, 0xc8, 0x1f, 0x3f, 0x81, 0xa9, 0xed, 0xd0, 0x3d, 0x43, 0xf3, 0xd1, 0x10,
- 0x5d, 0xa0, 0x31, 0x16, 0x92, 0xc4, 0xd4, 0xd0, 0x2d, 0xa8, 0x0c, 0xea, 0x0b, 0x28, 0xaa, 0x1e,
- 0xa4, 0xea, 0x2d, 0x46, 0x23, 0xdb, 0x91, 0x12, 0x73, 0x0c, 0x57, 0x13, 0xc0, 0x3e, 0xba, 0x1d,
- 0x69, 0xcd, 0xa9, 0x2f, 0x18, 0xcb, 0x05, 0xbd, 0x29, 0xcf, 0x3d, 0x07, 0xd0, 0x80, 0xbb, 0x5e,
- 0xe7, 0x4c, 0x51, 0x40, 0xaf, 0x73, 0x0e, 0x3e, 0x1f, 0xdb, 0x48, 0x59, 0xcc, 0x5c, 0x6f, 0xa4,
- 0x42, 0x0c, 0x5f, 0x6f, 0xa4, 0x62, 0xc8, 0x3d, 0xb2, 0x58, 0x28, 0x49, 0xa3, 0xdc, 0x71, 0x25,
- 0x05, 0xa8, 0x7b, 0x5c, 0x49, 0x11, 0x48, 0x1e, 0x29, 0xa1, 0xd9, 0xe2, 0xb5, 0x42, 0xa7, 0xd1,
- 0xc3, 0xa2, 0x3d, 0x94, 0x04, 0xcb, 0x8d, 0xb7, 0xbf, 0x97, 0x2f, 0xe5, 0xbd, 0x23, 0xa8, 0xc5,
- 0xd1, 0x69, 0x74, 0x2b, 0x29, 0x20, 0x01, 0xe3, 0x19, 0xb7, 0xf3, 0x3b, 0x93, 0xd3, 0x78, 0x5a,
- 0x42, 0xbf, 0x05, 0xa3, 0x18, 0xa0, 0x43, 0xef, 0x0c, 0xb3, 0x31, 0xa9, 0xf0, 0xdd, 0x37, 0x61,
- 0x4d, 0xce, 0xe8, 0x51, 0x09, 0xed, 0x42, 0x35, 0x02, 0x8d, 0x51, 0xa3, 0x08, 0xf2, 0x36, 0x96,
- 0x72, 0x7a, 0x52, 0xde, 0xf9, 0x18, 0x6a, 0x71, 0x10, 0x58, 0x7b, 0x27, 0x07, 0x7f, 0xd6, 0xde,
- 0xc9, 0xc5, 0x8d, 0xe3, 0x47, 0xb2, 0x86, 0x11, 0x63, 0x47, 0x72, 0x06, 0xab, 0x8c, 0x1d, 0xc9,
- 0x59, 0xdc, 0x31, 0x0a, 0x9a, 0x96, 0xf8, 0xff, 0x20, 0x89, 0xfd, 0xa1, 0xf8, 0x0f, 0x00, 0xb9,
- 0x60, 0xa3, 0x3e, 0x85, 0x0a, 0x81, 0xc3, 0xd8, 0x7a, 0x7e, 0x01, 0x73, 0x19, 0x30, 0x4f, 0xeb,
- 0x28, 0x42, 0x0f, 0xb5, 0x8e, 0x42, 0x24, 0x30, 0x9a, 0x45, 0x13, 0xca, 0xea, 0xaf, 0x21, 0x74,
- 0x33, 0x1a, 0x95, 0xf8, 0x25, 0xc9, 0x58, 0xcc, 0xd0, 0x53, 0x9e, 0x3d, 0x84, 0xd9, 0x18, 0xd2,
- 0x87, 0xe2, 0x77, 0x44, 0x0a, 0xc1, 0xd3, 0x9e, 0xcd, 0x81, 0x06, 0x63, 0xf3, 0xfe, 0x1d, 0xcf,
- 0x04, 0x86, 0xe0, 0x6e, 0xe8, 0xbd, 0x61, 0xf1, 0x99, 0x56, 0xfa, 0xf8, 0xcd, 0x98, 0x53, 0xb3,
- 0xfa, 0x25, 0x5c, 0x4d, 0x60, 0x48, 0xfa, 0x04, 0xce, 0x03, 0xfa, 0xf4, 0x09, 0x9c, 0x0b, 0x3c,
- 0xc5, 0xe6, 0x76, 0x06, 0x0b, 0x79, 0x39, 0x3f, 0xba, 0xaf, 0x77, 0x45, 0x21, 0x7a, 0x61, 0x3c,
- 0x18, 0xce, 0x94, 0x51, 0xd6, 0x82, 0xb9, 0x0c, 0x80, 0xa2, 0x03, 0xa8, 0x08, 0xd9, 0xd1, 0x01,
- 0x54, 0x88, 0xbe, 0xc4, 0x74, 0x60, 0x40, 0xd9, 0x6a, 0x09, 0x8a, 0x3d, 0x48, 0x0b, 0x8a, 0x36,
- 0xfa, 0x88, 0x1e, 0x52, 0x6c, 0xd1, 0x87, 0x4b, 0x0b, 0xe6, 0x32, 0x05, 0x12, 0x3d, 0x95, 0xa2,
- 0x8a, 0x8c, 0x9e, 0x4a, 0x61, 0x75, 0x25, 0x36, 0x95, 0x57, 0x70, 0x3d, 0x95, 0xe9, 0xa3, 0x3b,
- 0x89, 0x57, 0x43, 0x06, 0x98, 0x30, 0x56, 0x0a, 0xfb, 0x53, 0xf1, 0x44, 0xe0, 0x66, 0x7e, 0x3e,
- 0x8f, 0xde, 0x8a, 0x85, 0x4e, 0x31, 0x94, 0x60, 0x3c, 0xfc, 0x3e, 0xb6, 0xd4, 0xd6, 0x3e, 0x86,
- 0xab, 0x89, 0x54, 0x54, 0x07, 0x70, 0x1e, 0x40, 0xa0, 0x03, 0x38, 0x3f, 0x39, 0x1f, 0x4c, 0xc3,
- 0x4f, 0x65, 0xef, 0x83, 0x04, 0x17, 0x3d, 0xc8, 0x1d, 0x9f, 0x4a, 0xe1, 0x8d, 0xb7, 0xbe, 0x87,
- 0x2b, 0xfb, 0xee, 0x4d, 0x27, 0xb6, 0xf1, 0x64, 0x2b, 0x37, 0x8f, 0x8e, 0x27, 0x5b, 0x05, 0x39,
- 0x71, 0x42, 0x7c, 0x32, 0x43, 0x8d, 0x8b, 0xcf, 0xcd, 0x9a, 0xe3, 0xe2, 0x0b, 0x92, 0xdb, 0x81,
- 0xf8, 0x13, 0x98, 0xcf, 0xc9, 0x2f, 0x51, 0x2c, 0xee, 0x8b, 0x12, 0x60, 0xe3, 0xfe, 0x50, 0x9e,
- 0xa4, 0x9e, 0xe6, 0xd3, 0x57, 0x9c, 0xdb, 0x77, 0x5a, 0xab, 0x2e, 0xe9, 0x3e, 0x91, 0x9f, 0xef,
- 0x13, 0x7a, 0xfa, 0x44, 0xca, 0x78, 0x22, 0x7e, 0x42, 0x7e, 0x72, 0x4a, 0x54, 0xbb, 0xd7, 0x6a,
- 0xcd, 0x08, 0xd2, 0xb3, 0xff, 0x06, 0x00, 0x00, 0xff, 0xff, 0x92, 0xb6, 0xfd, 0x86, 0xc9, 0x2c,
- 0x00, 0x00,
+ // 2949 bytes of a gzipped FileDescriptorProto
+ 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x5a, 0xdd, 0x6f, 0x1b, 0xc7,
+ 0x11, 0x37, 0xa9, 0x0f, 0x92, 0x43, 0xda, 0xa6, 0x56, 0xb2, 0x45, 0x9d, 0x2d, 0xcb, 0x3e, 0x3b,
+ 0x8e, 0x93, 0x38, 0xb2, 0x63, 0xb7, 0x68, 0xda, 0xa2, 0x28, 0x44, 0x7d, 0xc7, 0xb6, 0xa4, 0x9c,
+ 0x94, 0x06, 0x31, 0x10, 0x5c, 0x8e, 0xc7, 0xa5, 0x78, 0xd5, 0xf1, 0x96, 0xde, 0x5b, 0x5a, 0x51,
+ 0xd0, 0x3e, 0xb4, 0x40, 0x5e, 0x03, 0x14, 0x28, 0x9a, 0x3e, 0xf6, 0xa9, 0x0f, 0xfd, 0x1f, 0x8a,
+ 0xa2, 0x2f, 0xfd, 0x1f, 0xf2, 0x57, 0xf4, 0xbd, 0x4f, 0xc5, 0x7e, 0xdc, 0xf7, 0x1d, 0xe3, 0x82,
+ 0x44, 0xfa, 0x76, 0x3b, 0x3b, 0x3b, 0x33, 0x3b, 0x3b, 0xfb, 0x31, 0xbf, 0x39, 0x68, 0x51, 0x3c,
+ 0x24, 0xbe, 0xc3, 0x08, 0xbd, 0x78, 0xdf, 0xc7, 0xf4, 0xb5, 0x63, 0xe3, 0xf5, 0x21, 0x25, 0x8c,
+ 0xa0, 0xf9, 0x53, 0x87, 0x59, 0xee, 0x85, 0xd6, 0xf0, 0xfb, 0x16, 0xc5, 0x5d, 0x49, 0xd5, 0x8f,
+ 0x61, 0xd9, 0x08, 0x47, 0x6c, 0x7f, 0xe9, 0xf8, 0xcc, 0x37, 0xf0, 0xab, 0x11, 0xf6, 0x19, 0xfa,
+ 0x10, 0x20, 0x12, 0xd6, 0x2a, 0xdd, 0x2e, 0x3d, 0xa8, 0x3f, 0x41, 0xeb, 0x52, 0xca, 0x7a, 0x34,
+ 0xa8, 0x3d, 0xfb, 0xe7, 0x7f, 0x3d, 0x2c, 0x19, 0x31, 0x5e, 0xfd, 0x09, 0xb4, 0xb2, 0x42, 0xfd,
+ 0x21, 0xf1, 0x7c, 0x8c, 0xae, 0xc3, 0x3c, 0x16, 0x14, 0x21, 0xb1, 0x6a, 0xa8, 0x96, 0x7e, 0x22,
+ 0xc6, 0x58, 0xf6, 0xd9, 0xbe, 0x67, 0x53, 0x3c, 0xc0, 0x1e, 0xb3, 0xdc, 0xc9, 0x2d, 0xb9, 0x01,
+ 0x2b, 0x39, 0x52, 0xa5, 0x29, 0x3a, 0x85, 0x05, 0xd9, 0xb9, 0x33, 0x72, 0x27, 0xd7, 0x85, 0xee,
+ 0xc2, 0x65, 0x9b, 0x62, 0x8b, 0x61, 0xb3, 0xe3, 0xb0, 0x81, 0x35, 0x6c, 0x95, 0xc5, 0x04, 0x1b,
+ 0x92, 0xd8, 0x16, 0x34, 0x7d, 0x09, 0x50, 0x5c, 0xa7, 0xb2, 0xe4, 0x35, 0x5c, 0xdb, 0xb5, 0x68,
+ 0xc7, 0x3a, 0xc5, 0x9b, 0xc4, 0x75, 0xb1, 0xcd, 0x7e, 0x20, 0x6b, 0x5a, 0x70, 0x3d, 0xad, 0x57,
+ 0x59, 0xf4, 0x11, 0x5c, 0xd9, 0x74, 0xb1, 0xe5, 0x8d, 0x86, 0x93, 0x2f, 0xc2, 0x02, 0x5c, 0x0d,
+ 0x65, 0x29, 0xf1, 0x1f, 0xc3, 0xb5, 0x68, 0xc8, 0xb1, 0xf3, 0x15, 0x9e, 0x5c, 0xcb, 0x43, 0xb8,
+ 0x9e, 0x16, 0xa9, 0x42, 0x0e, 0xc1, 0xac, 0xef, 0x7c, 0x85, 0x85, 0xb4, 0x19, 0x43, 0x7c, 0xeb,
+ 0xaf, 0x60, 0x65, 0x63, 0x38, 0x74, 0x2f, 0x76, 0x1d, 0x66, 0x31, 0x46, 0x9d, 0xce, 0x88, 0xe1,
+ 0xc9, 0x23, 0x1f, 0x69, 0x50, 0xa5, 0xf8, 0xb5, 0xe3, 0x3b, 0xc4, 0x13, 0x0e, 0x6f, 0x18, 0x61,
+ 0x5b, 0xbf, 0x09, 0x5a, 0x9e, 0x4a, 0xe5, 0x91, 0x7f, 0x94, 0x01, 0xed, 0x60, 0x66, 0xf7, 0x0d,
+ 0x3c, 0x20, 0x6c, 0x72, 0x7f, 0xf0, 0x8d, 0x46, 0x85, 0x28, 0x61, 0x48, 0xcd, 0x50, 0x2d, 0xb4,
+ 0x04, 0x73, 0x3d, 0x42, 0x6d, 0xdc, 0x9a, 0x11, 0x01, 0x21, 0x1b, 0x68, 0x19, 0x2a, 0x1e, 0x31,
+ 0x99, 0x75, 0xea, 0xb7, 0x66, 0xe5, 0xbe, 0xf4, 0xc8, 0x89, 0x75, 0xea, 0xa3, 0x16, 0x54, 0x98,
+ 0x33, 0xc0, 0x64, 0xc4, 0x5a, 0x73, 0xb7, 0x4b, 0x0f, 0xe6, 0x8c, 0xa0, 0xc9, 0x87, 0xf8, 0x7e,
+ 0xdf, 0x3c, 0xc3, 0x17, 0xad, 0x79, 0xa9, 0xc1, 0xf7, 0xfb, 0xcf, 0xf0, 0x05, 0x5a, 0x83, 0xfa,
+ 0x99, 0x47, 0xce, 0x3d, 0xb3, 0x4f, 0xf8, 0x3e, 0xaf, 0x88, 0x4e, 0x10, 0xa4, 0x3d, 0x4e, 0x41,
+ 0x2b, 0x50, 0xf5, 0x88, 0x39, 0xa4, 0x23, 0x0f, 0xb7, 0x6a, 0x42, 0x5b, 0xc5, 0x23, 0x47, 0xbc,
+ 0x89, 0x9e, 0xc2, 0x65, 0x69, 0xa7, 0x39, 0xb4, 0xa8, 0x35, 0xf0, 0x5b, 0x20, 0xa6, 0x7c, 0x25,
+ 0x9a, 0xb2, 0xf0, 0x4e, 0x43, 0x32, 0x1d, 0x09, 0x9e, 0x8f, 0x66, 0xab, 0xd5, 0x66, 0x4d, 0xbf,
+ 0x06, 0x8b, 0x09, 0x07, 0x2a, 0xc7, 0x1e, 0xc3, 0xf2, 0xa6, 0x88, 0xf9, 0xc8, 0x5b, 0x93, 0x07,
+ 0x9b, 0x06, 0xad, 0xac, 0x50, 0xa5, 0xf0, 0xeb, 0x32, 0x2c, 0xec, 0x62, 0xb6, 0x41, 0xed, 0xbe,
+ 0xf3, 0x7a, 0x0a, 0x0b, 0x79, 0x03, 0x6a, 0x36, 0x19, 0x0c, 0x1c, 0x66, 0x3a, 0x5d, 0xb5, 0x96,
+ 0x55, 0x49, 0xd8, 0xef, 0xf2, 0x55, 0x1e, 0x52, 0xdc, 0x73, 0xbe, 0x14, 0xcb, 0x59, 0x33, 0x54,
+ 0x0b, 0x7d, 0x08, 0xf3, 0x3d, 0x42, 0x07, 0x16, 0x13, 0xcb, 0x79, 0xe5, 0xc9, 0xed, 0x40, 0x55,
+ 0xc6, 0xb2, 0xf5, 0x1d, 0xc1, 0x67, 0x28, 0x7e, 0xbe, 0x5b, 0x86, 0x16, 0xeb, 0x8b, 0xd5, 0x6e,
+ 0x18, 0xe2, 0x5b, 0x7f, 0x0a, 0xf3, 0x92, 0x0b, 0x55, 0x60, 0xe6, 0xe5, 0xfe, 0x51, 0xf3, 0x12,
+ 0xff, 0x38, 0xd9, 0x30, 0x9a, 0x25, 0x04, 0x30, 0x7f, 0xb2, 0x61, 0x98, 0xbb, 0x2f, 0x9b, 0x65,
+ 0x54, 0x87, 0x0a, 0xff, 0x6e, 0xbf, 0x7c, 0xd2, 0x9c, 0xd1, 0x1f, 0x00, 0x8a, 0x2b, 0x8b, 0x36,
+ 0x63, 0xd7, 0x62, 0x96, 0xf0, 0x40, 0xc3, 0x10, 0xdf, 0x7c, 0x89, 0xf6, 0x2c, 0xff, 0x39, 0xb1,
+ 0x2d, 0xb7, 0x4d, 0x2d, 0xcf, 0xee, 0x4f, 0x61, 0x2b, 0xea, 0x8f, 0xa1, 0x95, 0x15, 0xaa, 0x8c,
+ 0x58, 0x82, 0xb9, 0xd7, 0x96, 0x3b, 0xc2, 0xea, 0x0e, 0x92, 0x0d, 0xfd, 0xbb, 0x12, 0xb4, 0x44,
+ 0x04, 0x1d, 0x93, 0x11, 0xb5, 0xb1, 0x1c, 0x35, 0xf9, 0xfa, 0xfd, 0x12, 0x16, 0x7c, 0x21, 0xd0,
+ 0x8c, 0x09, 0x28, 0x17, 0x09, 0x30, 0x9a, 0x92, 0xd9, 0x48, 0x1c, 0xe5, 0x4a, 0x40, 0x47, 0x98,
+ 0x24, 0x96, 0xba, 0x61, 0x34, 0xfc, 0x98, 0x99, 0x68, 0x15, 0x80, 0x59, 0xf4, 0x14, 0x33, 0x93,
+ 0xe2, 0x9e, 0x58, 0xf4, 0x86, 0x51, 0x93, 0x14, 0x03, 0xf7, 0xf4, 0xa7, 0xb0, 0x92, 0x33, 0xb5,
+ 0xe8, 0x4e, 0xa6, 0xd8, 0x1f, 0xb9, 0x2c, 0xb8, 0x93, 0x65, 0x4b, 0xdf, 0x85, 0xfa, 0x8e, 0x6f,
+ 0x9f, 0x4d, 0xbe, 0x16, 0xf7, 0xa0, 0x21, 0x05, 0x45, 0xfe, 0xc7, 0x94, 0x12, 0xaa, 0xa2, 0x40,
+ 0x36, 0xf4, 0x7f, 0x97, 0xe0, 0xea, 0xa7, 0xd4, 0xe1, 0x9b, 0xaa, 0x37, 0xb9, 0xdb, 0x9b, 0x30,
+ 0xc3, 0x3d, 0x21, 0x4f, 0x61, 0xfe, 0x99, 0x38, 0x9c, 0x67, 0x92, 0x87, 0x33, 0xba, 0x03, 0x0d,
+ 0xe2, 0x76, 0xcd, 0xb0, 0x5f, 0x3a, 0xb0, 0x4e, 0xdc, 0xae, 0x11, 0xb0, 0x84, 0x07, 0xe7, 0x5c,
+ 0xfc, 0xe0, 0xfc, 0x31, 0xd4, 0x19, 0xb5, 0x3c, 0xdf, 0xb2, 0x19, 0x1f, 0x57, 0x11, 0x16, 0x2e,
+ 0x06, 0x16, 0x9e, 0x44, 0x5d, 0x46, 0x9c, 0xef, 0xa3, 0xd9, 0xea, 0x7c, 0xb3, 0xa2, 0x1f, 0x42,
+ 0x33, 0x9a, 0xb0, 0xf2, 0x4d, 0x4a, 0x60, 0xf9, 0x8d, 0x05, 0x96, 0x9a, 0x65, 0xdd, 0x83, 0xa5,
+ 0x1d, 0xc7, 0xeb, 0xbe, 0xc0, 0xf4, 0x14, 0xb7, 0x2d, 0x7f, 0x0a, 0xa7, 0xcf, 0x4d, 0xa8, 0x05,
+ 0x4e, 0xf1, 0x5b, 0xe5, 0xdb, 0x33, 0x3c, 0xac, 0x42, 0x82, 0xfe, 0x1e, 0x5c, 0x4b, 0xe9, 0x8b,
+ 0xb6, 0x79, 0xc7, 0xf2, 0xe5, 0x06, 0xab, 0x19, 0xe2, 0x5b, 0xff, 0xa6, 0x04, 0x0b, 0xf2, 0xd4,
+ 0xdc, 0x21, 0xf4, 0xec, 0xff, 0xbf, 0xb1, 0xf8, 0x63, 0x2c, 0x6e, 0x4f, 0xf8, 0x2c, 0x5c, 0xd9,
+ 0xf7, 0x0d, 0xcc, 0x4d, 0xde, 0xf7, 0x8e, 0x28, 0x39, 0xa5, 0xd8, 0xf7, 0xa7, 0x72, 0x8c, 0x53,
+ 0x21, 0x34, 0x76, 0x8c, 0x4b, 0xc2, 0x7e, 0x57, 0xff, 0x05, 0x68, 0x79, 0x3a, 0x95, 0x33, 0xd7,
+ 0xa0, 0xee, 0x78, 0xe6, 0x50, 0x91, 0xd5, 0x26, 0x05, 0x27, 0x64, 0x94, 0x26, 0x1f, 0xbf, 0x1a,
+ 0x59, 0x7e, 0x7f, 0xca, 0x26, 0xfb, 0x42, 0x68, 0xcc, 0x64, 0x49, 0x08, 0x4c, 0xce, 0xea, 0x7c,
+ 0x53, 0x93, 0x5d, 0xb8, 0x95, 0xbe, 0x41, 0x77, 0x28, 0x19, 0x7c, 0x62, 0x3c, 0x9f, 0xca, 0xd6,
+ 0x1f, 0x51, 0x57, 0x59, 0xcc, 0x3f, 0xf5, 0x3b, 0xb0, 0x56, 0xa8, 0x4d, 0x2d, 0xfb, 0x21, 0x2c,
+ 0x4a, 0x96, 0xf6, 0xc8, 0xeb, 0xba, 0x53, 0x78, 0x90, 0xbe, 0x0b, 0x4b, 0x49, 0x81, 0x63, 0x6e,
+ 0xc0, 0x6f, 0xca, 0xd0, 0x3c, 0xc6, 0x6c, 0x93, 0x78, 0x3d, 0xe7, 0x74, 0x72, 0x07, 0x7c, 0x08,
+ 0x15, 0xec, 0x31, 0xea, 0x60, 0xb9, 0x65, 0xeb, 0x4f, 0x6e, 0x05, 0xc3, 0xd2, 0x4a, 0xd6, 0xb7,
+ 0x3d, 0x46, 0x2f, 0x8c, 0x80, 0x5d, 0xfb, 0xba, 0x04, 0x73, 0x82, 0xc4, 0x9d, 0xc8, 0x9f, 0x76,
+ 0x72, 0x03, 0xf3, 0x4f, 0xb4, 0x0a, 0x35, 0x71, 0x51, 0x9a, 0x3e, 0xa3, 0xd2, 0xb9, 0x7b, 0x97,
+ 0x8c, 0xaa, 0x20, 0x1d, 0x33, 0x8a, 0xee, 0x40, 0x5d, 0x76, 0x3b, 0x1e, 0x7b, 0xfa, 0x44, 0x9c,
+ 0xb0, 0x73, 0x7b, 0x97, 0x0c, 0x10, 0xc4, 0x7d, 0x4e, 0x43, 0x6b, 0x20, 0x5b, 0x66, 0x87, 0x10,
+ 0x57, 0x3e, 0x34, 0xf7, 0x2e, 0x19, 0x52, 0x6a, 0x9b, 0x10, 0xb7, 0x5d, 0x51, 0x17, 0xb3, 0xbe,
+ 0x08, 0x0b, 0x31, 0x53, 0xd5, 0x12, 0xd9, 0xb0, 0xb8, 0x85, 0x5d, 0xcc, 0xf0, 0xb4, 0xfc, 0x84,
+ 0x60, 0xf6, 0x0c, 0x5f, 0x48, 0x27, 0xd5, 0x0c, 0xf1, 0xad, 0x5f, 0x87, 0xa5, 0xa4, 0x12, 0xa5,
+ 0xdc, 0xe1, 0xa9, 0xa4, 0xcf, 0x08, 0xc5, 0x9b, 0x23, 0x9f, 0x91, 0xc1, 0x1e, 0x21, 0x67, 0xfe,
+ 0x54, 0x4c, 0x10, 0xd1, 0x50, 0x8e, 0x45, 0xc3, 0x4d, 0xd0, 0xf2, 0x54, 0x29, 0x43, 0x4e, 0xa0,
+ 0xd5, 0xb6, 0xec, 0xb3, 0xd1, 0x70, 0x9a, 0x76, 0xe8, 0x8f, 0x60, 0x25, 0x47, 0xea, 0x98, 0x90,
+ 0x7d, 0x05, 0x77, 0xf2, 0xb6, 0xd4, 0x94, 0x76, 0x4f, 0xae, 0x5f, 0xee, 0x81, 0x3e, 0x4e, 0xa5,
+ 0xf2, 0xcf, 0x01, 0x20, 0x7e, 0x27, 0x3d, 0x77, 0x6c, 0xec, 0x4d, 0xe1, 0x06, 0xd4, 0x37, 0x61,
+ 0x31, 0x21, 0x4f, 0xf9, 0xe4, 0x21, 0x20, 0x57, 0x92, 0x4c, 0xbf, 0x4f, 0x28, 0x33, 0x3d, 0x6b,
+ 0x10, 0xdc, 0x77, 0x4d, 0xd5, 0x73, 0xcc, 0x3b, 0x0e, 0xac, 0x81, 0x58, 0xb4, 0x5d, 0xcc, 0xf6,
+ 0xbd, 0x1e, 0xd9, 0x98, 0x5e, 0xba, 0xa9, 0xff, 0x1c, 0x56, 0x72, 0xa4, 0x2a, 0x03, 0x6f, 0x01,
+ 0x44, 0x79, 0xa6, 0x5a, 0xba, 0x18, 0x85, 0x9b, 0xb4, 0x69, 0xb9, 0xf6, 0xc8, 0xb5, 0x18, 0xde,
+ 0xec, 0x63, 0xfb, 0xcc, 0x1f, 0x0d, 0x26, 0x37, 0xe9, 0x27, 0xb0, 0x92, 0x23, 0x55, 0x99, 0xa4,
+ 0x41, 0xd5, 0x56, 0x34, 0xe5, 0xa9, 0xb0, 0xcd, 0x97, 0x6d, 0x17, 0xb3, 0x63, 0xcf, 0x1a, 0xfa,
+ 0x7d, 0x32, 0x39, 0x00, 0xa2, 0xbf, 0x03, 0x8b, 0x09, 0x79, 0x63, 0x42, 0xf9, 0xdb, 0x12, 0xdc,
+ 0xcd, 0x0b, 0xac, 0xa9, 0x19, 0xc3, 0x33, 0xde, 0x3e, 0x63, 0x43, 0x33, 0xba, 0x96, 0x2a, 0xbc,
+ 0xfd, 0x09, 0x75, 0xf9, 0x25, 0x2b, 0xba, 0xac, 0x11, 0xeb, 0xab, 0x24, 0x4e, 0xf0, 0x6e, 0x8c,
+ 0x58, 0x5f, 0xbf, 0x0f, 0xf7, 0xc6, 0x1b, 0xa6, 0x62, 0xfe, 0x4f, 0x25, 0x58, 0xda, 0xc5, 0xcc,
+ 0xb0, 0xce, 0x37, 0xfb, 0x96, 0x77, 0x3a, 0x0d, 0x28, 0xe3, 0x2e, 0x5c, 0xee, 0x51, 0x32, 0x30,
+ 0x13, 0x78, 0x46, 0xcd, 0x68, 0x70, 0x62, 0xf8, 0x26, 0x5e, 0x83, 0x3a, 0x23, 0x66, 0xe2, 0x55,
+ 0x5d, 0x33, 0x80, 0x91, 0x80, 0x41, 0xff, 0xfb, 0x2c, 0x5c, 0x4b, 0x19, 0xa6, 0x16, 0x62, 0x0f,
+ 0xea, 0xd4, 0x3a, 0x37, 0x6d, 0x49, 0x6e, 0x95, 0xc4, 0x3d, 0xf5, 0x76, 0x2c, 0x4d, 0xcd, 0x8e,
+ 0x59, 0x0f, 0x49, 0x06, 0xd0, 0xb0, 0x57, 0xfb, 0x6e, 0x06, 0x6a, 0x61, 0x0f, 0x5a, 0x86, 0x4a,
+ 0xc7, 0x25, 0x1d, 0xfe, 0x64, 0x91, 0x21, 0x36, 0xcf, 0x9b, 0xfb, 0xdd, 0x10, 0x06, 0x2a, 0x47,
+ 0x30, 0x10, 0x5a, 0x85, 0xaa, 0x87, 0xcf, 0x4d, 0x91, 0xf0, 0x0a, 0xe3, 0xdb, 0xe5, 0x56, 0xc9,
+ 0xa8, 0x78, 0xf8, 0xfc, 0xc8, 0x62, 0x3c, 0xa9, 0xaa, 0xf2, 0xac, 0x40, 0x74, 0xcf, 0x46, 0xdd,
+ 0xc4, 0xed, 0x8a, 0xee, 0x43, 0xa8, 0x91, 0x21, 0xa6, 0x96, 0x78, 0xa8, 0xcf, 0x89, 0x3c, 0xfb,
+ 0x83, 0x37, 0x9c, 0xc0, 0xfa, 0x61, 0x30, 0xd0, 0x88, 0x64, 0x70, 0x9f, 0x73, 0x9f, 0x44, 0x42,
+ 0x25, 0xb0, 0xd2, 0xa0, 0xd6, 0x79, 0xc8, 0xcf, 0x63, 0x89, 0x1b, 0x35, 0x20, 0x5d, 0x2c, 0xd2,
+ 0x8d, 0x39, 0x61, 0xd0, 0x0b, 0xd2, 0xc5, 0x02, 0x58, 0xc1, 0xe7, 0xb2, 0xab, 0x2a, 0xbb, 0x3c,
+ 0x7c, 0x2e, 0xba, 0xee, 0xc1, 0x95, 0x60, 0xa6, 0x66, 0xe7, 0x82, 0x9f, 0x08, 0x35, 0x99, 0x45,
+ 0xaa, 0xb9, 0xb6, 0x39, 0x8d, 0x73, 0x05, 0x13, 0x56, 0x5c, 0x20, 0xb9, 0xd4, 0x94, 0x05, 0x97,
+ 0xee, 0x40, 0x2d, 0x32, 0xa7, 0x0e, 0x95, 0x4f, 0x0e, 0x9e, 0x1d, 0x1c, 0x7e, 0x7a, 0xd0, 0xbc,
+ 0x84, 0x6a, 0x30, 0xb7, 0xb1, 0xb5, 0xb5, 0xbd, 0x25, 0x71, 0x81, 0xcd, 0xc3, 0xa3, 0xfd, 0xed,
+ 0x2d, 0x89, 0x0b, 0x6c, 0x6d, 0x3f, 0xdf, 0x3e, 0xd9, 0xde, 0x6a, 0xce, 0xa0, 0x06, 0x54, 0x5f,
+ 0x1c, 0x6e, 0xed, 0xef, 0xf0, 0xae, 0x59, 0xde, 0x65, 0x6c, 0x1f, 0x6c, 0xbc, 0xd8, 0xde, 0x6a,
+ 0xce, 0xa1, 0x26, 0x34, 0x4e, 0x3e, 0x3b, 0xda, 0x36, 0x37, 0xf7, 0x36, 0x0e, 0x76, 0xb7, 0xb7,
+ 0x9a, 0xf3, 0xfa, 0x6f, 0xa0, 0x75, 0x8c, 0x2d, 0x6a, 0xf7, 0x77, 0x1c, 0x17, 0xfb, 0xed, 0x0b,
+ 0x7e, 0x98, 0x4e, 0x1e, 0xdb, 0x4b, 0x30, 0xf7, 0x6a, 0x84, 0x55, 0xb6, 0x50, 0x33, 0x64, 0x23,
+ 0xc8, 0x18, 0x67, 0xc2, 0x8c, 0x51, 0xff, 0x00, 0x56, 0x72, 0xb4, 0x47, 0x49, 0x6c, 0x8f, 0x93,
+ 0x45, 0xe8, 0x36, 0x0c, 0xd9, 0xd0, 0xff, 0x56, 0x82, 0x1b, 0x89, 0x31, 0x9b, 0xc4, 0x63, 0xd8,
+ 0x63, 0x3f, 0x98, 0xd1, 0xe8, 0x1d, 0x68, 0xda, 0xfd, 0x91, 0x77, 0x86, 0x79, 0x3a, 0x2b, 0x6d,
+ 0x55, 0x98, 0xde, 0x55, 0x45, 0x0f, 0x8f, 0x8d, 0x0b, 0xb8, 0x99, 0x6f, 0xab, 0x9a, 0x62, 0x0b,
+ 0x2a, 0x03, 0x8b, 0xd9, 0xfd, 0x70, 0x92, 0x41, 0x13, 0xad, 0x02, 0x88, 0x4f, 0x33, 0x76, 0x49,
+ 0xd7, 0x04, 0x65, 0xcb, 0x62, 0x16, 0xba, 0x0d, 0x0d, 0xec, 0x75, 0x4d, 0xd2, 0x33, 0x05, 0x4d,
+ 0x61, 0x8d, 0x80, 0xbd, 0xee, 0x61, 0xef, 0x05, 0xa7, 0xe8, 0x7f, 0x28, 0xc1, 0xbc, 0x44, 0xea,
+ 0x82, 0xe7, 0x7a, 0x29, 0x7c, 0xae, 0xf3, 0xad, 0x2a, 0x6e, 0x53, 0x39, 0x53, 0xf1, 0x8d, 0x7e,
+ 0x06, 0x2b, 0xe1, 0x39, 0x49, 0xa8, 0xf3, 0x95, 0x88, 0x3e, 0xb3, 0x8f, 0xad, 0x2e, 0xa6, 0xea,
+ 0xe0, 0x59, 0x0e, 0xce, 0xcd, 0xb0, 0x7f, 0x4f, 0x74, 0xa3, 0xb7, 0xe0, 0xca, 0xc0, 0xa1, 0x94,
+ 0x50, 0x93, 0xe2, 0xde, 0xc0, 0x1a, 0xfa, 0xad, 0x59, 0xf1, 0xe2, 0xbb, 0x2c, 0xa9, 0x86, 0x24,
+ 0xea, 0x7f, 0x2c, 0xc1, 0x75, 0x81, 0x92, 0xec, 0x9d, 0x9c, 0x1c, 0x4d, 0x0b, 0x87, 0xbd, 0x9f,
+ 0xc0, 0x61, 0xb3, 0x50, 0x66, 0x80, 0xcb, 0xc6, 0x80, 0xd6, 0x99, 0x04, 0xd0, 0xaa, 0xaf, 0xc0,
+ 0x72, 0xc6, 0x2a, 0xb5, 0x80, 0x9f, 0xc1, 0xea, 0x2e, 0x66, 0x87, 0x9d, 0x5f, 0x63, 0x9b, 0x6d,
+ 0x39, 0x14, 0xdb, 0xd3, 0xc3, 0xd3, 0x7f, 0x04, 0xb7, 0x8a, 0x44, 0x8f, 0xc1, 0xd5, 0xff, 0x52,
+ 0x82, 0xa5, 0x4d, 0x97, 0x78, 0x98, 0x5f, 0x53, 0x47, 0x84, 0xb8, 0xd3, 0x70, 0xe0, 0xec, 0x90,
+ 0xa7, 0x0b, 0xa9, 0xcc, 0x5e, 0x5a, 0x26, 0x54, 0x88, 0xfe, 0x98, 0xa3, 0x67, 0xc6, 0x39, 0x5a,
+ 0x5f, 0x86, 0x6b, 0x29, 0x0b, 0x95, 0x33, 0xff, 0x59, 0x82, 0x9b, 0x89, 0x9e, 0x7d, 0x8f, 0x61,
+ 0xea, 0x59, 0x3f, 0xe0, 0x1c, 0x72, 0x21, 0x8d, 0x99, 0xff, 0x01, 0xd2, 0x58, 0x83, 0xd5, 0x82,
+ 0x29, 0x44, 0x70, 0x38, 0xf7, 0xc7, 0xeb, 0x69, 0xc3, 0xe1, 0x59, 0xa1, 0x4a, 0xe1, 0x97, 0x5c,
+ 0xa1, 0x27, 0x0e, 0xce, 0xa9, 0x29, 0x14, 0x17, 0x25, 0x76, 0x2d, 0xe6, 0xbc, 0xc6, 0xf2, 0x76,
+ 0x56, 0x8f, 0x93, 0x80, 0xc8, 0xef, 0x2a, 0x69, 0x55, 0x5a, 0xb3, 0xb2, 0xea, 0xf7, 0x25, 0x9e,
+ 0x63, 0x0d, 0x5d, 0xc7, 0x9e, 0x6e, 0x65, 0x00, 0xbd, 0x0b, 0xf3, 0x72, 0x51, 0xc6, 0x20, 0x51,
+ 0x8a, 0x43, 0x5f, 0x85, 0x1b, 0xb9, 0x36, 0x48, 0x1b, 0x9f, 0xfc, 0x75, 0x55, 0x14, 0x28, 0x83,
+ 0x92, 0x96, 0xac, 0xe6, 0xa2, 0xcf, 0xa1, 0x99, 0x2e, 0xae, 0xa2, 0xb5, 0xac, 0x92, 0x44, 0x2d,
+ 0x57, 0xbb, 0x5d, 0xcc, 0xa0, 0x1c, 0x32, 0xff, 0x9f, 0x6f, 0x1f, 0x94, 0xab, 0x65, 0xf4, 0x45,
+ 0x50, 0x14, 0x8d, 0x55, 0x4c, 0x51, 0x7c, 0x78, 0x6e, 0x89, 0x56, 0xbb, 0x33, 0x86, 0x23, 0xa1,
+ 0xa1, 0x84, 0x9e, 0x01, 0x44, 0x25, 0x50, 0xb4, 0x92, 0x1c, 0x18, 0x2b, 0xc5, 0x6a, 0x5a, 0x5e,
+ 0x57, 0x4a, 0xd8, 0xa7, 0x70, 0x25, 0x59, 0xc1, 0x44, 0xab, 0xe1, 0x0b, 0x2c, 0xaf, 0xa2, 0xaa,
+ 0xdd, 0x2a, 0xea, 0xce, 0x0a, 0x4e, 0x96, 0x13, 0x23, 0xc1, 0xb9, 0x95, 0xcb, 0x48, 0x70, 0x7e,
+ 0x15, 0x32, 0x14, 0x6c, 0x03, 0xca, 0x96, 0x01, 0x51, 0xe8, 0xbf, 0xc2, 0xaa, 0xa4, 0xa6, 0x8f,
+ 0x63, 0x49, 0x29, 0x39, 0x80, 0x7a, 0xac, 0x16, 0x86, 0x42, 0x4f, 0x66, 0x2b, 0x8c, 0xda, 0x8d,
+ 0xdc, 0xbe, 0x94, 0xbc, 0xcf, 0xa1, 0x99, 0xce, 0x43, 0xa2, 0xa0, 0x2b, 0x28, 0xaf, 0x45, 0x41,
+ 0x57, 0x58, 0x2a, 0x0b, 0xc4, 0xbf, 0x00, 0x88, 0x4a, 0x45, 0x51, 0x48, 0x64, 0x6a, 0x55, 0x51,
+ 0x48, 0x64, 0x2b, 0x4b, 0x81, 0xb0, 0xc7, 0xc2, 0xda, 0x74, 0xe9, 0x27, 0xb2, 0xb6, 0xa0, 0xd2,
+ 0x14, 0x59, 0x5b, 0x54, 0x35, 0x8a, 0x6f, 0x91, 0x4c, 0x2d, 0x25, 0xda, 0x22, 0x45, 0x15, 0xa4,
+ 0x68, 0x8b, 0x14, 0x16, 0x62, 0x42, 0x7f, 0xfc, 0x14, 0x66, 0x77, 0x7c, 0xfb, 0x0c, 0x85, 0xb0,
+ 0x7f, 0xac, 0x0c, 0xa3, 0x2d, 0x25, 0x89, 0xa9, 0xa1, 0xdb, 0x50, 0x0d, 0x4a, 0x0a, 0x68, 0x39,
+ 0xe0, 0x4c, 0x55, 0x55, 0xb4, 0x56, 0xb6, 0x23, 0x25, 0xe6, 0x04, 0x2e, 0x27, 0x80, 0x7d, 0x74,
+ 0x33, 0xd4, 0x9a, 0x53, 0x5f, 0xd0, 0x56, 0x0b, 0x7a, 0x53, 0x9e, 0x7b, 0x06, 0x10, 0x01, 0xee,
+ 0xd1, 0x3a, 0x67, 0x8a, 0x02, 0xd1, 0x3a, 0xe7, 0xe0, 0xf3, 0xb1, 0x8d, 0x94, 0xc5, 0xcc, 0xa3,
+ 0x8d, 0x54, 0x88, 0xe1, 0x47, 0x1b, 0xa9, 0x18, 0x72, 0x0f, 0x2d, 0x16, 0x4a, 0xd2, 0x28, 0x77,
+ 0x5c, 0x49, 0x01, 0xea, 0x1e, 0x57, 0x52, 0x04, 0x92, 0x87, 0x4a, 0x68, 0xb6, 0x44, 0xad, 0xd0,
+ 0x69, 0x74, 0xbf, 0x68, 0x0f, 0x25, 0xc1, 0x72, 0xed, 0xed, 0xef, 0xe5, 0x4b, 0x79, 0xef, 0x18,
+ 0x1a, 0x71, 0x74, 0x1a, 0xdd, 0x48, 0x0a, 0x48, 0xc0, 0x78, 0xda, 0xcd, 0xfc, 0xce, 0xe4, 0x34,
+ 0x1e, 0x97, 0xd0, 0x6f, 0x41, 0x2b, 0x06, 0xe8, 0xd0, 0x3b, 0xe3, 0x6c, 0x4c, 0x2a, 0x7c, 0xf7,
+ 0x4d, 0x58, 0x93, 0x33, 0x7a, 0x50, 0x42, 0x7b, 0x50, 0x0b, 0x41, 0x63, 0xd4, 0x2a, 0x82, 0xbc,
+ 0xb5, 0x95, 0x9c, 0x9e, 0x94, 0x77, 0x3e, 0x86, 0x46, 0x1c, 0x04, 0x8e, 0xbc, 0x93, 0x83, 0x3f,
+ 0x47, 0xde, 0xc9, 0xc5, 0x8d, 0xe3, 0x47, 0x72, 0x04, 0x23, 0xc6, 0x8e, 0xe4, 0x0c, 0x56, 0x19,
+ 0x3b, 0x92, 0xb3, 0xb8, 0x63, 0x18, 0x34, 0x1d, 0xf1, 0x97, 0x41, 0x12, 0xfb, 0x43, 0xf1, 0x32,
+ 0x7f, 0x2e, 0xd8, 0x18, 0x9d, 0x42, 0x85, 0xc0, 0x61, 0x6c, 0x3d, 0xbf, 0x80, 0x85, 0x0c, 0x98,
+ 0x17, 0xe9, 0x28, 0x42, 0x0f, 0x23, 0x1d, 0x85, 0x48, 0x60, 0x38, 0x8b, 0x36, 0x54, 0xd4, 0xbf,
+ 0x41, 0xe8, 0x7a, 0x38, 0x2a, 0xf1, 0xe3, 0x91, 0xb6, 0x9c, 0xa1, 0xa7, 0x3c, 0x7b, 0x04, 0xf5,
+ 0x18, 0xd2, 0x87, 0xe2, 0x77, 0x44, 0x0a, 0xc1, 0x8b, 0x3c, 0x9b, 0x03, 0x0d, 0xc6, 0xe6, 0xfd,
+ 0x3b, 0x9e, 0x09, 0x8c, 0xc1, 0xdd, 0xd0, 0x7b, 0xe3, 0xe2, 0x33, 0xad, 0xf4, 0xe1, 0x9b, 0x31,
+ 0xa7, 0x66, 0xf5, 0x2b, 0xb8, 0x9c, 0xc0, 0x90, 0xa2, 0x13, 0x38, 0x0f, 0xe8, 0x8b, 0x4e, 0xe0,
+ 0x5c, 0xe0, 0x29, 0x36, 0xb7, 0x33, 0x58, 0xca, 0xcb, 0xf9, 0xd1, 0xdd, 0x68, 0x57, 0x14, 0xa2,
+ 0x17, 0xda, 0xbd, 0xf1, 0x4c, 0x19, 0x65, 0x1d, 0x58, 0xc8, 0x00, 0x28, 0x51, 0x00, 0x15, 0x21,
+ 0x3b, 0x51, 0x00, 0x15, 0xa2, 0x2f, 0x31, 0x1d, 0x18, 0x50, 0xb6, 0x5a, 0x82, 0x62, 0x0f, 0xd2,
+ 0x82, 0xa2, 0x4d, 0x74, 0x44, 0x8f, 0x29, 0xb6, 0x44, 0x87, 0x4b, 0x07, 0x16, 0x32, 0x05, 0x92,
+ 0x68, 0x2a, 0x45, 0x15, 0x99, 0x68, 0x2a, 0x85, 0xd5, 0x95, 0xd8, 0x54, 0x5e, 0xc2, 0xd5, 0x54,
+ 0xa6, 0x8f, 0x6e, 0x25, 0x5e, 0x0d, 0x19, 0x60, 0x42, 0x5b, 0x2b, 0xec, 0x4f, 0xc5, 0x13, 0x81,
+ 0xeb, 0xf9, 0xf9, 0x3c, 0x7a, 0x2b, 0x16, 0x3a, 0xc5, 0x50, 0x82, 0x76, 0xff, 0xfb, 0xd8, 0x52,
+ 0x5b, 0xfb, 0x04, 0x2e, 0x27, 0x52, 0xd1, 0x28, 0x80, 0xf3, 0x00, 0x82, 0x28, 0x80, 0xf3, 0x93,
+ 0xf3, 0x60, 0x1a, 0x6e, 0x2a, 0x7b, 0x0f, 0x12, 0x5c, 0x74, 0x2f, 0x77, 0x7c, 0x2a, 0x85, 0xd7,
+ 0xde, 0xfa, 0x1e, 0xae, 0xec, 0xbb, 0x37, 0x9d, 0xd8, 0xc6, 0x93, 0xad, 0xdc, 0x3c, 0x3a, 0x9e,
+ 0x6c, 0x15, 0xe4, 0xc4, 0x09, 0xf1, 0xc9, 0x0c, 0x35, 0x2e, 0x3e, 0x37, 0x6b, 0x8e, 0x8b, 0x2f,
+ 0x48, 0x6e, 0x03, 0xf1, 0x3d, 0x58, 0xcc, 0xc9, 0x2f, 0x51, 0x2c, 0xee, 0x8b, 0x12, 0x60, 0xed,
+ 0xee, 0x58, 0x9e, 0xa4, 0x9e, 0xf6, 0xe3, 0x97, 0x9c, 0xdb, 0xb5, 0x3a, 0xeb, 0x36, 0x19, 0x3c,
+ 0x92, 0x9f, 0xef, 0x13, 0x7a, 0xfa, 0x48, 0xca, 0x78, 0x24, 0x7e, 0x35, 0x7e, 0x74, 0x4a, 0x54,
+ 0x7b, 0xd8, 0xe9, 0xcc, 0x0b, 0xd2, 0xd3, 0xff, 0x06, 0x00, 0x00, 0xff, 0xff, 0x6a, 0xc3, 0xa4,
+ 0x50, 0xaf, 0x2c, 0x00, 0x00,
}
// Reference imports to suppress errors if they are not otherwise used.
diff --git a/proto/go/gitalypb/shared.pb.go b/proto/go/gitalypb/shared.pb.go
index cd5daac5d..e053478f0 100644
--- a/proto/go/gitalypb/shared.pb.go
+++ b/proto/go/gitalypb/shared.pb.go
@@ -140,31 +140,31 @@ func (OperationMsg_Scope) EnumDescriptor() ([]byte, []int) {
return fileDescriptor_d8a4e87e678c5ced, []int{0, 1}
}
-type TransactionStep_TransactionStep int32
+type Transaction_TransactionStep int32
const (
- TransactionStep_PRECOMMIT TransactionStep_TransactionStep = 0
- TransactionStep_COMMIT TransactionStep_TransactionStep = 1
- TransactionStep_ROLLBACK TransactionStep_TransactionStep = 2
+ Transaction_PRECOMMIT Transaction_TransactionStep = 0
+ Transaction_COMMIT Transaction_TransactionStep = 1
+ Transaction_ROLLBACK Transaction_TransactionStep = 2
)
-var TransactionStep_TransactionStep_name = map[int32]string{
+var Transaction_TransactionStep_name = map[int32]string{
0: "PRECOMMIT",
1: "COMMIT",
2: "ROLLBACK",
}
-var TransactionStep_TransactionStep_value = map[string]int32{
+var Transaction_TransactionStep_value = map[string]int32{
"PRECOMMIT": 0,
"COMMIT": 1,
"ROLLBACK": 2,
}
-func (x TransactionStep_TransactionStep) String() string {
- return proto.EnumName(TransactionStep_TransactionStep_name, int32(x))
+func (x Transaction_TransactionStep) String() string {
+ return proto.EnumName(Transaction_TransactionStep_name, int32(x))
}
-func (TransactionStep_TransactionStep) EnumDescriptor() ([]byte, []int) {
+func (Transaction_TransactionStep) EnumDescriptor() ([]byte, []int) {
return fileDescriptor_d8a4e87e678c5ced, []int{1, 0}
}
@@ -219,51 +219,43 @@ func (m *OperationMsg) GetScopeLevel() OperationMsg_Scope {
return OperationMsg_REPOSITORY
}
-type TransactionStep struct {
- Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"`
- Step TransactionStep_TransactionStep `protobuf:"varint,2,opt,name=step,proto3,enum=gitaly.TransactionStep_TransactionStep" json:"step,omitempty"`
- XXX_NoUnkeyedLiteral struct{} `json:"-"`
- XXX_unrecognized []byte `json:"-"`
- XXX_sizecache int32 `json:"-"`
+type Transaction struct {
+ Step Transaction_TransactionStep `protobuf:"varint,1,opt,name=step,proto3,enum=gitaly.Transaction_TransactionStep" json:"step,omitempty"`
+ XXX_NoUnkeyedLiteral struct{} `json:"-"`
+ XXX_unrecognized []byte `json:"-"`
+ XXX_sizecache int32 `json:"-"`
}
-func (m *TransactionStep) Reset() { *m = TransactionStep{} }
-func (m *TransactionStep) String() string { return proto.CompactTextString(m) }
-func (*TransactionStep) ProtoMessage() {}
-func (*TransactionStep) Descriptor() ([]byte, []int) {
+func (m *Transaction) Reset() { *m = Transaction{} }
+func (m *Transaction) String() string { return proto.CompactTextString(m) }
+func (*Transaction) ProtoMessage() {}
+func (*Transaction) Descriptor() ([]byte, []int) {
return fileDescriptor_d8a4e87e678c5ced, []int{1}
}
-func (m *TransactionStep) XXX_Unmarshal(b []byte) error {
- return xxx_messageInfo_TransactionStep.Unmarshal(m, b)
+func (m *Transaction) XXX_Unmarshal(b []byte) error {
+ return xxx_messageInfo_Transaction.Unmarshal(m, b)
}
-func (m *TransactionStep) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
- return xxx_messageInfo_TransactionStep.Marshal(b, m, deterministic)
+func (m *Transaction) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+ return xxx_messageInfo_Transaction.Marshal(b, m, deterministic)
}
-func (m *TransactionStep) XXX_Merge(src proto.Message) {
- xxx_messageInfo_TransactionStep.Merge(m, src)
+func (m *Transaction) XXX_Merge(src proto.Message) {
+ xxx_messageInfo_Transaction.Merge(m, src)
}
-func (m *TransactionStep) XXX_Size() int {
- return xxx_messageInfo_TransactionStep.Size(m)
+func (m *Transaction) XXX_Size() int {
+ return xxx_messageInfo_Transaction.Size(m)
}
-func (m *TransactionStep) XXX_DiscardUnknown() {
- xxx_messageInfo_TransactionStep.DiscardUnknown(m)
+func (m *Transaction) XXX_DiscardUnknown() {
+ xxx_messageInfo_Transaction.DiscardUnknown(m)
}
-var xxx_messageInfo_TransactionStep proto.InternalMessageInfo
+var xxx_messageInfo_Transaction proto.InternalMessageInfo
-func (m *TransactionStep) GetId() string {
- if m != nil {
- return m.Id
- }
- return ""
-}
-
-func (m *TransactionStep) GetStep() TransactionStep_TransactionStep {
+func (m *Transaction) GetStep() Transaction_TransactionStep {
if m != nil {
return m.Step
}
- return TransactionStep_PRECOMMIT
+ return Transaction_PRECOMMIT
}
type Repository struct {
@@ -848,9 +840,9 @@ func init() {
proto.RegisterEnum("gitaly.SignatureType", SignatureType_name, SignatureType_value)
proto.RegisterEnum("gitaly.OperationMsg_Operation", OperationMsg_Operation_name, OperationMsg_Operation_value)
proto.RegisterEnum("gitaly.OperationMsg_Scope", OperationMsg_Scope_name, OperationMsg_Scope_value)
- proto.RegisterEnum("gitaly.TransactionStep_TransactionStep", TransactionStep_TransactionStep_name, TransactionStep_TransactionStep_value)
+ proto.RegisterEnum("gitaly.Transaction_TransactionStep", Transaction_TransactionStep_name, Transaction_TransactionStep_value)
proto.RegisterType((*OperationMsg)(nil), "gitaly.OperationMsg")
- proto.RegisterType((*TransactionStep)(nil), "gitaly.TransactionStep")
+ proto.RegisterType((*Transaction)(nil), "gitaly.Transaction")
proto.RegisterType((*Repository)(nil), "gitaly.Repository")
proto.RegisterType((*GitCommit)(nil), "gitaly.GitCommit")
proto.RegisterType((*CommitAuthor)(nil), "gitaly.CommitAuthor")
@@ -869,74 +861,74 @@ func init() {
func init() { proto.RegisterFile("shared.proto", fileDescriptor_d8a4e87e678c5ced) }
var fileDescriptor_d8a4e87e678c5ced = []byte{
- // 1096 bytes of a gzipped FileDescriptorProto
- 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x94, 0x56, 0x4d, 0x6f, 0xdb, 0x46,
- 0x13, 0x8e, 0x28, 0xea, 0x6b, 0x24, 0x39, 0xcc, 0xbe, 0x0e, 0x20, 0xf8, 0x45, 0x12, 0x97, 0x05,
- 0xda, 0x20, 0x70, 0x65, 0xc3, 0x41, 0x8b, 0x46, 0x29, 0x50, 0x48, 0xae, 0x62, 0x38, 0xb1, 0x45,
- 0x61, 0x45, 0xf7, 0xeb, 0x42, 0xac, 0xc4, 0xcd, 0x6a, 0x0b, 0x4a, 0x4b, 0x90, 0xab, 0xa0, 0xf6,
- 0xb1, 0xe8, 0xa9, 0xa7, 0x9e, 0x7b, 0xef, 0xb5, 0x3f, 0xa3, 0xe8, 0xdf, 0x68, 0xda, 0xff, 0xd1,
- 0x62, 0x77, 0x49, 0x99, 0xfe, 0x68, 0x9b, 0xdc, 0x76, 0x86, 0xcf, 0x3c, 0x33, 0xf3, 0xec, 0xec,
- 0x48, 0xd0, 0x4a, 0xe7, 0x24, 0xa1, 0x61, 0x37, 0x4e, 0x84, 0x14, 0xa8, 0xca, 0xb8, 0x24, 0xd1,
- 0xd9, 0xd6, 0x03, 0x26, 0x04, 0x8b, 0xe8, 0xae, 0xf6, 0x4e, 0x57, 0x2f, 0x77, 0x25, 0x5f, 0xd0,
- 0x54, 0x92, 0x45, 0x6c, 0x80, 0x5b, 0xdb, 0x57, 0x01, 0x21, 0x4d, 0x67, 0x09, 0x8f, 0xa5, 0x48,
- 0x0c, 0xc2, 0x7d, 0x5d, 0x82, 0x96, 0x17, 0xd3, 0x84, 0x48, 0x2e, 0x96, 0x27, 0x29, 0x43, 0x5d,
- 0xb0, 0x44, 0xdc, 0x29, 0x6d, 0x97, 0x1e, 0x6e, 0xec, 0xdf, 0xef, 0x9a, 0x44, 0xdd, 0x22, 0xe2,
- 0xc2, 0xc0, 0x96, 0x88, 0xd1, 0x53, 0x68, 0xa6, 0x33, 0x11, 0xd3, 0x20, 0xa2, 0xaf, 0x68, 0xd4,
- 0xb1, 0x74, 0xe0, 0xd6, 0x8d, 0x81, 0x13, 0x85, 0xc3, 0xa0, 0xe1, 0xc7, 0x0a, 0xed, 0x3e, 0x86,
- 0xc6, 0x1a, 0x81, 0x9a, 0x50, 0x3b, 0x1d, 0xbd, 0x18, 0x79, 0x5f, 0x8c, 0x9c, 0x5b, 0xca, 0x38,
- 0x39, 0xf5, 0xfb, 0xbe, 0x87, 0x9d, 0x12, 0x6a, 0x41, 0xbd, 0x7f, 0x70, 0x30, 0x9c, 0x4c, 0x3c,
- 0xec, 0x58, 0xee, 0x1e, 0x54, 0x34, 0x13, 0xda, 0x00, 0xc0, 0xc3, 0xb1, 0x37, 0x39, 0xf2, 0x3d,
- 0xfc, 0x95, 0x73, 0x0b, 0x01, 0x54, 0x27, 0x43, 0xfc, 0xf9, 0x50, 0x85, 0x34, 0xa1, 0x36, 0xf1,
- 0x3d, 0xdc, 0x3f, 0x1c, 0x3a, 0x96, 0xfb, 0x53, 0x09, 0x6e, 0xfb, 0x09, 0x59, 0xa6, 0x64, 0xa6,
- 0x32, 0x4d, 0x24, 0x8d, 0xd1, 0x06, 0x58, 0x3c, 0xd4, 0x7d, 0x36, 0xb0, 0xc5, 0x43, 0xf4, 0x14,
- 0xec, 0x54, 0xd2, 0x38, 0x6b, 0xe0, 0xfd, 0xbc, 0x81, 0x2b, 0x61, 0x57, 0x6d, 0xac, 0x83, 0xdc,
- 0xde, 0x75, 0xfe, 0x36, 0x34, 0xc6, 0x78, 0x78, 0xe0, 0x9d, 0x9c, 0x1c, 0xf9, 0xa6, 0xb6, 0xec,
- 0xac, 0xdb, 0xc1, 0xde, 0xf1, 0xf1, 0xa0, 0x7f, 0xf0, 0xc2, 0xb1, 0xdc, 0x5f, 0x2c, 0x00, 0x4c,
- 0x63, 0x91, 0x72, 0x29, 0x92, 0x33, 0xf4, 0x0e, 0xb4, 0x52, 0x29, 0x12, 0xc2, 0x68, 0xb0, 0x24,
- 0x0b, 0xaa, 0xeb, 0x69, 0xe0, 0x66, 0xe6, 0x1b, 0x91, 0x05, 0x45, 0xef, 0x42, 0x3b, 0xa1, 0x11,
- 0x91, 0xfc, 0x15, 0x0d, 0x62, 0x22, 0xe7, 0x9d, 0xb2, 0xc6, 0xb4, 0x72, 0xe7, 0x98, 0xc8, 0x39,
- 0xda, 0x83, 0x4d, 0xc6, 0x65, 0x20, 0xa6, 0xdf, 0xd0, 0x99, 0x0c, 0x42, 0x9e, 0xd0, 0x99, 0xe2,
- 0xef, 0xd8, 0x1a, 0x8b, 0x18, 0x97, 0x9e, 0xfe, 0xf4, 0x59, 0xfe, 0x05, 0x1d, 0xc2, 0xb6, 0x8a,
- 0x20, 0x91, 0xa4, 0xc9, 0x92, 0x48, 0x7a, 0x35, 0x96, 0xd3, 0xb4, 0x53, 0xd9, 0x2e, 0x3f, 0x6c,
- 0xe0, 0x7b, 0x8c, 0xcb, 0x7e, 0x0e, 0xbb, 0x4c, 0xc3, 0x69, 0xaa, 0xea, 0x63, 0x51, 0x90, 0xac,
- 0x7b, 0xea, 0x54, 0x4d, 0x7d, 0x2c, 0x2a, 0xf4, 0xf9, 0x1e, 0xdc, 0x66, 0x51, 0x10, 0x27, 0x42,
- 0xe7, 0xd0, 0x6d, 0xd4, 0x35, 0xac, 0xcd, 0xa2, 0xb1, 0xf1, 0xaa, 0x3e, 0x9e, 0xdb, 0xf5, 0x92,
- 0x63, 0x3d, 0xb7, 0xeb, 0x35, 0xa7, 0x8e, 0x6d, 0x05, 0x73, 0x7f, 0xb6, 0xa0, 0x71, 0xc8, 0xe5,
- 0x81, 0x58, 0x2c, 0xb8, 0xbc, 0x76, 0x8f, 0x1d, 0xa8, 0xa5, 0x2b, 0x5d, 0x92, 0x96, 0xae, 0x85,
- 0x73, 0x13, 0x21, 0xb0, 0xa7, 0x22, 0x3c, 0xd3, 0x6a, 0xb5, 0xb0, 0x3e, 0xa3, 0x1d, 0xa8, 0x92,
- 0x95, 0x9c, 0x8b, 0x44, 0xeb, 0xd2, 0xdc, 0xdf, 0xcc, 0xef, 0xdd, 0xb0, 0xf7, 0xf5, 0x37, 0x9c,
- 0x61, 0xd0, 0x3e, 0x34, 0x66, 0xda, 0x2f, 0x69, 0xd2, 0xa9, 0xfc, 0x4b, 0xc0, 0x05, 0x0c, 0xdd,
- 0x03, 0x88, 0x49, 0x42, 0x97, 0x32, 0xe0, 0x61, 0xda, 0xa9, 0x6a, 0xfd, 0x1a, 0xc6, 0x73, 0x14,
- 0xa6, 0xe8, 0xff, 0xd0, 0x50, 0x85, 0x04, 0x29, 0x3f, 0xa7, 0x9d, 0xda, 0x76, 0xe9, 0x61, 0x19,
- 0xd7, 0x95, 0x63, 0xc2, 0xcf, 0x29, 0xfa, 0x04, 0x36, 0x52, 0xce, 0x96, 0x44, 0xae, 0x12, 0x1a,
- 0xc8, 0xb3, 0x98, 0x6a, 0x89, 0x36, 0xf6, 0xef, 0xe6, 0x49, 0x27, 0xf9, 0x57, 0xff, 0x2c, 0xa6,
- 0xb8, 0x9d, 0x16, 0x4d, 0xf7, 0xfb, 0x12, 0xb4, 0x8a, 0x55, 0x29, 0x01, 0xf4, 0x48, 0x95, 0x8c,
- 0x00, 0xea, 0x8c, 0x36, 0xa1, 0x42, 0x17, 0x84, 0x47, 0x99, 0x58, 0xc6, 0x40, 0x5d, 0xb0, 0x43,
- 0x22, 0xa9, 0x96, 0xaa, 0xa9, 0x5e, 0xb3, 0x5e, 0x23, 0xdd, 0x7c, 0x8d, 0x74, 0xfd, 0x7c, 0xcf,
- 0x60, 0x8d, 0x43, 0x5b, 0x50, 0x57, 0xab, 0xe7, 0x5c, 0x2c, 0xa9, 0x16, 0xb2, 0x85, 0xd7, 0xb6,
- 0xeb, 0x02, 0x0c, 0xbf, 0xe5, 0x72, 0x22, 0x89, 0x5c, 0xa5, 0x2a, 0xdf, 0x2b, 0x12, 0xad, 0x4c,
- 0x11, 0x15, 0x6c, 0x0c, 0xd7, 0x87, 0xea, 0x20, 0x21, 0xcb, 0xd9, 0xfc, 0xc6, 0x1a, 0x3f, 0x82,
- 0xb6, 0x24, 0x09, 0xa3, 0x32, 0x30, 0xb2, 0xea, 0x5a, 0x9b, 0xfb, 0x77, 0x72, 0x15, 0xd6, 0xc3,
- 0x80, 0x5b, 0x06, 0x67, 0x2c, 0xf7, 0x07, 0x0b, 0xca, 0x3e, 0x61, 0x37, 0x72, 0x9a, 0xb1, 0xb1,
- 0xd6, 0x63, 0x73, 0x2d, 0x47, 0xf9, 0x8d, 0x72, 0xa8, 0x71, 0x5b, 0xd0, 0x34, 0x25, 0x2c, 0x6f,
- 0x3c, 0x37, 0xd5, 0x43, 0xce, 0x8e, 0xe6, 0x72, 0x2b, 0xfa, 0x72, 0x9b, 0x99, 0x4f, 0xdf, 0xef,
- 0x0e, 0x54, 0x25, 0x61, 0x8c, 0x26, 0xfa, 0x85, 0xfc, 0xe3, 0xf4, 0x19, 0xcc, 0x0d, 0xd3, 0x50,
- 0x7b, 0x8b, 0x69, 0x78, 0x09, 0xf6, 0x69, 0x4a, 0x13, 0xf4, 0x3f, 0xa8, 0xb0, 0x28, 0x58, 0x3f,
- 0x19, 0x9b, 0x45, 0x47, 0xe1, 0x5a, 0x21, 0xeb, 0xa6, 0xc9, 0x28, 0x17, 0x27, 0xe3, 0x01, 0x34,
- 0x59, 0x14, 0xac, 0x52, 0xf5, 0xf6, 0x17, 0x34, 0xdb, 0x26, 0xc0, 0xa2, 0xd3, 0xcc, 0xe3, 0x3e,
- 0x03, 0x30, 0x1b, 0x61, 0x2c, 0x44, 0x84, 0x3e, 0x06, 0x28, 0xec, 0x81, 0x92, 0xee, 0x12, 0xe5,
- 0xf5, 0x5e, 0x6c, 0x83, 0x81, 0xfd, 0xe3, 0xaf, 0x3b, 0x25, 0x5c, 0xc0, 0x3e, 0x1a, 0xe4, 0x3c,
- 0xaa, 0xfa, 0xcb, 0xbf, 0x0d, 0xc5, 0x5d, 0x5a, 0x07, 0x7b, 0x70, 0xec, 0x0d, 0x1c, 0x4b, 0x9d,
- 0x7c, 0x3c, 0x1c, 0x3a, 0x65, 0x54, 0x83, 0xb2, 0xdf, 0x3f, 0x74, 0xec, 0x47, 0x3b, 0xd0, 0xbe,
- 0xa4, 0x89, 0xc2, 0x8c, 0xbc, 0xd1, 0xd0, 0xb9, 0xa5, 0x30, 0xe3, 0xc3, 0xb1, 0x21, 0xf8, 0xf2,
- 0xc3, 0xbd, 0x27, 0x8e, 0xd5, 0xf3, 0xa0, 0x26, 0x62, 0x2d, 0x2c, 0xba, 0x7f, 0x6d, 0xe2, 0x4f,
- 0xa8, 0x9c, 0x8b, 0xd0, 0x8b, 0xd5, 0x7e, 0x4f, 0x3b, 0x7f, 0x7d, 0x77, 0xe5, 0xf5, 0x17, 0x7f,
- 0xe7, 0x70, 0x55, 0xc4, 0x2a, 0x5b, 0xef, 0x09, 0xd4, 0xb2, 0xb5, 0x8d, 0xee, 0x5d, 0x23, 0x7c,
- 0xc6, 0x69, 0xb4, 0xe6, 0xfb, 0xfd, 0x37, 0xc5, 0x57, 0xc7, 0x39, 0xbe, 0xf7, 0x69, 0x51, 0xb7,
- 0xff, 0x8a, 0x7e, 0x9d, 0x45, 0x17, 0x42, 0x7a, 0xc7, 0x70, 0x27, 0x9b, 0xe7, 0x37, 0xe7, 0xf9,
- 0x23, 0xe3, 0x71, 0x4c, 0xe4, 0xc5, 0xf5, 0xf4, 0x7c, 0xb8, 0x4b, 0xc2, 0x90, 0x2b, 0x18, 0x89,
- 0xde, 0x82, 0xf1, 0xcf, 0x8c, 0x71, 0xf3, 0x22, 0xba, 0x70, 0xe9, 0x7b, 0x5f, 0x2b, 0xf9, 0x22,
- 0x32, 0xed, 0xce, 0xc4, 0x62, 0xd7, 0x1c, 0x3f, 0x10, 0x09, 0xdb, 0x35, 0xa2, 0x9a, 0x7f, 0x2d,
- 0xbb, 0x4c, 0x64, 0x76, 0x3c, 0x9d, 0x56, 0xb5, 0xeb, 0xf1, 0xdf, 0x01, 0x00, 0x00, 0xff, 0xff,
- 0xe9, 0xa7, 0x3b, 0x4d, 0x0f, 0x09, 0x00, 0x00,
+ // 1093 bytes of a gzipped FileDescriptorProto
+ 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x94, 0x56, 0x5d, 0x6f, 0xdb, 0x36,
+ 0x17, 0xae, 0x65, 0xf9, 0xeb, 0xd8, 0x4e, 0x55, 0xbe, 0x29, 0x60, 0xe4, 0x45, 0xdb, 0x4c, 0x05,
+ 0x86, 0xa2, 0xc8, 0x9c, 0x20, 0xc5, 0x3e, 0xea, 0x0d, 0x18, 0xec, 0xcc, 0x0d, 0xd2, 0x26, 0x96,
+ 0x41, 0x2b, 0xfb, 0xba, 0x11, 0x68, 0x8b, 0xa5, 0x39, 0xc8, 0xa6, 0x20, 0xd1, 0xc5, 0x92, 0xcb,
+ 0x62, 0x57, 0xbb, 0xda, 0x9f, 0xd8, 0xed, 0x7e, 0xc6, 0xb0, 0xbf, 0xb1, 0x6e, 0xff, 0x63, 0x03,
+ 0x49, 0xc9, 0x51, 0x3e, 0xb6, 0xb5, 0x77, 0x3c, 0x87, 0xcf, 0x79, 0xce, 0xe1, 0xc3, 0xc3, 0x23,
+ 0x41, 0x2b, 0x9d, 0x93, 0x84, 0x86, 0xdd, 0x38, 0x11, 0x52, 0xa0, 0x2a, 0xe3, 0x92, 0x44, 0x67,
+ 0x5b, 0x0f, 0x98, 0x10, 0x2c, 0xa2, 0xbb, 0xda, 0x3b, 0x5d, 0xbd, 0xdc, 0x95, 0x7c, 0x41, 0x53,
+ 0x49, 0x16, 0xb1, 0x01, 0x6e, 0x6d, 0x5f, 0x05, 0x84, 0x34, 0x9d, 0x25, 0x3c, 0x96, 0x22, 0x31,
+ 0x08, 0xf7, 0x4d, 0x09, 0x5a, 0x5e, 0x4c, 0x13, 0x22, 0xb9, 0x58, 0x9e, 0xa4, 0x0c, 0x75, 0xc1,
+ 0x12, 0x71, 0xa7, 0xb4, 0x5d, 0x7a, 0xb4, 0xb1, 0x7f, 0xbf, 0x6b, 0x12, 0x75, 0x8b, 0x88, 0x0b,
+ 0x03, 0x5b, 0x22, 0x46, 0x9f, 0x42, 0x33, 0x9d, 0x89, 0x98, 0x06, 0x11, 0x7d, 0x45, 0xa3, 0x8e,
+ 0xa5, 0x03, 0xb7, 0x6e, 0x0c, 0x9c, 0x28, 0x1c, 0x06, 0x0d, 0x3f, 0x56, 0x68, 0xf7, 0x09, 0x34,
+ 0xd6, 0x08, 0xd4, 0x84, 0xda, 0xe9, 0xe8, 0xc5, 0xc8, 0xfb, 0x6a, 0xe4, 0xdc, 0x52, 0xc6, 0xc9,
+ 0xa9, 0xdf, 0xf7, 0x3d, 0xec, 0x94, 0x50, 0x0b, 0xea, 0xfd, 0x83, 0x83, 0xe1, 0x64, 0xe2, 0x61,
+ 0xc7, 0x72, 0xf7, 0xa0, 0xa2, 0x99, 0xd0, 0x06, 0x00, 0x1e, 0x8e, 0xbd, 0xc9, 0x91, 0xef, 0xe1,
+ 0x6f, 0x9c, 0x5b, 0x08, 0xa0, 0x3a, 0x19, 0xe2, 0x2f, 0x87, 0x2a, 0xa4, 0x09, 0xb5, 0x89, 0xef,
+ 0xe1, 0xfe, 0xe1, 0xd0, 0xb1, 0xdc, 0xd7, 0x25, 0x68, 0xfa, 0x09, 0x59, 0xa6, 0x64, 0xa6, 0x33,
+ 0x7d, 0x0c, 0x76, 0x2a, 0x69, 0x7e, 0xca, 0x87, 0x79, 0xb1, 0x05, 0x48, 0x71, 0x3d, 0x91, 0x34,
+ 0xc6, 0x3a, 0xc0, 0xed, 0xc1, 0xed, 0x2b, 0x1b, 0xa8, 0x0d, 0x8d, 0x31, 0x1e, 0x1e, 0x78, 0x27,
+ 0x27, 0x47, 0xbe, 0xa9, 0x21, 0x5b, 0xeb, 0xb2, 0xb1, 0x77, 0x7c, 0x3c, 0xe8, 0x1f, 0xbc, 0x70,
+ 0x2c, 0xf7, 0x17, 0x0b, 0x00, 0xd3, 0x58, 0xa4, 0x5c, 0x8a, 0xe4, 0x0c, 0xbd, 0x07, 0xad, 0x54,
+ 0x8a, 0x84, 0x30, 0x1a, 0x2c, 0xc9, 0x82, 0x6a, 0xe1, 0x1a, 0xb8, 0x99, 0xf9, 0x46, 0x64, 0x41,
+ 0xd1, 0x43, 0x68, 0x27, 0x34, 0x22, 0x92, 0xbf, 0xa2, 0x41, 0x4c, 0xe4, 0xbc, 0x53, 0xd6, 0x98,
+ 0x56, 0xee, 0x1c, 0x13, 0x39, 0x47, 0x7b, 0xb0, 0xc9, 0xb8, 0x0c, 0xc4, 0xf4, 0x3b, 0x3a, 0x93,
+ 0x41, 0xc8, 0x13, 0x3a, 0x53, 0xfc, 0x1d, 0x5b, 0x63, 0x11, 0xe3, 0xd2, 0xd3, 0x5b, 0x5f, 0xe4,
+ 0x3b, 0xe8, 0x10, 0xb6, 0x55, 0x04, 0x89, 0x24, 0x4d, 0x96, 0x44, 0xd2, 0xab, 0xb1, 0x9c, 0xa6,
+ 0x9d, 0xca, 0x76, 0xf9, 0x51, 0x03, 0xdf, 0x63, 0x5c, 0xf6, 0x73, 0xd8, 0x65, 0x1a, 0x4e, 0x53,
+ 0x55, 0x1f, 0x8b, 0x82, 0x64, 0x7d, 0xa6, 0x4e, 0xd5, 0xd4, 0xc7, 0xa2, 0xc2, 0x39, 0xdf, 0x87,
+ 0xdb, 0x2c, 0x0a, 0xe2, 0x44, 0xe8, 0x1c, 0xfa, 0x18, 0x75, 0x0d, 0x6b, 0xb3, 0x68, 0x6c, 0xbc,
+ 0xea, 0x1c, 0xcf, 0xed, 0x7a, 0xc9, 0xb1, 0x9e, 0xdb, 0xf5, 0x9a, 0x53, 0xc7, 0xb6, 0x82, 0xb9,
+ 0x3f, 0x5b, 0xd0, 0x38, 0xe4, 0xf2, 0x40, 0x2c, 0x16, 0x5c, 0xa2, 0x0d, 0xb0, 0x78, 0xa8, 0x6f,
+ 0xac, 0x81, 0x2d, 0x1e, 0xa2, 0x0e, 0xd4, 0xd2, 0x95, 0x2e, 0x49, 0x4b, 0xd7, 0xc2, 0xb9, 0x89,
+ 0x10, 0xd8, 0x53, 0x11, 0x9e, 0x69, 0xb5, 0x5a, 0x58, 0xaf, 0xd1, 0x0e, 0x54, 0xc9, 0x4a, 0xce,
+ 0x45, 0xa2, 0x75, 0x69, 0xee, 0x6f, 0xe6, 0x77, 0x6e, 0xd8, 0xfb, 0x7a, 0x0f, 0x67, 0x18, 0xb4,
+ 0x0f, 0x8d, 0x99, 0xf6, 0x4b, 0x9a, 0x74, 0x2a, 0xff, 0x12, 0x70, 0x01, 0x43, 0xf7, 0x00, 0x62,
+ 0x92, 0xd0, 0xa5, 0x0c, 0x78, 0x98, 0x76, 0xaa, 0x5a, 0xbf, 0x86, 0xf1, 0x1c, 0x85, 0x29, 0xfa,
+ 0x3f, 0x34, 0x54, 0x21, 0x41, 0xca, 0xcf, 0x69, 0xa7, 0xb6, 0x5d, 0x7a, 0x54, 0xc6, 0x75, 0xe5,
+ 0x98, 0xf0, 0x73, 0x8a, 0x3e, 0x83, 0x8d, 0x94, 0xb3, 0x25, 0x91, 0xab, 0x84, 0x06, 0xf2, 0x2c,
+ 0xa6, 0x5a, 0xa2, 0x8d, 0xfd, 0xbb, 0x79, 0xd2, 0x49, 0xbe, 0xeb, 0x9f, 0xc5, 0x14, 0xb7, 0xd3,
+ 0xa2, 0xe9, 0xfe, 0x50, 0x82, 0x56, 0xb1, 0x2a, 0x25, 0x80, 0x6e, 0xa9, 0x92, 0x11, 0x40, 0xad,
+ 0xd1, 0x26, 0x54, 0xe8, 0x82, 0xf0, 0x28, 0x13, 0xcb, 0x18, 0xa8, 0x0b, 0x76, 0x48, 0x24, 0xd5,
+ 0x52, 0x35, 0xd5, 0xab, 0xd5, 0xe3, 0xa2, 0x9b, 0x8f, 0x8b, 0xae, 0x9f, 0xcf, 0x13, 0xac, 0x71,
+ 0x68, 0x0b, 0xea, 0x6a, 0xc4, 0x9c, 0x8b, 0x25, 0xd5, 0x42, 0xb6, 0xf0, 0xda, 0x76, 0x5d, 0x80,
+ 0xe1, 0xf7, 0x5c, 0x4e, 0x24, 0x91, 0xab, 0x54, 0xe5, 0x7b, 0x45, 0xa2, 0x95, 0x29, 0xa2, 0x82,
+ 0x8d, 0xe1, 0xfa, 0x50, 0x1d, 0x24, 0x64, 0x39, 0x9b, 0xdf, 0x58, 0xe3, 0x47, 0xd0, 0x96, 0x24,
+ 0x61, 0x54, 0x06, 0x46, 0x56, 0x5d, 0x6b, 0x73, 0xff, 0x4e, 0xae, 0xc2, 0xba, 0x19, 0x70, 0xcb,
+ 0xe0, 0x8c, 0xe5, 0xfe, 0x68, 0x41, 0xd9, 0x27, 0xec, 0x46, 0x4e, 0xd3, 0x36, 0xd6, 0xba, 0x6d,
+ 0xae, 0xe5, 0x28, 0xbf, 0x55, 0x0e, 0xd5, 0x6e, 0x0b, 0x9a, 0xa6, 0x84, 0xe5, 0x07, 0xcf, 0x4d,
+ 0xf5, 0x90, 0xb3, 0xa5, 0xb9, 0xdc, 0x8a, 0xbe, 0xdc, 0x66, 0xe6, 0xd3, 0xf7, 0xbb, 0x03, 0x55,
+ 0x49, 0x18, 0xa3, 0x89, 0x7e, 0x21, 0xff, 0xd8, 0x7d, 0x06, 0x73, 0x43, 0x37, 0xd4, 0xde, 0xa1,
+ 0x1b, 0x5e, 0x82, 0x7d, 0x9a, 0xd2, 0x04, 0xfd, 0x0f, 0x2a, 0x2c, 0x0a, 0xd6, 0x4f, 0xc6, 0x66,
+ 0xd1, 0x51, 0xb8, 0x56, 0xc8, 0xba, 0xa9, 0x33, 0xca, 0xc5, 0xce, 0x78, 0x00, 0x4d, 0x16, 0x05,
+ 0xab, 0x54, 0xbd, 0xfd, 0x05, 0xcd, 0xa6, 0x09, 0xb0, 0xe8, 0x34, 0xf3, 0xb8, 0xcf, 0x00, 0xcc,
+ 0x44, 0x18, 0x0b, 0x11, 0xa1, 0x4f, 0x00, 0x0a, 0x73, 0xa0, 0xa4, 0x4f, 0x89, 0xf2, 0x7a, 0x2f,
+ 0xa6, 0xc1, 0xc0, 0xfe, 0xe9, 0xd7, 0x9d, 0x12, 0x2e, 0x60, 0x1f, 0x0f, 0x72, 0x1e, 0x55, 0xfd,
+ 0xe5, 0x6f, 0x40, 0x71, 0x96, 0xd6, 0xc1, 0x1e, 0x1c, 0x7b, 0x03, 0xc7, 0x52, 0x2b, 0x1f, 0x0f,
+ 0x87, 0x4e, 0x19, 0xd5, 0xa0, 0xec, 0xf7, 0x0f, 0x1d, 0xfb, 0xf1, 0x0e, 0xb4, 0x2f, 0x69, 0xa2,
+ 0x30, 0x23, 0x6f, 0x34, 0x74, 0x6e, 0x29, 0xcc, 0xf8, 0x70, 0x6c, 0x08, 0xbe, 0xfe, 0x70, 0xef,
+ 0xa9, 0x63, 0xf5, 0x3c, 0xa8, 0x89, 0x58, 0x0b, 0x8b, 0xee, 0x5f, 0xeb, 0xf8, 0x13, 0x2a, 0xe7,
+ 0x22, 0xf4, 0x62, 0x35, 0xdf, 0xd3, 0xce, 0x5f, 0xaf, 0xaf, 0xbc, 0xfe, 0xe2, 0xf7, 0x0c, 0x57,
+ 0x45, 0xac, 0xb2, 0xf5, 0x9e, 0x42, 0x2d, 0x1b, 0xdb, 0xe8, 0xde, 0x35, 0xc2, 0x67, 0x9c, 0x46,
+ 0x6b, 0xbe, 0xdf, 0x7f, 0x53, 0x7c, 0x75, 0x9c, 0xe3, 0x7b, 0x9f, 0x17, 0x75, 0xfb, 0xaf, 0xe8,
+ 0x37, 0x59, 0x74, 0x21, 0xa4, 0x77, 0x0c, 0x77, 0xb2, 0x7e, 0x7e, 0x7b, 0x9e, 0x3f, 0x32, 0x1e,
+ 0xc7, 0x44, 0x5e, 0x5c, 0x4f, 0xcf, 0x87, 0xbb, 0x24, 0x0c, 0xb9, 0x82, 0x91, 0xe8, 0x1d, 0x18,
+ 0xff, 0xcc, 0x18, 0x37, 0x2f, 0xa2, 0x0b, 0x97, 0xbe, 0xf7, 0xad, 0x92, 0x2f, 0x22, 0xd3, 0xee,
+ 0x4c, 0x2c, 0x76, 0xcd, 0xf2, 0x03, 0x91, 0xb0, 0x5d, 0x23, 0xaa, 0xf9, 0x3b, 0xd9, 0x65, 0x22,
+ 0xb3, 0xe3, 0xe9, 0xb4, 0xaa, 0x5d, 0x4f, 0xfe, 0x0e, 0x00, 0x00, 0xff, 0xff, 0xee, 0xd3, 0x49,
+ 0x70, 0xf7, 0x08, 0x00, 0x00,
}
diff --git a/proto/go/internal/linter/method.go b/proto/go/internal/linter/method.go
index 4fce1d8ff..a93d4cd85 100644
--- a/proto/go/internal/linter/method.go
+++ b/proto/go/internal/linter/method.go
@@ -105,42 +105,33 @@ func (ml methodLinter) ensureValidStorage(expected int) error {
}
func (ml methodLinter) ensureValidTargetRepository(expected int) error {
- /*
- topLevelMsgs, err := ml.getTopLevelMsgs()
- if err != nil {
- return err
- }
-
- reqMsgName, err := lastName(ml.methodDesc.GetInputType())
- if err != nil {
- return err
- }
-
- msgT := topLevelMsgs[reqMsgName]
-
- m := matcher{
- match: internal.GetTargetRepositoryExtension,
- subMatch: internal.GetRepositoryExtension,
- expectedType: ".gitaly.Repository",
- topLevelMsgs: topLevelMsgs,
- }
+ topLevelMsgs, err := ml.getTopLevelMsgs()
+ if err != nil {
+ return err
+ }
- */
+ reqMsgName, err := lastName(ml.methodDesc.GetInputType())
+ if err != nil {
+ return err
+ }
- /*
- storageFields, err := m.findMatchingFields(reqMsgName, msgT)
- if err != nil {
- return err
- }
+ msgT := topLevelMsgs[reqMsgName]
- */
+ m := matcher{
+ match: internal.GetTargetRepositoryExtension,
+ subMatch: internal.GetRepositoryExtension,
+ expectedType: ".gitaly.Repository",
+ topLevelMsgs: topLevelMsgs,
+ }
- /*
- if len(storageFields) != expected {
- return fmt.Errorf("unexpected count of target_repository fields %d, expected %d, found target_repository label at: %v", len(storageFields), expected, storageFields)
- }
+ storageFields, err := m.findMatchingFields(reqMsgName, msgT)
+ if err != nil {
+ return err
+ }
- */
+ if len(storageFields) != expected {
+ return fmt.Errorf("unexpected count of target_repository fields %d, expected %d, found target_repository label at: %v", len(storageFields), expected, storageFields)
+ }
return nil
}
diff --git a/proto/repository-service.proto b/proto/repository-service.proto
index 67ce7605a..4462eea49 100644
--- a/proto/repository-service.proto
+++ b/proto/repository-service.proto
@@ -336,14 +336,14 @@ message WriteRefRequest {
// This used to be a boolean indicating whether or not to shell out or use
// the rugged implementation
reserved 6;
- TransactionStep transaction_step = 7;
+ Transaction transaction = 7;
}
message WriteRefResponse {
// This used to contain an error message. Since we're shelling out
// all exceptions are wrapped in GRPC errors.
reserved 1;
- TransactionStep transaction_step = 2;
+ Transaction transaction = 2;
}
message FindMergeBaseRequest {
diff --git a/proto/shared.proto b/proto/shared.proto
index 29afd1eaa..cd8447762 100644
--- a/proto/shared.proto
+++ b/proto/shared.proto
@@ -29,19 +29,16 @@ message OperationMsg {
Scope scope_level = 2;
}
-message TransactionStep {
+message Transaction {
enum TransactionStep {
PRECOMMIT = 0;
COMMIT = 1;
ROLLBACK = 2;
}
- string id = 1;
- TransactionStep step = 2;
+ TransactionStep step = 1;
}
-
-
enum ObjectType {
UNKNOWN = 0;
COMMIT = 1;
diff --git a/ruby/proto/gitaly/repository-service_pb.rb b/ruby/proto/gitaly/repository-service_pb.rb
index 00686bd90..9b1f7b069 100644
--- a/ruby/proto/gitaly/repository-service_pb.rb
+++ b/ruby/proto/gitaly/repository-service_pb.rb
@@ -106,10 +106,10 @@ Google::Protobuf::DescriptorPool.generated_pool.build do
optional :revision, :bytes, 3
optional :old_revision, :bytes, 4
optional :force, :bool, 5
- optional :transaction_step, :message, 7, "gitaly.TransactionStep"
+ optional :transaction, :message, 7, "gitaly.Transaction"
end
add_message "gitaly.WriteRefResponse" do
- optional :transaction_step, :message, 2, "gitaly.TransactionStep"
+ optional :transaction, :message, 2, "gitaly.Transaction"
end
add_message "gitaly.FindMergeBaseRequest" do
optional :repository, :message, 1, "gitaly.Repository"
diff --git a/ruby/proto/gitaly/shared_pb.rb b/ruby/proto/gitaly/shared_pb.rb
index ed42cba99..b70b4d900 100644
--- a/ruby/proto/gitaly/shared_pb.rb
+++ b/ruby/proto/gitaly/shared_pb.rb
@@ -19,11 +19,10 @@ Google::Protobuf::DescriptorPool.generated_pool.build do
value :SERVER, 1
value :STORAGE, 2
end
- add_message "gitaly.TransactionStep" do
- optional :id, :string, 1
- optional :step, :enum, 2, "gitaly.TransactionStep.TransactionStep"
+ add_message "gitaly.Transaction" do
+ optional :step, :enum, 1, "gitaly.Transaction.TransactionStep"
end
- add_enum "gitaly.TransactionStep.TransactionStep" do
+ add_enum "gitaly.Transaction.TransactionStep" do
value :PRECOMMIT, 0
value :COMMIT, 1
value :ROLLBACK, 2
@@ -95,8 +94,8 @@ module Gitaly
OperationMsg = Google::Protobuf::DescriptorPool.generated_pool.lookup("gitaly.OperationMsg").msgclass
OperationMsg::Operation = Google::Protobuf::DescriptorPool.generated_pool.lookup("gitaly.OperationMsg.Operation").enummodule
OperationMsg::Scope = Google::Protobuf::DescriptorPool.generated_pool.lookup("gitaly.OperationMsg.Scope").enummodule
- TransactionStep = Google::Protobuf::DescriptorPool.generated_pool.lookup("gitaly.TransactionStep").msgclass
- TransactionStep::TransactionStep = Google::Protobuf::DescriptorPool.generated_pool.lookup("gitaly.TransactionStep.TransactionStep").enummodule
+ Transaction = Google::Protobuf::DescriptorPool.generated_pool.lookup("gitaly.Transaction").msgclass
+ Transaction::TransactionStep = Google::Protobuf::DescriptorPool.generated_pool.lookup("gitaly.Transaction.TransactionStep").enummodule
Repository = Google::Protobuf::DescriptorPool.generated_pool.lookup("gitaly.Repository").msgclass
GitCommit = Google::Protobuf::DescriptorPool.generated_pool.lookup("gitaly.GitCommit").msgclass
CommitAuthor = Google::Protobuf::DescriptorPool.generated_pool.lookup("gitaly.CommitAuthor").msgclass