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:
authorKim Carlbäcker <kim.carlbacker@gmail.com>2018-04-11 13:50:33 +0300
committerKim Carlbäcker <kim.carlbacker@gmail.com>2018-04-11 13:50:33 +0300
commit4670e7da8549506752627357bb05ad62332274c9 (patch)
treeadc3173fe2187db0b025b03793513341d25ff18c
parent7192e387ebd44510cc110442bcc07d9b89d3af6c (diff)
parentb5db970a8d2d679444743d4f16bc99db0c141a5b (diff)
Merge branch '990-gitaly-raw-archive' into 'master'
Gitaly Snapshot RPCs Closes #990 See merge request gitlab-org/gitaly!644
-rw-r--r--internal/archive/match_walker.go23
-rw-r--r--internal/archive/tar_builder.go201
-rw-r--r--internal/archive/tar_entries.go27
-rw-r--r--internal/service/repository/create_from_snapshot.go115
-rw-r--r--internal/service/repository/create_from_snapshot_test.go226
-rw-r--r--internal/service/repository/snapshot.go71
-rw-r--r--internal/service/repository/snapshot_test.go113
-rw-r--r--internal/tempdir/tempdir.go27
-rw-r--r--internal/tempdir/tempdir_test.go14
-rw-r--r--vendor/gitlab.com/gitlab-org/gitaly-proto/go/VERSION2
-rw-r--r--vendor/gitlab.com/gitlab-org/gitaly-proto/go/blob.pb.go4
-rw-r--r--vendor/gitlab.com/gitlab-org/gitaly-proto/go/repository-service.pb.go382
-rw-r--r--vendor/vendor.json10
13 files changed, 1098 insertions, 117 deletions
diff --git a/internal/archive/match_walker.go b/internal/archive/match_walker.go
new file mode 100644
index 000000000..327e0a7f0
--- /dev/null
+++ b/internal/archive/match_walker.go
@@ -0,0 +1,23 @@
+package archive
+
+import (
+ "os"
+ "path/filepath"
+ "regexp"
+)
+
+// Walk a directory tree, only calling the wrapped WalkFunc if the path matches
+type matchWalker struct {
+ wrapped filepath.WalkFunc
+ patterns []*regexp.Regexp
+}
+
+func (m matchWalker) Walk(path string, info os.FileInfo, err error) error {
+ for _, pattern := range m.patterns {
+ if pattern.MatchString(path) {
+ return m.wrapped(path, info, err)
+ }
+ }
+
+ return nil
+}
diff --git a/internal/archive/tar_builder.go b/internal/archive/tar_builder.go
new file mode 100644
index 000000000..a8bd06e26
--- /dev/null
+++ b/internal/archive/tar_builder.go
@@ -0,0 +1,201 @@
+package archive
+
+import (
+ "archive/tar"
+ "fmt"
+ "io"
+ "os"
+ "path/filepath"
+ "regexp"
+ "strings"
+
+ "golang.org/x/sys/unix"
+)
+
+// TarBuilder writes a .tar archive to an io.Writer. The contents of the archive
+// are determined by successive calls to `File` and `RecursiveDir`.
+//
+// If an error occurs during processing, all subsequent calls to TarWriter will
+// fail with that same error. The same error will be returned by `Err()`.
+//
+// TarBuilder is **not** safe for concurrent use.
+type TarBuilder struct {
+ basePath string
+ tarWriter *tar.Writer
+
+ // The first error stops all further processing
+ err error
+}
+
+// NewTarBuilder creates a TarBuilder that writes files from basePath on the
+// filesystem to the given io.Writer
+func NewTarBuilder(basePath string, w io.Writer) *TarBuilder {
+ return &TarBuilder{
+ basePath: basePath,
+ tarWriter: tar.NewWriter(w),
+ }
+}
+
+func (t *TarBuilder) join(rel string) string {
+ return filepath.Join(t.basePath, rel)
+}
+
+func (t *TarBuilder) setErr(err error) error {
+ t.err = err
+ return err
+}
+
+func (t *TarBuilder) entry(fi os.FileInfo, filename string, r io.Reader) error {
+ if !fi.Mode().IsRegular() && !fi.Mode().IsDir() {
+ return fmt.Errorf("Unsupported mode for %v: %v", filename, fi.Mode())
+ }
+
+ hdr, err := tar.FileInfoHeader(fi, "")
+ if err != nil {
+ return err
+ }
+
+ if fi.IsDir() && !strings.HasSuffix(filename, "/") {
+ filename = filename + "/"
+ }
+
+ hdr.Name = filename
+
+ if err := t.tarWriter.WriteHeader(hdr); err != nil {
+ return err
+ }
+
+ if fi.Mode().IsRegular() {
+ // Size is included in the tar header, so ensure exactly that many bytes
+ // are written. This may lead to an inconsistent file with concurrent
+ // writes, but the archive itself will be well-formed. Archive creation
+ // will fail outright if the file is shortened.
+ if _, err := io.CopyN(t.tarWriter, r, fi.Size()); err != nil {
+ return err
+ }
+ }
+
+ return nil
+}
+
+func (t *TarBuilder) walk(path string, fi os.FileInfo, err error) error {
+ // Stop completely if an error is encountered walking the directory
+ if err != nil {
+ return err
+ }
+
+ // This condition strongly suggests an application bug
+ rel, err := filepath.Rel(t.basePath, path)
+ if err != nil {
+ return err
+ }
+
+ if fi.Mode().IsDir() {
+ return t.entry(fi, rel, nil)
+ }
+
+ // Ignore symlinks and special files in directories
+ if !fi.Mode().IsRegular() {
+ return nil
+ }
+
+ return t.File(rel, true)
+}
+
+// File writes a single regular file to the archive. It is an error if the file
+// exists, but is not a regular file - including symlinks.
+//
+// If `mustExist` is set, an error is returned if the file doesn't exist.
+// Otherwise, the error is hidden.
+func (t *TarBuilder) File(rel string, mustExist bool) error {
+ if t.err != nil {
+ return t.err
+ }
+
+ filename := t.join(rel)
+
+ // O_NOFOLLOW causes an error to be returned if the file is a symlink
+ file, err := os.OpenFile(filename, os.O_RDONLY|unix.O_NOFOLLOW, 0)
+ if err != nil {
+ // The file doesn't exist, but we've been told that's OK
+ if os.IsNotExist(err) && !mustExist {
+ return nil
+ }
+
+ // Halt in any other circumstance
+ return t.setErr(err)
+ }
+
+ defer file.Close()
+
+ fi, err := file.Stat()
+ if err != nil {
+ return t.setErr(err)
+ }
+
+ return t.setErr(t.entry(fi, rel, file))
+}
+
+// RecursiveDir adds a complete directory to the archive, including all
+// subdirectories and any regular files in the tree. Anything that is not a
+// regular file (including symlinks, etc) will be **skipped**.
+//
+// If `mustExist` is true, an error is returned if the root directory doesn't
+// exist. Otherwise, the error is hidden.
+//
+// If patterns is non-empty, only those matching files and directories will be
+// included. Otherwise, all are included.
+func (t *TarBuilder) RecursiveDir(rel string, mustExist bool, patterns ...*regexp.Regexp) error {
+ if t.err != nil {
+ return t.err
+ }
+
+ root := t.join(rel)
+
+ if _, err := os.Lstat(root); err != nil {
+ if os.IsNotExist(err) && !mustExist {
+ return nil
+ }
+
+ return t.setErr(err)
+ }
+
+ walker := t.walk
+ if len(patterns) > 0 {
+ walker = matchWalker{wrapped: t.walk, patterns: patterns}.Walk
+ }
+
+ // Walk the root and its children, recursively
+ return t.setErr(filepath.Walk(root, walker))
+}
+
+// FileIfExist is a helper for File that sets `mustExist` to false.
+func (t *TarBuilder) FileIfExist(rel string) error {
+ return t.File(rel, false)
+}
+
+// RecursiveDirIfExist is a helper for RecursiveDir that sets `mustExist` to
+// false.
+func (t *TarBuilder) RecursiveDirIfExist(rel string, patterns ...*regexp.Regexp) error {
+ return t.RecursiveDir(rel, false, patterns...)
+}
+
+// Close finalizes the archive and releases any underlying resources. It should
+// always be called, whether an error has been encountered in processing or not.
+func (t *TarBuilder) Close() error {
+ if t.err != nil {
+ // Ignore any close error in favour of reporting the previous one, but
+ // ensure the tar writer is closed to avoid resource leaks
+ t.tarWriter.Close()
+ return t.err
+ }
+
+ return t.tarWriter.Close()
+}
+
+// Err returns the last error seen during operation of a TarBuilder. Once an
+// error has been encountered, the TarBuilder will cease further operations. It
+// is safe to make a series of calls, then just check `Err()` at the end.
+func (t *TarBuilder) Err() error {
+ return t.err
+}
diff --git a/internal/archive/tar_entries.go b/internal/archive/tar_entries.go
new file mode 100644
index 000000000..f43ee4080
--- /dev/null
+++ b/internal/archive/tar_entries.go
@@ -0,0 +1,27 @@
+package archive
+
+import (
+ "archive/tar"
+ "io"
+)
+
+// TarEntries interprets the given io.Reader as a tar archive, outputting a list
+// of filenames contained within it
+func TarEntries(r io.Reader) ([]string, error) {
+ entries := []string{}
+ tr := tar.NewReader(r)
+
+ for {
+ hdr, err := tr.Next()
+ if err == io.EOF {
+ break
+ }
+ if err != nil {
+ return nil, err
+ }
+
+ entries = append(entries, hdr.Name)
+ }
+
+ return entries, nil
+}
diff --git a/internal/service/repository/create_from_snapshot.go b/internal/service/repository/create_from_snapshot.go
new file mode 100644
index 000000000..66055123b
--- /dev/null
+++ b/internal/service/repository/create_from_snapshot.go
@@ -0,0 +1,115 @@
+package repository
+
+import (
+ "net"
+ "net/http"
+ "os"
+ "os/exec"
+ "time"
+
+ pb "gitlab.com/gitlab-org/gitaly-proto/go"
+ "golang.org/x/net/context"
+ "google.golang.org/grpc/codes"
+ "google.golang.org/grpc/status"
+
+ "gitlab.com/gitlab-org/gitaly/internal/command"
+ "gitlab.com/gitlab-org/gitaly/internal/helper"
+ "gitlab.com/gitlab-org/gitaly/internal/tempdir"
+)
+
+// httpTransport defines a http.Transport with values that are more restrictive
+// than for http.DefaultTransport.
+//
+// They define shorter TLS Handshake, and more agressive connection closing
+// to prevent the connection hanging and reduce FD usage.
+var httpTransport = &http.Transport{
+ Proxy: http.ProxyFromEnvironment,
+ DialContext: (&net.Dialer{
+ Timeout: 30 * time.Second,
+ KeepAlive: 10 * time.Second,
+ }).DialContext,
+ MaxIdleConns: 2,
+ IdleConnTimeout: 30 * time.Second,
+ TLSHandshakeTimeout: 10 * time.Second,
+ ExpectContinueTimeout: 10 * time.Second,
+ ResponseHeaderTimeout: 30 * time.Second,
+}
+
+// httpClient defines a http.Client that uses the specialized httpTransport
+// (above). It also disables following redirects, as we don't expect this to be
+// required for this RPC.
+var httpClient = &http.Client{
+ Transport: httpTransport,
+ CheckRedirect: func(*http.Request, []*http.Request) error {
+ return http.ErrUseLastResponse
+ },
+}
+
+func untar(ctx context.Context, path string, in *pb.CreateRepositoryFromSnapshotRequest) error {
+ req, err := http.NewRequest("GET", in.HttpUrl, nil)
+ if err != nil {
+ return status.Errorf(codes.InvalidArgument, "Bad HTTP URL: %v", err)
+ }
+
+ if in.HttpAuth != "" {
+ req.Header.Set("Authorization", in.HttpAuth)
+ }
+
+ rsp, err := httpClient.Do(req)
+ if err != nil {
+ return status.Errorf(codes.Internal, "HTTP request failed: %v", err)
+ }
+ defer rsp.Body.Close()
+
+ if rsp.StatusCode < http.StatusOK || rsp.StatusCode >= http.StatusMultipleChoices {
+ return status.Errorf(codes.Internal, "HTTP server: %v", rsp.Status)
+ }
+
+ cmd, err := command.New(ctx, exec.Command("tar", "-C", path, "-xvf", "-"), rsp.Body, nil, nil)
+ if err != nil {
+ return err
+ }
+
+ return cmd.Wait()
+}
+
+func (s *server) CreateRepositoryFromSnapshot(ctx context.Context, in *pb.CreateRepositoryFromSnapshotRequest) (*pb.CreateRepositoryFromSnapshotResponse, error) {
+ realPath, err := helper.GetPath(in.Repository)
+ if err != nil {
+ return nil, err
+ }
+
+ if _, err := os.Stat(realPath); !os.IsNotExist(err) {
+ return nil, status.Errorf(codes.InvalidArgument, "destination directory exists")
+ }
+
+ // Perform all operations against a temporary directory, only moving it to
+ // the canonical location if retrieving and unpacking the snapshot is a
+ // success
+ tempRepo, tempPath, err := tempdir.NewAsRepository(ctx, in.Repository)
+ if err != nil {
+ return nil, status.Errorf(codes.Internal, "couldn't create temporary directory: %v", err)
+ }
+
+ // The archive contains a partial git repository, missing a config file and
+ // other important items. Initializing a new bare one and extracting the
+ // archive on top of it ensures the created git repository has everything
+ // it needs (especially, the config file and hooks directory).
+ //
+ // NOTE: The received archive is trusted *a lot*. Before pointing this RPC
+ // at endpoints not under our control, it should undergo a lot of hardning.
+ crr := &pb.CreateRepositoryRequest{Repository: tempRepo}
+ if _, err := s.CreateRepository(ctx, crr); err != nil {
+ return nil, status.Errorf(codes.Internal, "couldn't create empty bare repository: %v", err)
+ }
+
+ if err := untar(ctx, tempPath, in); err != nil {
+ return nil, err
+ }
+
+ if err := os.Rename(tempPath, realPath); err != nil {
+ return nil, status.Errorf(codes.Internal, "Promoting temporary directory failed: %v", err)
+ }
+
+ return &pb.CreateRepositoryFromSnapshotResponse{}, nil
+}
diff --git a/internal/service/repository/create_from_snapshot_test.go b/internal/service/repository/create_from_snapshot_test.go
new file mode 100644
index 000000000..c44630810
--- /dev/null
+++ b/internal/service/repository/create_from_snapshot_test.go
@@ -0,0 +1,226 @@
+package repository
+
+import (
+ "bytes"
+ "io"
+ "net/http"
+ "net/http/httptest"
+ "os"
+ "path/filepath"
+ "strings"
+ "testing"
+
+ "github.com/stretchr/testify/require"
+ pb "gitlab.com/gitlab-org/gitaly-proto/go"
+ "google.golang.org/grpc/codes"
+
+ "gitlab.com/gitlab-org/gitaly/internal/archive"
+ "gitlab.com/gitlab-org/gitaly/internal/testhelper"
+)
+
+var (
+ secret = "Magic secret"
+ redirectPath = "/redirecting-snapshot.tar"
+ tarPath = "/snapshot.tar"
+)
+
+type testhandler struct {
+ tarData io.Reader
+}
+
+func (h *testhandler) ServeHTTP(w http.ResponseWriter, r *http.Request) {
+ if r.Header.Get("Authorization") != secret {
+ http.Error(w, "Unuthorized", 401)
+ return
+ }
+
+ switch r.RequestURI {
+ case redirectPath:
+ http.Redirect(w, r, tarPath, http.StatusFound)
+ case tarPath:
+ io.Copy(w, h.tarData)
+ default:
+ http.Error(w, "Not found", 404)
+ }
+}
+
+// Create a tar file for the repo in memory, without relying on TarBuilder
+func generateTarFile(t *testing.T, path string) ([]byte, []string) {
+ data := testhelper.MustRunCommand(t, nil, "tar", "-C", path, "-cf", "-", ".")
+
+ entries, err := archive.TarEntries(bytes.NewReader(data))
+ require.NoError(t, err)
+
+ return data, entries
+}
+
+func createFromSnapshot(t *testing.T, req *pb.CreateRepositoryFromSnapshotRequest) (*pb.CreateRepositoryFromSnapshotResponse, error) {
+ server, serverSocketPath := runRepoServer(t)
+ defer server.Stop()
+
+ client, conn := newRepositoryClient(t, serverSocketPath)
+ defer conn.Close()
+
+ ctx, cancel := testhelper.Context()
+ defer cancel()
+
+ return client.CreateRepositoryFromSnapshot(ctx, req)
+}
+
+func TestCreateRepositoryFromSnapshotSuccess(t *testing.T) {
+ testRepo, repoPath, cleanupFn := testhelper.NewTestRepo(t)
+ defer cleanupFn()
+
+ // Ensure these won't be in the archive
+ require.NoError(t, os.Remove(filepath.Join(repoPath, "config")))
+ require.NoError(t, os.RemoveAll(filepath.Join(repoPath, "hooks")))
+
+ data, entries := generateTarFile(t, repoPath)
+
+ // Create a HTTP server that serves a given tar file
+ srv := httptest.NewServer(&testhandler{bytes.NewReader(data)})
+ defer srv.Close()
+
+ // Delete the repository so we can re-use the path
+ require.NoError(t, os.RemoveAll(repoPath))
+
+ req := &pb.CreateRepositoryFromSnapshotRequest{
+ Repository: testRepo,
+ HttpUrl: srv.URL + tarPath,
+ HttpAuth: secret,
+ }
+
+ rsp, err := createFromSnapshot(t, req)
+
+ require.NoError(t, err)
+ require.Equal(t, rsp, &pb.CreateRepositoryFromSnapshotResponse{})
+
+ require.DirExists(t, repoPath)
+ for _, entry := range entries {
+ if strings.HasSuffix(entry, "/") {
+ require.DirExists(t, filepath.Join(repoPath, entry), "directory %q not unpacked", entry)
+ } else {
+ require.FileExists(t, filepath.Join(repoPath, entry), "file %q not unpacked", entry)
+ }
+ }
+
+ // hooks/ and config were excluded, but the RPC should create them
+ require.FileExists(t, filepath.Join(repoPath, "config"), "Config file not created")
+
+ fi, err := os.Lstat(filepath.Join(repoPath, "hooks"))
+ require.NoError(t, err)
+ require.Equal(t, os.ModeSymlink, fi.Mode()&os.ModeSymlink, "Symlink to global hooks not created")
+}
+
+func TestCreateRepositoryFromSnapshotFailsIfRepositoryExists(t *testing.T) {
+ testRepo, _, cleanupFn := testhelper.NewTestRepo(t)
+ defer cleanupFn()
+
+ req := &pb.CreateRepositoryFromSnapshotRequest{Repository: testRepo}
+ rsp, err := createFromSnapshot(t, req)
+ testhelper.AssertGrpcError(t, err, codes.InvalidArgument, "destination directory exists")
+ require.Nil(t, rsp)
+}
+
+func TestCreateRepositoryFromSnapshotFailsIfBadURL(t *testing.T) {
+ testRepo, _, cleanupFn := testhelper.NewTestRepo(t)
+ cleanupFn() // free up the destination dir for use
+
+ req := &pb.CreateRepositoryFromSnapshotRequest{
+ Repository: testRepo,
+ HttpUrl: "invalid!scheme://invalid.invalid",
+ }
+
+ rsp, err := createFromSnapshot(t, req)
+ testhelper.AssertGrpcError(t, err, codes.InvalidArgument, "Bad HTTP URL")
+ require.Nil(t, rsp)
+}
+
+func TestCreateRepositoryFromSnapshotFailsIfBadAuth(t *testing.T) {
+ testRepo, _, cleanupFn := testhelper.NewTestRepo(t)
+ cleanupFn() // free up the destination dir for use
+
+ // Create a HTTP server that serves a given tar file
+ srv := httptest.NewServer(&testhandler{})
+ defer srv.Close()
+
+ req := &pb.CreateRepositoryFromSnapshotRequest{
+ Repository: testRepo,
+ HttpUrl: srv.URL + tarPath,
+ HttpAuth: "Bad authentication",
+ }
+
+ rsp, err := createFromSnapshot(t, req)
+ testhelper.AssertGrpcError(t, err, codes.Internal, "HTTP server: 401 Unauthorized")
+ require.Nil(t, rsp)
+}
+
+func TestCreateRepositoryFromSnapshotFailsIfHttp404(t *testing.T) {
+ testRepo, _, cleanupFn := testhelper.NewTestRepo(t)
+ cleanupFn() // free up the destination dir for use
+
+ // Create a HTTP server that serves a given tar file
+ srv := httptest.NewServer(&testhandler{})
+ defer srv.Close()
+
+ req := &pb.CreateRepositoryFromSnapshotRequest{
+ Repository: testRepo,
+ HttpUrl: srv.URL + tarPath + ".does-not-exist",
+ HttpAuth: secret,
+ }
+
+ rsp, err := createFromSnapshot(t, req)
+ testhelper.AssertGrpcError(t, err, codes.Internal, "HTTP server: 404 Not Found")
+ require.Nil(t, rsp)
+}
+
+func TestCreateRepositoryFromSnapshotDoesNotFollowRedirects(t *testing.T) {
+ testRepo, _, cleanupFn := testhelper.NewTestRepo(t)
+ cleanupFn() // free up the destination dir for use
+
+ // Create a HTTP server that serves a given tar file
+ srv := httptest.NewServer(&testhandler{})
+ defer srv.Close()
+
+ req := &pb.CreateRepositoryFromSnapshotRequest{
+ Repository: testRepo,
+ HttpUrl: srv.URL + redirectPath,
+ HttpAuth: secret,
+ }
+
+ rsp, err := createFromSnapshot(t, req)
+ testhelper.AssertGrpcError(t, err, codes.Internal, "HTTP server: 302 Found")
+ require.Nil(t, rsp)
+}
+
+func TestCreateRepositoryFromSnapshotHandlesMalformedResponse(t *testing.T) {
+ testRepo, repoPath, cleanupFn := testhelper.NewTestRepo(t)
+ defer cleanupFn()
+
+ require.NoError(t, os.Remove(filepath.Join(repoPath, "config")))
+ require.NoError(t, os.RemoveAll(filepath.Join(repoPath, "hooks")))
+
+ data, _ := generateTarFile(t, repoPath)
+ // Only serve half of the tar file
+ dataReader := io.LimitReader(bytes.NewReader(data), int64(len(data)/2))
+
+ srv := httptest.NewServer(&testhandler{dataReader})
+ defer srv.Close()
+
+ // Delete the repository so we can re-use the path
+ require.NoError(t, os.RemoveAll(repoPath))
+
+ req := &pb.CreateRepositoryFromSnapshotRequest{
+ Repository: testRepo,
+ HttpUrl: srv.URL + tarPath,
+ HttpAuth: secret,
+ }
+
+ rsp, err := createFromSnapshot(t, req)
+
+ require.Error(t, err)
+ require.Nil(t, rsp)
+
+ // Ensure that a partial result is not left in place
+ testhelper.AssertFileNotExists(t, repoPath)
+}
diff --git a/internal/service/repository/snapshot.go b/internal/service/repository/snapshot.go
new file mode 100644
index 000000000..418a274a5
--- /dev/null
+++ b/internal/service/repository/snapshot.go
@@ -0,0 +1,71 @@
+package repository
+
+import (
+ "regexp"
+
+ pb "gitlab.com/gitlab-org/gitaly-proto/go"
+ "google.golang.org/grpc/codes"
+ "google.golang.org/grpc/status"
+
+ "gitlab.com/gitlab-org/gitaly/internal/archive"
+ "gitlab.com/gitlab-org/gitaly/internal/helper"
+ "gitlab.com/gitlab-org/gitaly/streamio"
+)
+
+var objectFiles = []*regexp.Regexp{
+ regexp.MustCompile(`/[[:xdigit:]]{2}/[[:xdigit:]]{38}\z`),
+ regexp.MustCompile(`/pack/pack\-[[:xdigit:]]{40}\.(pack|idx)\z`),
+}
+
+func (s *server) GetSnapshot(in *pb.GetSnapshotRequest, stream pb.RepositoryService_GetSnapshotServer) error {
+ path, err := helper.GetRepoPath(in.Repository)
+ if err != nil {
+ return err
+ }
+
+ writer := streamio.NewWriter(func(p []byte) error {
+ return stream.Send(&pb.GetSnapshotResponse{Data: p})
+ })
+
+ // Building a raw archive may race with `git push`, but GitLab can enforce
+ // concurrency control if necessary. Using `TarBuilder` means we can keep
+ // going even if some files are added or removed during the operation.
+ builder := archive.NewTarBuilder(path, writer)
+
+ // Pick files directly by filename so we can get a snapshot even if the
+ // repository is corrupted. https://gitirc.eu/gitrepository-layout.html
+ // documents the various files and directories. We exclude the following
+ // on purpose:
+ //
+ // * branches - legacy, not replicated by git fetch
+ // * commondir - may differ between sites
+ // * config - may contain credentials, and cannot be managed by client
+ // * custom-hooks - GitLab-specific, no supported in Geo, may differ between sites
+ // * hooks - symlink, may differ between sites
+ // * {shared,}index[.*] - not found in bare repositories
+ // * info/{attributes,exclude,grafts} - not replicated by git fetch
+ // * info/refs - dumb protocol only
+ // * logs/* - not replicated by git fetch
+ // * modules/* - not replicated by git fetch
+ // * objects/info/* - unneeded (dumb protocol) or to do with alternates
+ // * worktrees/* - not replicated by git fetch
+
+ // References
+ builder.FileIfExist("HEAD")
+ builder.FileIfExist("packed-refs")
+ builder.RecursiveDirIfExist("refs")
+ builder.RecursiveDirIfExist("branches")
+
+ // The packfiles + any loose objects.
+ builder.RecursiveDirIfExist("objects", objectFiles...)
+
+ // In case this repository is a shallow clone. Seems unlikely, but better
+ // safe than sorry.
+ builder.FileIfExist("shallow")
+
+ if err := builder.Close(); err != nil {
+ return status.Errorf(codes.Internal, "Building snapshot failed: %v", err)
+ }
+
+ return nil
+}
diff --git a/internal/service/repository/snapshot_test.go b/internal/service/repository/snapshot_test.go
new file mode 100644
index 000000000..ce0f4e926
--- /dev/null
+++ b/internal/service/repository/snapshot_test.go
@@ -0,0 +1,113 @@
+package repository
+
+import (
+ "bytes"
+ "fmt"
+ "io"
+ "io/ioutil"
+ "os"
+ "path/filepath"
+ "strings"
+ "testing"
+
+ "github.com/stretchr/testify/require"
+ pb "gitlab.com/gitlab-org/gitaly-proto/go"
+ "google.golang.org/grpc/codes"
+
+ "gitlab.com/gitlab-org/gitaly/internal/archive"
+ "gitlab.com/gitlab-org/gitaly/internal/testhelper"
+ "gitlab.com/gitlab-org/gitaly/streamio"
+)
+
+func getSnapshot(t *testing.T, req *pb.GetSnapshotRequest) ([]byte, error) {
+ server, serverSocketPath := runRepoServer(t)
+ defer server.Stop()
+
+ client, conn := newRepositoryClient(t, serverSocketPath)
+ defer conn.Close()
+
+ ctx, cancel := testhelper.Context()
+ defer cancel()
+
+ stream, err := client.GetSnapshot(ctx, req)
+ if err != nil {
+ return nil, err
+ }
+
+ reader := streamio.NewReader(func() ([]byte, error) {
+ response, err := stream.Recv()
+ return response.GetData(), err
+ })
+
+ buf := bytes.NewBuffer(nil)
+ _, err = io.Copy(buf, reader)
+
+ return buf.Bytes(), err
+}
+
+func touch(t *testing.T, format string, args ...interface{}) {
+ path := fmt.Sprintf(format, args...)
+ require.NoError(t, ioutil.WriteFile(path, nil, 0644))
+}
+
+func TestGetSnapshotSuccess(t *testing.T) {
+ testRepo, repoPath, cleanupFn := testhelper.NewTestRepo(t)
+ defer cleanupFn()
+
+ // Ensure certain files exist in the test repo.
+ // CreateCommit produces a loose object with the given sha
+ sha := testhelper.CreateCommit(t, repoPath, "master", nil)
+ zeroes := strings.Repeat("0", 40)
+ require.NoError(t, os.MkdirAll(filepath.Join(repoPath, "hooks"), 0755))
+ require.NoError(t, os.MkdirAll(filepath.Join(repoPath, "objects/pack"), 0755))
+ touch(t, filepath.Join(repoPath, "shallow"))
+ touch(t, filepath.Join(repoPath, "objects/pack/pack-%s.pack"), zeroes)
+ touch(t, filepath.Join(repoPath, "objects/pack/pack-%s.idx"), zeroes)
+ touch(t, filepath.Join(repoPath, "objects/this-should-not-be-included"))
+
+ req := &pb.GetSnapshotRequest{Repository: testRepo}
+ data, err := getSnapshot(t, req)
+ require.NoError(t, err)
+
+ entries, err := archive.TarEntries(bytes.NewReader(data))
+ require.NoError(t, err)
+
+ require.Contains(t, entries, "HEAD")
+ require.Contains(t, entries, "packed-refs")
+ require.Contains(t, entries, "refs/heads/")
+ require.Contains(t, entries, "refs/tags/")
+ require.Contains(t, entries, fmt.Sprintf("objects/%s/%s", sha[0:2], sha[2:40]))
+ require.Contains(t, entries, "objects/pack/pack-"+zeroes+".idx")
+ require.Contains(t, entries, "objects/pack/pack-"+zeroes+".pack")
+ require.Contains(t, entries, "shallow")
+ require.NotContains(t, entries, "objects/this-should-not-be-included")
+ require.NotContains(t, entries, "config")
+ require.NotContains(t, entries, "hooks/")
+}
+
+func TestGetSnapshotFailsIfRepositoryMissing(t *testing.T) {
+ testRepo, _, cleanupFn := testhelper.NewTestRepo(t)
+ cleanupFn() // Remove the repo
+
+ req := &pb.GetSnapshotRequest{Repository: testRepo}
+ data, err := getSnapshot(t, req)
+ testhelper.AssertGrpcError(t, err, codes.NotFound, "not a git repository")
+ require.Empty(t, data)
+}
+
+func TestGetSnapshotFailsIfRepositoryContainsSymlink(t *testing.T) {
+ testRepo, repoPath, cleanupFn := testhelper.NewTestRepo(t)
+ defer cleanupFn()
+
+ // Make packed-refs into a symlink to break GetSnapshot()
+ packedRefsFile := filepath.Join(repoPath, "packed-refs")
+ require.NoError(t, os.Remove(packedRefsFile))
+ require.NoError(t, os.Symlink("HEAD", packedRefsFile))
+
+ req := &pb.GetSnapshotRequest{Repository: testRepo}
+ data, err := getSnapshot(t, req)
+ testhelper.AssertGrpcError(t, err, codes.Internal, "Building snapshot failed")
+
+ // At least some of the tar file should have been written so far
+ require.NotEmpty(t, data)
+}
diff --git a/internal/tempdir/tempdir.go b/internal/tempdir/tempdir.go
index 69565c590..649a91f0b 100644
--- a/internal/tempdir/tempdir.go
+++ b/internal/tempdir/tempdir.go
@@ -4,7 +4,7 @@ import (
"context"
"io/ioutil"
"os"
- "path"
+ "path/filepath"
"strings"
"time"
@@ -29,19 +29,30 @@ const (
// repository. The directory is removed with os.RemoveAll when ctx
// expires.
func New(ctx context.Context, repo *pb.Repository) (string, error) {
- storageDir, err := helper.GetStorageByName(repo.StorageName)
+ _, path, err := NewAsRepository(ctx, repo)
if err != nil {
return "", err
}
+ return path, nil
+}
+
+// NewAsRepository is the same as New, but it returns a *pb.Repository for the
+// created directory as well as the bare path as a string
+func NewAsRepository(ctx context.Context, repo *pb.Repository) (*pb.Repository, string, error) {
+ storageDir, err := helper.GetStorageByName(repo.StorageName)
+ if err != nil {
+ return nil, "", err
+ }
+
root := tmpRoot(storageDir)
if err := os.MkdirAll(root, 0700); err != nil {
- return "", err
+ return nil, "", err
}
tempDir, err := ioutil.TempDir(root, "repo")
if err != nil {
- return "", err
+ return nil, "", err
}
go func() {
@@ -49,11 +60,13 @@ func New(ctx context.Context, repo *pb.Repository) (string, error) {
os.RemoveAll(tempDir)
}()
- return tempDir, nil
+ newAsRepo := &pb.Repository{StorageName: repo.StorageName}
+ newAsRepo.RelativePath, err = filepath.Rel(storageDir, tempDir)
+ return newAsRepo, tempDir, err
}
func tmpRoot(storageRoot string) string {
- return path.Join(storageRoot, tmpRootPrefix)
+ return filepath.Join(storageRoot, tmpRootPrefix)
}
// StartCleaning starts tempdir cleanup goroutines.
@@ -100,7 +113,7 @@ func clean(dir string) error {
continue
}
- fullPath := path.Join(dir, info.Name())
+ fullPath := filepath.Join(dir, info.Name())
if err := housekeeping.FixDirectoryPermissions(fullPath); err != nil {
return err
}
diff --git a/internal/tempdir/tempdir_test.go b/internal/tempdir/tempdir_test.go
index e4dcc3e74..112659473 100644
--- a/internal/tempdir/tempdir_test.go
+++ b/internal/tempdir/tempdir_test.go
@@ -8,18 +8,26 @@ import (
"time"
pb "gitlab.com/gitlab-org/gitaly-proto/go"
+ "gitlab.com/gitlab-org/gitaly/internal/helper"
"gitlab.com/gitlab-org/gitaly/internal/testhelper"
"github.com/stretchr/testify/require"
)
-func TestNewSuccess(t *testing.T) {
+func TestNewAsRepositorySuccess(t *testing.T) {
ctx, cancel := testhelper.Context()
defer cancel()
repo := testhelper.TestRepository()
- tempDir, err := New(ctx, repo)
+ tempRepo, tempDir, err := NewAsRepository(ctx, repo)
require.NoError(t, err)
+ require.NotEqual(t, repo, tempRepo)
+ require.Equal(t, repo.StorageName, tempRepo.StorageName)
+ require.NotEqual(t, repo.RelativePath, tempRepo.RelativePath)
+
+ calculatedPath, err := helper.GetPath(tempRepo)
+ require.NoError(t, err)
+ require.Equal(t, tempDir, calculatedPath)
err = ioutil.WriteFile(path.Join(tempDir, "test"), []byte("hello"), 0644)
require.NoError(t, err, "write file in tempdir")
@@ -38,7 +46,7 @@ func TestNewSuccess(t *testing.T) {
require.True(t, os.IsNotExist(err), "expected directory to have been removed, got error %v", err)
}
-func TestNewFailStorageUnknown(t *testing.T) {
+func TestNewAsRepositoryFailStorageUnknown(t *testing.T) {
ctx, cancel := testhelper.Context()
defer cancel()
_, err := New(ctx, &pb.Repository{StorageName: "does-not-exist", RelativePath: "foobar.git"})
diff --git a/vendor/gitlab.com/gitlab-org/gitaly-proto/go/VERSION b/vendor/gitlab.com/gitlab-org/gitaly-proto/go/VERSION
index 5f8cbfdb7..13bb62857 100644
--- a/vendor/gitlab.com/gitlab-org/gitaly-proto/go/VERSION
+++ b/vendor/gitlab.com/gitlab-org/gitaly-proto/go/VERSION
@@ -1 +1 @@
-0.95.0
+0.96.0
diff --git a/vendor/gitlab.com/gitlab-org/gitaly-proto/go/blob.pb.go b/vendor/gitlab.com/gitlab-org/gitaly-proto/go/blob.pb.go
index 5fa4af8c3..e11ab9ab9 100644
--- a/vendor/gitlab.com/gitlab-org/gitaly-proto/go/blob.pb.go
+++ b/vendor/gitlab.com/gitlab-org/gitaly-proto/go/blob.pb.go
@@ -219,6 +219,10 @@ It has these top-level messages:
GetInfoAttributesResponse
CalculateChecksumRequest
CalculateChecksumResponse
+ GetSnapshotRequest
+ GetSnapshotResponse
+ CreateRepositoryFromSnapshotRequest
+ CreateRepositoryFromSnapshotResponse
ServerInfoRequest
ServerInfoResponse
Repository
diff --git a/vendor/gitlab.com/gitlab-org/gitaly-proto/go/repository-service.pb.go b/vendor/gitlab.com/gitlab-org/gitaly-proto/go/repository-service.pb.go
index 5ee68bb39..a5baefe25 100644
--- a/vendor/gitlab.com/gitlab-org/gitaly-proto/go/repository-service.pb.go
+++ b/vendor/gitlab.com/gitlab-org/gitaly-proto/go/repository-service.pb.go
@@ -1003,6 +1003,82 @@ func (m *CalculateChecksumResponse) GetChecksum() string {
return ""
}
+type GetSnapshotRequest struct {
+ Repository *Repository `protobuf:"bytes,1,opt,name=repository" json:"repository,omitempty"`
+}
+
+func (m *GetSnapshotRequest) Reset() { *m = GetSnapshotRequest{} }
+func (m *GetSnapshotRequest) String() string { return proto.CompactTextString(m) }
+func (*GetSnapshotRequest) ProtoMessage() {}
+func (*GetSnapshotRequest) Descriptor() ([]byte, []int) { return fileDescriptor10, []int{50} }
+
+func (m *GetSnapshotRequest) GetRepository() *Repository {
+ if m != nil {
+ return m.Repository
+ }
+ return nil
+}
+
+type GetSnapshotResponse struct {
+ Data []byte `protobuf:"bytes,1,opt,name=data,proto3" json:"data,omitempty"`
+}
+
+func (m *GetSnapshotResponse) Reset() { *m = GetSnapshotResponse{} }
+func (m *GetSnapshotResponse) String() string { return proto.CompactTextString(m) }
+func (*GetSnapshotResponse) ProtoMessage() {}
+func (*GetSnapshotResponse) Descriptor() ([]byte, []int) { return fileDescriptor10, []int{51} }
+
+func (m *GetSnapshotResponse) GetData() []byte {
+ if m != nil {
+ return m.Data
+ }
+ return nil
+}
+
+type CreateRepositoryFromSnapshotRequest struct {
+ Repository *Repository `protobuf:"bytes,1,opt,name=repository" json:"repository,omitempty"`
+ HttpUrl string `protobuf:"bytes,2,opt,name=http_url,json=httpUrl" json:"http_url,omitempty"`
+ HttpAuth string `protobuf:"bytes,3,opt,name=http_auth,json=httpAuth" json:"http_auth,omitempty"`
+}
+
+func (m *CreateRepositoryFromSnapshotRequest) Reset() { *m = CreateRepositoryFromSnapshotRequest{} }
+func (m *CreateRepositoryFromSnapshotRequest) String() string { return proto.CompactTextString(m) }
+func (*CreateRepositoryFromSnapshotRequest) ProtoMessage() {}
+func (*CreateRepositoryFromSnapshotRequest) Descriptor() ([]byte, []int) {
+ return fileDescriptor10, []int{52}
+}
+
+func (m *CreateRepositoryFromSnapshotRequest) GetRepository() *Repository {
+ if m != nil {
+ return m.Repository
+ }
+ return nil
+}
+
+func (m *CreateRepositoryFromSnapshotRequest) GetHttpUrl() string {
+ if m != nil {
+ return m.HttpUrl
+ }
+ return ""
+}
+
+func (m *CreateRepositoryFromSnapshotRequest) GetHttpAuth() string {
+ if m != nil {
+ return m.HttpAuth
+ }
+ return ""
+}
+
+type CreateRepositoryFromSnapshotResponse struct {
+}
+
+func (m *CreateRepositoryFromSnapshotResponse) Reset() { *m = CreateRepositoryFromSnapshotResponse{} }
+func (m *CreateRepositoryFromSnapshotResponse) String() string { return proto.CompactTextString(m) }
+func (*CreateRepositoryFromSnapshotResponse) ProtoMessage() {}
+func (*CreateRepositoryFromSnapshotResponse) Descriptor() ([]byte, []int) {
+ return fileDescriptor10, []int{53}
+}
+
func init() {
proto.RegisterType((*RepositoryExistsRequest)(nil), "gitaly.RepositoryExistsRequest")
proto.RegisterType((*RepositoryExistsResponse)(nil), "gitaly.RepositoryExistsResponse")
@@ -1054,6 +1130,10 @@ func init() {
proto.RegisterType((*GetInfoAttributesResponse)(nil), "gitaly.GetInfoAttributesResponse")
proto.RegisterType((*CalculateChecksumRequest)(nil), "gitaly.CalculateChecksumRequest")
proto.RegisterType((*CalculateChecksumResponse)(nil), "gitaly.CalculateChecksumResponse")
+ proto.RegisterType((*GetSnapshotRequest)(nil), "gitaly.GetSnapshotRequest")
+ proto.RegisterType((*GetSnapshotResponse)(nil), "gitaly.GetSnapshotResponse")
+ proto.RegisterType((*CreateRepositoryFromSnapshotRequest)(nil), "gitaly.CreateRepositoryFromSnapshotRequest")
+ proto.RegisterType((*CreateRepositoryFromSnapshotResponse)(nil), "gitaly.CreateRepositoryFromSnapshotResponse")
proto.RegisterEnum("gitaly.GetArchiveRequest_Format", GetArchiveRequest_Format_name, GetArchiveRequest_Format_value)
}
@@ -1093,6 +1173,8 @@ type RepositoryServiceClient interface {
GetInfoAttributes(ctx context.Context, in *GetInfoAttributesRequest, opts ...grpc.CallOption) (RepositoryService_GetInfoAttributesClient, error)
CalculateChecksum(ctx context.Context, in *CalculateChecksumRequest, opts ...grpc.CallOption) (*CalculateChecksumResponse, error)
Cleanup(ctx context.Context, in *CleanupRequest, opts ...grpc.CallOption) (*CleanupResponse, error)
+ GetSnapshot(ctx context.Context, in *GetSnapshotRequest, opts ...grpc.CallOption) (RepositoryService_GetSnapshotClient, error)
+ CreateRepositoryFromSnapshot(ctx context.Context, in *CreateRepositoryFromSnapshotRequest, opts ...grpc.CallOption) (*CreateRepositoryFromSnapshotResponse, error)
}
type repositoryServiceClient struct {
@@ -1422,6 +1504,47 @@ func (c *repositoryServiceClient) Cleanup(ctx context.Context, in *CleanupReques
return out, nil
}
+func (c *repositoryServiceClient) GetSnapshot(ctx context.Context, in *GetSnapshotRequest, opts ...grpc.CallOption) (RepositoryService_GetSnapshotClient, error) {
+ stream, err := grpc.NewClientStream(ctx, &_RepositoryService_serviceDesc.Streams[4], c.cc, "/gitaly.RepositoryService/GetSnapshot", opts...)
+ if err != nil {
+ return nil, err
+ }
+ x := &repositoryServiceGetSnapshotClient{stream}
+ if err := x.ClientStream.SendMsg(in); err != nil {
+ return nil, err
+ }
+ if err := x.ClientStream.CloseSend(); err != nil {
+ return nil, err
+ }
+ return x, nil
+}
+
+type RepositoryService_GetSnapshotClient interface {
+ Recv() (*GetSnapshotResponse, error)
+ grpc.ClientStream
+}
+
+type repositoryServiceGetSnapshotClient struct {
+ grpc.ClientStream
+}
+
+func (x *repositoryServiceGetSnapshotClient) Recv() (*GetSnapshotResponse, error) {
+ m := new(GetSnapshotResponse)
+ if err := x.ClientStream.RecvMsg(m); err != nil {
+ return nil, err
+ }
+ return m, nil
+}
+
+func (c *repositoryServiceClient) CreateRepositoryFromSnapshot(ctx context.Context, in *CreateRepositoryFromSnapshotRequest, opts ...grpc.CallOption) (*CreateRepositoryFromSnapshotResponse, error) {
+ out := new(CreateRepositoryFromSnapshotResponse)
+ err := grpc.Invoke(ctx, "/gitaly.RepositoryService/CreateRepositoryFromSnapshot", in, out, c.cc, opts...)
+ if err != nil {
+ return nil, err
+ }
+ return out, nil
+}
+
// Server API for RepositoryService service
type RepositoryServiceServer interface {
@@ -1450,6 +1573,8 @@ type RepositoryServiceServer interface {
GetInfoAttributes(*GetInfoAttributesRequest, RepositoryService_GetInfoAttributesServer) error
CalculateChecksum(context.Context, *CalculateChecksumRequest) (*CalculateChecksumResponse, error)
Cleanup(context.Context, *CleanupRequest) (*CleanupResponse, error)
+ GetSnapshot(*GetSnapshotRequest, RepositoryService_GetSnapshotServer) error
+ CreateRepositoryFromSnapshot(context.Context, *CreateRepositoryFromSnapshotRequest) (*CreateRepositoryFromSnapshotResponse, error)
}
func RegisterRepositoryServiceServer(s *grpc.Server, srv RepositoryServiceServer) {
@@ -1923,6 +2048,45 @@ func _RepositoryService_Cleanup_Handler(srv interface{}, ctx context.Context, de
return interceptor(ctx, in, info, handler)
}
+func _RepositoryService_GetSnapshot_Handler(srv interface{}, stream grpc.ServerStream) error {
+ m := new(GetSnapshotRequest)
+ if err := stream.RecvMsg(m); err != nil {
+ return err
+ }
+ return srv.(RepositoryServiceServer).GetSnapshot(m, &repositoryServiceGetSnapshotServer{stream})
+}
+
+type RepositoryService_GetSnapshotServer interface {
+ Send(*GetSnapshotResponse) error
+ grpc.ServerStream
+}
+
+type repositoryServiceGetSnapshotServer struct {
+ grpc.ServerStream
+}
+
+func (x *repositoryServiceGetSnapshotServer) Send(m *GetSnapshotResponse) error {
+ return x.ServerStream.SendMsg(m)
+}
+
+func _RepositoryService_CreateRepositoryFromSnapshot_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+ in := new(CreateRepositoryFromSnapshotRequest)
+ if err := dec(in); err != nil {
+ return nil, err
+ }
+ if interceptor == nil {
+ return srv.(RepositoryServiceServer).CreateRepositoryFromSnapshot(ctx, in)
+ }
+ info := &grpc.UnaryServerInfo{
+ Server: srv,
+ FullMethod: "/gitaly.RepositoryService/CreateRepositoryFromSnapshot",
+ }
+ handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+ return srv.(RepositoryServiceServer).CreateRepositoryFromSnapshot(ctx, req.(*CreateRepositoryFromSnapshotRequest))
+ }
+ return interceptor(ctx, in, info, handler)
+}
+
var _RepositoryService_serviceDesc = grpc.ServiceDesc{
ServiceName: "gitaly.RepositoryService",
HandlerType: (*RepositoryServiceServer)(nil),
@@ -2011,6 +2175,10 @@ var _RepositoryService_serviceDesc = grpc.ServiceDesc{
MethodName: "Cleanup",
Handler: _RepositoryService_Cleanup_Handler,
},
+ {
+ MethodName: "CreateRepositoryFromSnapshot",
+ Handler: _RepositoryService_CreateRepositoryFromSnapshot_Handler,
+ },
},
Streams: []grpc.StreamDesc{
{
@@ -2033,6 +2201,11 @@ var _RepositoryService_serviceDesc = grpc.ServiceDesc{
Handler: _RepositoryService_GetInfoAttributes_Handler,
ServerStreams: true,
},
+ {
+ StreamName: "GetSnapshot",
+ Handler: _RepositoryService_GetSnapshot_Handler,
+ ServerStreams: true,
+ },
},
Metadata: "repository-service.proto",
}
@@ -2040,105 +2213,112 @@ var _RepositoryService_serviceDesc = grpc.ServiceDesc{
func init() { proto.RegisterFile("repository-service.proto", fileDescriptor10) }
var fileDescriptor10 = []byte{
- // 1587 bytes of a gzipped FileDescriptorProto
- 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x58, 0xef, 0x6e, 0xdb, 0x36,
- 0x10, 0xb7, 0xf3, 0xd7, 0xbe, 0xb8, 0xad, 0xc3, 0xfc, 0x93, 0x95, 0xb4, 0x49, 0xb8, 0x62, 0xcb,
- 0xda, 0x2e, 0x28, 0x92, 0x0f, 0x1b, 0xb0, 0x0d, 0x45, 0x92, 0x35, 0x89, 0xd7, 0xa6, 0xc8, 0x94,
- 0x0e, 0x05, 0x82, 0x0d, 0x86, 0x22, 0xd3, 0xb6, 0x60, 0x59, 0x72, 0x49, 0x2a, 0x6d, 0xfa, 0x79,
- 0x1f, 0xf6, 0x5c, 0x7d, 0x85, 0x3d, 0xc6, 0x5e, 0x62, 0x10, 0x45, 0x89, 0x92, 0x25, 0x65, 0x05,
- 0xd4, 0x61, 0xdf, 0xc4, 0xe3, 0xf1, 0x77, 0xa7, 0x3b, 0xde, 0xe9, 0x77, 0x02, 0x8d, 0x92, 0xb1,
- 0xc7, 0x6c, 0xee, 0xd1, 0x9b, 0x6f, 0x18, 0xa1, 0xd7, 0xb6, 0x45, 0x76, 0xc7, 0xd4, 0xe3, 0x1e,
- 0x9a, 0xeb, 0xdb, 0xdc, 0x74, 0x6e, 0xf4, 0x06, 0x1b, 0x98, 0x94, 0x74, 0x43, 0x29, 0x3e, 0x83,
- 0x35, 0x23, 0x3e, 0xf1, 0xfc, 0xbd, 0xcd, 0x38, 0x33, 0xc8, 0x5b, 0x9f, 0x30, 0x8e, 0xf6, 0x00,
- 0x14, 0x98, 0x56, 0xdd, 0xaa, 0xee, 0x2c, 0xec, 0xa1, 0xdd, 0x10, 0x65, 0x57, 0x1d, 0x32, 0x12,
- 0x5a, 0x78, 0x0f, 0xb4, 0x2c, 0x1c, 0x1b, 0x7b, 0x2e, 0x23, 0x68, 0x15, 0xe6, 0x88, 0x90, 0x08,
- 0xac, 0x9a, 0x21, 0x57, 0xf8, 0x95, 0x38, 0x63, 0x5a, 0xc3, 0xb6, 0x6b, 0x51, 0x32, 0x22, 0x2e,
- 0x37, 0x9d, 0x32, 0x3e, 0xac, 0x43, 0x2b, 0x07, 0x2f, 0x74, 0x02, 0x3b, 0xb0, 0x18, 0x6e, 0x1e,
- 0xfb, 0x4e, 0x19, 0x2b, 0xe8, 0x0b, 0xb8, 0x63, 0x51, 0x62, 0x72, 0xd2, 0xb9, 0xb2, 0xf9, 0xc8,
- 0x1c, 0x6b, 0x53, 0xe2, 0xa5, 0x1a, 0xa1, 0xf0, 0x50, 0xc8, 0xf0, 0x32, 0xa0, 0xa4, 0x35, 0xe9,
- 0xc3, 0x18, 0x56, 0x4e, 0x4c, 0x7a, 0x65, 0xf6, 0xc9, 0x91, 0xe7, 0x38, 0xc4, 0xe2, 0xff, 0xb9,
- 0x1f, 0x1a, 0xac, 0x4e, 0x5a, 0x94, 0xbe, 0xfc, 0x04, 0x77, 0x8f, 0x1c, 0x62, 0xba, 0xfe, 0xb8,
- 0x4c, 0xc8, 0x17, 0xe1, 0x5e, 0x8c, 0x22, 0x81, 0x5f, 0xc0, 0x8a, 0x52, 0xbe, 0xb0, 0x3f, 0x90,
- 0x32, 0xf8, 0x4f, 0x60, 0x75, 0x12, 0x4c, 0x5e, 0x2a, 0x04, 0x33, 0xcc, 0xfe, 0x40, 0x04, 0xce,
- 0xb4, 0x21, 0x9e, 0xf1, 0x10, 0x5a, 0x07, 0xe3, 0xb1, 0x73, 0x73, 0x62, 0x73, 0x93, 0x73, 0x6a,
- 0x5f, 0xf9, 0x9c, 0x94, 0xb9, 0xd5, 0x48, 0x87, 0x1a, 0x25, 0xd7, 0x36, 0xb3, 0x3d, 0x57, 0x84,
- 0xb7, 0x61, 0xc4, 0x6b, 0xbc, 0x01, 0x7a, 0x9e, 0x31, 0x19, 0x85, 0x3f, 0xa6, 0x00, 0x1d, 0x13,
- 0x6e, 0x0d, 0x0c, 0x32, 0xf2, 0x78, 0x99, 0x18, 0x04, 0xe5, 0x43, 0x05, 0x88, 0x70, 0xa1, 0x6e,
- 0xc8, 0x15, 0x5a, 0x86, 0xd9, 0x9e, 0x47, 0x2d, 0xa2, 0x4d, 0x8b, 0xc4, 0x87, 0x0b, 0xb4, 0x06,
- 0xf3, 0xae, 0xd7, 0xe1, 0x66, 0x9f, 0x69, 0x33, 0x61, 0xb5, 0xb9, 0xde, 0x6b, 0xb3, 0xcf, 0x90,
- 0x06, 0xf3, 0xdc, 0x1e, 0x11, 0xcf, 0xe7, 0xda, 0xec, 0x56, 0x75, 0x67, 0xd6, 0x88, 0x96, 0xc1,
- 0x11, 0xc6, 0x06, 0x9d, 0x21, 0xb9, 0xd1, 0xe6, 0x42, 0x0b, 0x8c, 0x0d, 0x5e, 0x90, 0x1b, 0xb4,
- 0x09, 0x0b, 0x43, 0xd7, 0x7b, 0xe7, 0x76, 0x06, 0x5e, 0x50, 0xbd, 0xf3, 0x62, 0x13, 0x84, 0xe8,
- 0x34, 0x90, 0xa0, 0x16, 0xd4, 0x5c, 0xaf, 0x33, 0xa6, 0xbe, 0x4b, 0xb4, 0xba, 0xb0, 0x36, 0xef,
- 0x7a, 0xe7, 0xc1, 0xf2, 0xe7, 0x99, 0x5a, 0xad, 0x59, 0xc7, 0x2b, 0xb0, 0x94, 0x8a, 0x82, 0x8c,
- 0xce, 0x19, 0xac, 0x1d, 0x89, 0x6b, 0x9a, 0x78, 0xe5, 0x12, 0xb7, 0x44, 0x07, 0x2d, 0x0b, 0x27,
- 0x4d, 0xfd, 0x5d, 0x85, 0xc5, 0x13, 0xc2, 0x0f, 0xa8, 0x35, 0xb0, 0xaf, 0x4b, 0xe5, 0x61, 0x1d,
- 0xea, 0x96, 0x37, 0x1a, 0xd9, 0xbc, 0x63, 0x77, 0x65, 0x2a, 0x6a, 0xa1, 0xa0, 0xdd, 0x0d, 0x92,
- 0x34, 0xa6, 0xa4, 0x67, 0xbf, 0x17, 0xd9, 0xa8, 0x1b, 0x72, 0x85, 0xbe, 0x83, 0xb9, 0x9e, 0x47,
- 0x47, 0x26, 0x17, 0xd9, 0xb8, 0xbb, 0xb7, 0x15, 0x19, 0xc9, 0xf8, 0xb4, 0x7b, 0x2c, 0xf4, 0x0c,
- 0xa9, 0x8f, 0xf7, 0x61, 0x2e, 0x94, 0xa0, 0x79, 0x98, 0xbe, 0x6c, 0x9f, 0x37, 0x2b, 0xc1, 0xc3,
- 0xeb, 0x03, 0xa3, 0x59, 0x45, 0x00, 0x73, 0xaf, 0x0f, 0x8c, 0xce, 0xc9, 0x65, 0x73, 0x0a, 0x2d,
- 0xc0, 0x7c, 0xf0, 0x7c, 0x78, 0xb9, 0xd7, 0x9c, 0xc6, 0x3b, 0x80, 0x92, 0xc0, 0xaa, 0x56, 0xba,
- 0x26, 0x37, 0xc5, 0x7b, 0x36, 0x0c, 0xf1, 0x1c, 0xa4, 0xe0, 0xd4, 0x64, 0x2f, 0x3d, 0xcb, 0x74,
- 0x0e, 0xa9, 0xe9, 0x5a, 0x83, 0x52, 0x95, 0x82, 0x9f, 0x82, 0x96, 0x85, 0x93, 0xe6, 0x97, 0x61,
- 0xf6, 0xda, 0x74, 0x7c, 0x22, 0xdb, 0x7f, 0xb8, 0xc0, 0x7f, 0x55, 0x41, 0x13, 0x77, 0xe3, 0xc2,
- 0xf3, 0xa9, 0x45, 0xc2, 0x53, 0x65, 0xf2, 0xf3, 0x0c, 0x16, 0x99, 0x80, 0xea, 0x24, 0x8e, 0x4e,
- 0x15, 0x1e, 0x6d, 0x86, 0xca, 0x46, 0xaa, 0xa3, 0x4a, 0x80, 0x2b, 0xe1, 0x8c, 0x48, 0x65, 0xc3,
- 0x68, 0xb0, 0x84, 0x83, 0xe8, 0x3e, 0x00, 0x37, 0x69, 0x9f, 0xf0, 0x0e, 0x25, 0x3d, 0x91, 0xd4,
- 0x86, 0x51, 0x0f, 0x25, 0x06, 0xe9, 0xe1, 0x7d, 0x68, 0xe5, 0xbc, 0x94, 0xfa, 0x10, 0x52, 0xc2,
- 0x7c, 0x87, 0x47, 0x1f, 0xc2, 0x70, 0x85, 0x0f, 0x60, 0xe1, 0x98, 0x59, 0xc3, 0x32, 0xf1, 0x7f,
- 0x08, 0x8d, 0x10, 0x42, 0xc5, 0x9c, 0x50, 0xea, 0x51, 0x99, 0xf3, 0x70, 0x81, 0x3f, 0x56, 0xe1,
- 0xde, 0x1b, 0x6a, 0x07, 0x85, 0xd2, 0x2b, 0x13, 0xea, 0x26, 0x4c, 0x07, 0x6f, 0x1f, 0xb6, 0xc4,
- 0xe0, 0x31, 0xd5, 0x29, 0xa7, 0xd3, 0x9d, 0x12, 0x6d, 0x43, 0xc3, 0x73, 0xba, 0x9d, 0x78, 0x3f,
- 0x0c, 0xda, 0x82, 0xe7, 0x74, 0x8d, 0x48, 0x25, 0xee, 0x65, 0xb3, 0xc9, 0x5e, 0xb6, 0x0c, 0xb3,
- 0x6c, 0x40, 0x1c, 0x47, 0xb4, 0xa5, 0x9a, 0x11, 0x2e, 0xf0, 0x0e, 0x34, 0xd5, 0x3b, 0xdc, 0xfa,
- 0xba, 0x03, 0x58, 0x3e, 0xb6, 0xdd, 0xee, 0x19, 0xa1, 0x7d, 0x72, 0x68, 0xb2, 0x52, 0xd5, 0xbf,
- 0x01, 0xf5, 0xe8, 0x05, 0x98, 0x36, 0xb5, 0x35, 0x1d, 0xa4, 0x3d, 0x16, 0xe0, 0xc7, 0xb0, 0x32,
- 0x61, 0x49, 0x95, 0xde, 0x95, 0xc9, 0xc2, 0xab, 0x5f, 0x37, 0xc4, 0x33, 0xfe, 0xb3, 0x0a, 0x8b,
- 0x61, 0xbf, 0x3a, 0xf6, 0xe8, 0xf0, 0xff, 0xbc, 0xf2, 0x01, 0x4f, 0x49, 0x7a, 0x12, 0x73, 0xa5,
- 0x56, 0x9b, 0x19, 0x24, 0x70, 0xb6, 0xed, 0x9e, 0x53, 0xaf, 0x4f, 0x09, 0x63, 0x25, 0x5b, 0x27,
- 0x15, 0x70, 0x89, 0xd6, 0x19, 0x0a, 0xda, 0x5d, 0xfc, 0x23, 0xe8, 0x79, 0xd6, 0x64, 0x00, 0x37,
- 0x61, 0xc1, 0x76, 0x3b, 0x63, 0x29, 0x96, 0x85, 0x03, 0x76, 0xac, 0x18, 0x3a, 0x7b, 0xf1, 0xd6,
- 0x37, 0xd9, 0xe0, 0xb3, 0x39, 0xcb, 0x04, 0x5c, 0xc2, 0xd9, 0x50, 0x10, 0x39, 0x9b, 0xb5, 0xf6,
- 0xa9, 0xce, 0xf6, 0xe0, 0xc1, 0xe4, 0x97, 0xea, 0x98, 0x7a, 0xa3, 0x5f, 0x8d, 0x97, 0x25, 0xcb,
- 0xd1, 0xa7, 0x8e, 0xf4, 0x35, 0x78, 0xc4, 0xdb, 0xb0, 0x59, 0x68, 0x47, 0x26, 0xb9, 0x0d, 0x4b,
- 0xa1, 0xca, 0xa1, 0xef, 0x76, 0x9d, 0x52, 0x2c, 0xed, 0x11, 0x2c, 0xa7, 0xa1, 0x6e, 0xf9, 0xee,
- 0x10, 0x40, 0xa2, 0x7a, 0x8f, 0x3c, 0xb7, 0x67, 0xf7, 0x4b, 0xe6, 0xa9, 0xe7, 0x3b, 0x4e, 0x67,
- 0x6c, 0xf2, 0x41, 0x94, 0xa7, 0x40, 0x70, 0x6e, 0xf2, 0x01, 0x7e, 0x0c, 0x4b, 0x29, 0x33, 0xb7,
- 0xf6, 0x89, 0x21, 0x6c, 0xe7, 0x45, 0xab, 0x74, 0x60, 0xe2, 0x00, 0x4c, 0x25, 0x02, 0xf0, 0x10,
- 0xf0, 0x6d, 0xc6, 0x64, 0x76, 0x4e, 0x01, 0x05, 0x0d, 0xe5, 0xa5, 0x6d, 0x11, 0xb7, 0x54, 0xe3,
- 0xc2, 0x47, 0xb0, 0x94, 0x42, 0x92, 0x91, 0x78, 0x02, 0xc8, 0x09, 0x45, 0x1d, 0x36, 0xf0, 0x28,
- 0xef, 0xb8, 0xe6, 0x28, 0x6a, 0x53, 0x4d, 0xb9, 0x73, 0x11, 0x6c, 0xbc, 0x32, 0x47, 0x24, 0x18,
- 0xd5, 0x4e, 0x08, 0x6f, 0xbb, 0x3d, 0xef, 0xe0, 0x73, 0x10, 0x6b, 0xfc, 0x3d, 0xb4, 0x72, 0xf0,
- 0xa4, 0x6b, 0x0f, 0x00, 0x14, 0xa3, 0x96, 0x99, 0x4a, 0x48, 0x02, 0x67, 0x8e, 0x4c, 0xc7, 0xf2,
- 0x1d, 0x93, 0x93, 0xa3, 0x01, 0xb1, 0x86, 0xcc, 0x1f, 0x95, 0x71, 0xe6, 0x5b, 0x68, 0xe5, 0xe0,
- 0x49, 0x67, 0x74, 0xa8, 0x59, 0x52, 0x26, 0xa3, 0x13, 0xaf, 0xf7, 0x3e, 0x36, 0xc5, 0x50, 0x19,
- 0x8d, 0x27, 0xe1, 0xd4, 0x8d, 0xde, 0x40, 0x73, 0x72, 0x14, 0x46, 0x9b, 0x59, 0x17, 0x52, 0x33,
- 0xb7, 0xbe, 0x55, 0xac, 0x20, 0x6f, 0x44, 0x05, 0x5d, 0x46, 0x23, 0x6c, 0x62, 0xbe, 0x45, 0xc9,
- 0x83, 0xb9, 0xa3, 0xb4, 0xbe, 0x7d, 0x8b, 0x46, 0x8c, 0xfd, 0x1c, 0x40, 0x0d, 0xac, 0xa8, 0x95,
- 0x3e, 0x92, 0x18, 0x99, 0x75, 0x3d, 0x6f, 0x2b, 0x86, 0xf9, 0x05, 0xee, 0xa6, 0xe7, 0x4d, 0x74,
- 0x3f, 0x26, 0xbc, 0x79, 0x93, 0xaf, 0xfe, 0xa0, 0x68, 0x3b, 0x09, 0x99, 0x1e, 0x01, 0x15, 0x64,
- 0xee, 0x9c, 0xa9, 0x20, 0xf3, 0x27, 0x47, 0x5c, 0x41, 0xbf, 0x03, 0xca, 0x8e, 0x6e, 0x28, 0x8e,
- 0x53, 0xe1, 0x0c, 0xa9, 0xe3, 0xdb, 0x54, 0x62, 0xf8, 0x53, 0x58, 0x48, 0x0c, 0x3d, 0x28, 0x8e,
- 0x58, 0x76, 0x1e, 0xd4, 0xd7, 0x73, 0xf7, 0x62, 0xa4, 0x37, 0xd0, 0x9c, 0xec, 0x15, 0xea, 0x2a,
- 0x15, 0x4c, 0x50, 0xea, 0x2a, 0x15, 0xce, 0x44, 0x15, 0x74, 0x02, 0xa0, 0xe6, 0x04, 0x95, 0xee,
- 0xcc, 0x50, 0xa2, 0xd2, 0x9d, 0x1d, 0x2b, 0x70, 0xe5, 0x69, 0x35, 0xf0, 0x70, 0x92, 0xf7, 0x2b,
- 0x0f, 0x0b, 0x06, 0x0c, 0xe5, 0x61, 0xd1, 0xc8, 0x10, 0x5e, 0xf6, 0x0c, 0x91, 0x56, 0x97, 0xbd,
- 0x68, 0x70, 0x50, 0x97, 0xbd, 0x90, 0x85, 0xe3, 0x0a, 0xda, 0x87, 0x99, 0x80, 0x2c, 0xa3, 0xa5,
- 0x58, 0x59, 0xb1, 0x6f, 0x7d, 0x39, 0x2d, 0x8c, 0x0f, 0x3d, 0x83, 0x5a, 0x44, 0x3b, 0xd1, 0x5a,
- 0xa4, 0x33, 0x41, 0xa6, 0x75, 0x2d, 0xbb, 0x11, 0x03, 0xbc, 0x82, 0x3b, 0x29, 0x8e, 0x88, 0x36,
- 0x62, 0x4b, 0x39, 0x24, 0x55, 0xbf, 0x5f, 0xb0, 0x9b, 0x2c, 0x59, 0xc5, 0xdd, 0x54, 0x0e, 0x33,
- 0xcc, 0x52, 0xe5, 0x30, 0x87, 0xea, 0x89, 0x62, 0xc8, 0xd2, 0x2f, 0x55, 0x0c, 0x85, 0x44, 0x50,
- 0x15, 0x43, 0x31, 0x7b, 0x8b, 0xe0, 0x27, 0x09, 0x53, 0x12, 0xbe, 0x80, 0xba, 0x25, 0xe1, 0x8b,
- 0xf8, 0x16, 0xae, 0x20, 0x27, 0xfb, 0x27, 0x41, 0x12, 0x1d, 0xf4, 0x65, 0x51, 0x1d, 0xa4, 0x19,
- 0x97, 0xfe, 0xd5, 0xbf, 0xea, 0xc5, 0xd6, 0xce, 0xa0, 0x91, 0x24, 0x3a, 0x68, 0x3d, 0x7d, 0x34,
- 0x45, 0x18, 0xf4, 0x8d, 0xfc, 0xcd, 0x44, 0xf1, 0xbc, 0x03, 0xbd, 0x98, 0x0a, 0xa0, 0xaf, 0x6f,
- 0xf3, 0x2b, 0x6d, 0xea, 0xd1, 0xa7, 0xa8, 0x46, 0x86, 0x77, 0xaa, 0x41, 0x87, 0x4a, 0xb0, 0x23,
- 0xd5, 0xa1, 0xb2, 0xcc, 0x4c, 0x75, 0xa8, 0x1c, 0x3a, 0x25, 0x7b, 0x9d, 0x62, 0x17, 0x89, 0x5e,
- 0x97, 0x21, 0x2f, 0x89, 0x5e, 0x97, 0xa5, 0x23, 0xb8, 0x82, 0x7e, 0x13, 0xff, 0x69, 0xd2, 0x94,
- 0x00, 0x25, 0x7f, 0x97, 0xe4, 0xb2, 0x0f, 0x55, 0xf0, 0x85, 0x7c, 0x42, 0x84, 0xfa, 0x12, 0x16,
- 0x33, 0xdf, 0x78, 0x85, 0x5e, 0x44, 0x27, 0x14, 0x7a, 0x21, 0x41, 0xc0, 0x15, 0xf4, 0x03, 0xcc,
- 0xcb, 0x9f, 0xa0, 0x68, 0x35, 0xd6, 0x4f, 0xfd, 0x5b, 0xd5, 0xd7, 0x32, 0xf2, 0xe8, 0xf4, 0xd5,
- 0x9c, 0xf8, 0x1f, 0xbf, 0xff, 0x4f, 0x00, 0x00, 0x00, 0xff, 0xff, 0x23, 0xc7, 0xc8, 0xe3, 0xc1,
- 0x17, 0x00, 0x00,
+ // 1698 bytes of a gzipped FileDescriptorProto
+ 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x59, 0x5f, 0x6f, 0xdb, 0xc8,
+ 0x11, 0x97, 0xfc, 0x4f, 0xd2, 0x48, 0x97, 0x93, 0xd7, 0xff, 0x28, 0xda, 0x89, 0xed, 0xbd, 0xe0,
+ 0xea, 0xbb, 0xa4, 0xc6, 0xc1, 0x7e, 0x68, 0x81, 0xb6, 0x38, 0xd8, 0x6e, 0x6c, 0x2b, 0x89, 0x03,
+ 0x97, 0x4e, 0x10, 0xc0, 0x68, 0x21, 0xd0, 0xd4, 0x4a, 0x24, 0x44, 0x91, 0xca, 0xee, 0xd2, 0x89,
+ 0xd3, 0xd7, 0x3e, 0x14, 0xe8, 0x53, 0xbf, 0x52, 0xbf, 0x42, 0x3f, 0x46, 0xbf, 0x44, 0xc1, 0xe5,
+ 0x8a, 0x4b, 0x8a, 0xa4, 0x1a, 0x80, 0x29, 0xee, 0x8d, 0x3b, 0xbb, 0xfb, 0x9b, 0xd9, 0x99, 0x9d,
+ 0xd1, 0x6f, 0x56, 0xa0, 0x51, 0x32, 0xf1, 0x99, 0xc3, 0x7d, 0xfa, 0xf0, 0x6b, 0x46, 0xe8, 0xbd,
+ 0x63, 0x91, 0xc3, 0x09, 0xf5, 0xb9, 0x8f, 0x56, 0x86, 0x0e, 0x37, 0xdd, 0x07, 0xbd, 0xc5, 0x6c,
+ 0x93, 0x92, 0x7e, 0x24, 0xc5, 0x57, 0xb0, 0x65, 0xc4, 0x3b, 0x5e, 0x7c, 0x72, 0x18, 0x67, 0x06,
+ 0xf9, 0x10, 0x10, 0xc6, 0xd1, 0x11, 0x80, 0x02, 0xd3, 0xaa, 0x7b, 0xd5, 0x83, 0xe6, 0x11, 0x3a,
+ 0x8c, 0x50, 0x0e, 0xd5, 0x26, 0x23, 0xb1, 0x0a, 0x1f, 0x81, 0x96, 0x85, 0x63, 0x13, 0xdf, 0x63,
+ 0x04, 0x6d, 0xc2, 0x0a, 0x11, 0x12, 0x81, 0x55, 0x37, 0xe4, 0x08, 0xbf, 0x11, 0x7b, 0x4c, 0x6b,
+ 0xd4, 0xf5, 0x2c, 0x4a, 0xc6, 0xc4, 0xe3, 0xa6, 0x5b, 0xc6, 0x86, 0x6d, 0xe8, 0xe4, 0xe0, 0x45,
+ 0x46, 0x60, 0x17, 0x56, 0xa3, 0xc9, 0xf3, 0xc0, 0x2d, 0xa3, 0x05, 0x7d, 0x07, 0xdf, 0x58, 0x94,
+ 0x98, 0x9c, 0xf4, 0xee, 0x1c, 0x3e, 0x36, 0x27, 0xda, 0x82, 0x38, 0x54, 0x2b, 0x12, 0x9e, 0x0a,
+ 0x19, 0x5e, 0x07, 0x94, 0xd4, 0x26, 0x6d, 0x98, 0xc0, 0xc6, 0x85, 0x49, 0xef, 0xcc, 0x21, 0x39,
+ 0xf3, 0x5d, 0x97, 0x58, 0xfc, 0xff, 0x6e, 0x87, 0x06, 0x9b, 0xb3, 0x1a, 0xa5, 0x2d, 0x7f, 0x84,
+ 0x47, 0x67, 0x2e, 0x31, 0xbd, 0x60, 0x52, 0xc6, 0xe5, 0xab, 0xf0, 0x6d, 0x8c, 0x22, 0x81, 0x5f,
+ 0xc1, 0x86, 0x5a, 0x7c, 0xe3, 0x7c, 0x26, 0x65, 0xf0, 0x9f, 0xc3, 0xe6, 0x2c, 0x98, 0xbc, 0x54,
+ 0x08, 0x96, 0x98, 0xf3, 0x99, 0x08, 0x9c, 0x45, 0x43, 0x7c, 0xe3, 0x11, 0x74, 0x4e, 0x26, 0x13,
+ 0xf7, 0xe1, 0xc2, 0xe1, 0x26, 0xe7, 0xd4, 0xb9, 0x0b, 0x38, 0x29, 0x73, 0xab, 0x91, 0x0e, 0x75,
+ 0x4a, 0xee, 0x1d, 0xe6, 0xf8, 0x9e, 0x70, 0x6f, 0xcb, 0x88, 0xc7, 0x78, 0x07, 0xf4, 0x3c, 0x65,
+ 0xd2, 0x0b, 0x7f, 0x5b, 0x00, 0x74, 0x4e, 0xb8, 0x65, 0x1b, 0x64, 0xec, 0xf3, 0x32, 0x3e, 0x08,
+ 0xd3, 0x87, 0x0a, 0x10, 0x61, 0x42, 0xc3, 0x90, 0x23, 0xb4, 0x0e, 0xcb, 0x03, 0x9f, 0x5a, 0x44,
+ 0x5b, 0x14, 0x81, 0x8f, 0x06, 0x68, 0x0b, 0x6a, 0x9e, 0xdf, 0xe3, 0xe6, 0x90, 0x69, 0x4b, 0x51,
+ 0xb6, 0x79, 0xfe, 0x5b, 0x73, 0xc8, 0x90, 0x06, 0x35, 0xee, 0x8c, 0x89, 0x1f, 0x70, 0x6d, 0x79,
+ 0xaf, 0x7a, 0xb0, 0x6c, 0x4c, 0x87, 0xe1, 0x16, 0xc6, 0xec, 0xde, 0x88, 0x3c, 0x68, 0x2b, 0x91,
+ 0x06, 0xc6, 0xec, 0x57, 0xe4, 0x01, 0xed, 0x42, 0x73, 0xe4, 0xf9, 0x1f, 0xbd, 0x9e, 0xed, 0x87,
+ 0xd9, 0x5b, 0x13, 0x93, 0x20, 0x44, 0x97, 0xa1, 0x04, 0x75, 0xa0, 0xee, 0xf9, 0xbd, 0x09, 0x0d,
+ 0x3c, 0xa2, 0x35, 0x84, 0xb6, 0x9a, 0xe7, 0x5f, 0x87, 0xc3, 0x97, 0x4b, 0xf5, 0x7a, 0xbb, 0x81,
+ 0x37, 0x60, 0x2d, 0xe5, 0x05, 0xe9, 0x9d, 0x2b, 0xd8, 0x3a, 0x13, 0xd7, 0x34, 0x71, 0xe4, 0x12,
+ 0xb7, 0x44, 0x07, 0x2d, 0x0b, 0x27, 0x55, 0xfd, 0xa7, 0x0a, 0xab, 0x17, 0x84, 0x9f, 0x50, 0xcb,
+ 0x76, 0xee, 0x4b, 0xc5, 0x61, 0x1b, 0x1a, 0x96, 0x3f, 0x1e, 0x3b, 0xbc, 0xe7, 0xf4, 0x65, 0x28,
+ 0xea, 0x91, 0xa0, 0xdb, 0x0f, 0x83, 0x34, 0xa1, 0x64, 0xe0, 0x7c, 0x12, 0xd1, 0x68, 0x18, 0x72,
+ 0x84, 0x7e, 0x0b, 0x2b, 0x03, 0x9f, 0x8e, 0x4d, 0x2e, 0xa2, 0xf1, 0xe8, 0x68, 0x6f, 0xaa, 0x24,
+ 0x63, 0xd3, 0xe1, 0xb9, 0x58, 0x67, 0xc8, 0xf5, 0xf8, 0x18, 0x56, 0x22, 0x09, 0xaa, 0xc1, 0xe2,
+ 0x6d, 0xf7, 0xba, 0x5d, 0x09, 0x3f, 0xde, 0x9e, 0x18, 0xed, 0x2a, 0x02, 0x58, 0x79, 0x7b, 0x62,
+ 0xf4, 0x2e, 0x6e, 0xdb, 0x0b, 0xa8, 0x09, 0xb5, 0xf0, 0xfb, 0xf4, 0xf6, 0xa8, 0xbd, 0x88, 0x0f,
+ 0x00, 0x25, 0x81, 0x55, 0xae, 0xf4, 0x4d, 0x6e, 0x8a, 0x73, 0xb6, 0x0c, 0xf1, 0x1d, 0x86, 0xe0,
+ 0xd2, 0x64, 0xaf, 0x7d, 0xcb, 0x74, 0x4f, 0xa9, 0xe9, 0x59, 0x76, 0xa9, 0x4c, 0xc1, 0x3f, 0x81,
+ 0x96, 0x85, 0x93, 0xea, 0xd7, 0x61, 0xf9, 0xde, 0x74, 0x03, 0x22, 0xcb, 0x7f, 0x34, 0xc0, 0xff,
+ 0xae, 0x82, 0x26, 0xee, 0xc6, 0x8d, 0x1f, 0x50, 0x8b, 0x44, 0xbb, 0xca, 0xc4, 0xe7, 0x67, 0x58,
+ 0x65, 0x02, 0xaa, 0x97, 0xd8, 0xba, 0x50, 0xb8, 0xb5, 0x1d, 0x2d, 0x36, 0x52, 0x15, 0x55, 0x02,
+ 0xdc, 0x09, 0x63, 0x44, 0x28, 0x5b, 0x46, 0x8b, 0x25, 0x0c, 0x44, 0x8f, 0x01, 0xb8, 0x49, 0x87,
+ 0x84, 0xf7, 0x28, 0x19, 0x88, 0xa0, 0xb6, 0x8c, 0x46, 0x24, 0x31, 0xc8, 0x00, 0x1f, 0x43, 0x27,
+ 0xe7, 0x50, 0xea, 0x87, 0x90, 0x12, 0x16, 0xb8, 0x7c, 0xfa, 0x43, 0x18, 0x8d, 0xf0, 0x09, 0x34,
+ 0xcf, 0x99, 0x35, 0x2a, 0xe3, 0xff, 0xa7, 0xd0, 0x8a, 0x20, 0x94, 0xcf, 0x09, 0xa5, 0x3e, 0x95,
+ 0x31, 0x8f, 0x06, 0xf8, 0x5f, 0x55, 0xf8, 0xf6, 0x3d, 0x75, 0xc2, 0x44, 0x19, 0x94, 0x71, 0x75,
+ 0x1b, 0x16, 0xc3, 0xd3, 0x47, 0x25, 0x31, 0xfc, 0x4c, 0x55, 0xca, 0xc5, 0x74, 0xa5, 0x44, 0xfb,
+ 0xd0, 0xf2, 0xdd, 0x7e, 0x2f, 0x9e, 0x8f, 0x9c, 0xd6, 0xf4, 0xdd, 0xbe, 0x31, 0x5d, 0x12, 0xd7,
+ 0xb2, 0xe5, 0x64, 0x2d, 0x5b, 0x87, 0x65, 0x66, 0x13, 0xd7, 0x15, 0x65, 0xa9, 0x6e, 0x44, 0x03,
+ 0x7c, 0x00, 0x6d, 0x75, 0x86, 0xb9, 0xc7, 0xb5, 0x61, 0xfd, 0xdc, 0xf1, 0xfa, 0x57, 0x84, 0x0e,
+ 0xc9, 0xa9, 0xc9, 0x4a, 0x65, 0xff, 0x0e, 0x34, 0xa6, 0x07, 0x60, 0xda, 0xc2, 0xde, 0x62, 0x18,
+ 0xf6, 0x58, 0x80, 0x9f, 0xc1, 0xc6, 0x8c, 0x26, 0x95, 0x7a, 0x77, 0x26, 0x8b, 0xae, 0x7e, 0xc3,
+ 0x10, 0xdf, 0xf8, 0xef, 0x55, 0x58, 0x8d, 0xea, 0xd5, 0xb9, 0x4f, 0x47, 0xbf, 0xe4, 0x95, 0x0f,
+ 0x79, 0x4a, 0xd2, 0x92, 0x98, 0x2b, 0x75, 0xba, 0xcc, 0x20, 0xa1, 0xb1, 0x5d, 0xef, 0x9a, 0xfa,
+ 0x43, 0x4a, 0x18, 0x2b, 0x59, 0x3a, 0xa9, 0x80, 0x4b, 0x94, 0xce, 0x48, 0xd0, 0xed, 0xe3, 0x3f,
+ 0x80, 0x9e, 0xa7, 0x4d, 0x3a, 0x70, 0x17, 0x9a, 0x8e, 0xd7, 0x9b, 0x48, 0xb1, 0x4c, 0x1c, 0x70,
+ 0xe2, 0x85, 0x91, 0xb1, 0x37, 0x1f, 0x02, 0x93, 0xd9, 0x5f, 0xcd, 0x58, 0x26, 0xe0, 0x12, 0xc6,
+ 0x46, 0x82, 0xa9, 0xb1, 0x59, 0x6d, 0x5f, 0x6a, 0xec, 0x00, 0x9e, 0xcc, 0xfe, 0x52, 0x9d, 0x53,
+ 0x7f, 0xfc, 0xce, 0x78, 0x5d, 0x32, 0x1d, 0x03, 0xea, 0x4a, 0x5b, 0xc3, 0x4f, 0xbc, 0x0f, 0xbb,
+ 0x85, 0x7a, 0x64, 0x90, 0xbb, 0xb0, 0x16, 0x2d, 0x39, 0x0d, 0xbc, 0xbe, 0x5b, 0x8a, 0xa5, 0xfd,
+ 0x08, 0xeb, 0x69, 0xa8, 0x39, 0xbf, 0x3b, 0x04, 0x90, 0xc8, 0xde, 0x33, 0xdf, 0x1b, 0x38, 0xc3,
+ 0x92, 0x71, 0x1a, 0x04, 0xae, 0xdb, 0x9b, 0x98, 0xdc, 0x9e, 0xc6, 0x29, 0x14, 0x5c, 0x9b, 0xdc,
+ 0xc6, 0xcf, 0x60, 0x2d, 0xa5, 0x66, 0x6e, 0x9d, 0x18, 0xc1, 0x7e, 0x9e, 0xb7, 0x4a, 0x3b, 0x26,
+ 0x76, 0xc0, 0x42, 0xc2, 0x01, 0x4f, 0x01, 0xcf, 0x53, 0x26, 0xa3, 0x73, 0x09, 0x28, 0x2c, 0x28,
+ 0xaf, 0x1d, 0x8b, 0x78, 0xa5, 0x0a, 0x17, 0x3e, 0x83, 0xb5, 0x14, 0x92, 0xf4, 0xc4, 0x73, 0x40,
+ 0x6e, 0x24, 0xea, 0x31, 0xdb, 0xa7, 0xbc, 0xe7, 0x99, 0xe3, 0x69, 0x99, 0x6a, 0xcb, 0x99, 0x9b,
+ 0x70, 0xe2, 0x8d, 0x39, 0x26, 0x61, 0xab, 0x76, 0x41, 0x78, 0xd7, 0x1b, 0xf8, 0x27, 0x5f, 0x83,
+ 0x58, 0xe3, 0xdf, 0x41, 0x27, 0x07, 0x4f, 0x9a, 0xf6, 0x04, 0x40, 0x31, 0x6a, 0x19, 0xa9, 0x84,
+ 0x24, 0x34, 0xe6, 0xcc, 0x74, 0xad, 0xc0, 0x35, 0x39, 0x39, 0xb3, 0x89, 0x35, 0x62, 0xc1, 0xb8,
+ 0x8c, 0x31, 0xbf, 0x81, 0x4e, 0x0e, 0x9e, 0x34, 0x46, 0x87, 0xba, 0x25, 0x65, 0xd2, 0x3b, 0xf1,
+ 0x38, 0x0c, 0xd2, 0x05, 0xe1, 0x37, 0x9e, 0x39, 0x61, 0xb6, 0x5f, 0xa6, 0x99, 0xc3, 0x3f, 0xc0,
+ 0x5a, 0x0a, 0x69, 0x4e, 0x02, 0xfd, 0xb3, 0x0a, 0xdf, 0xe5, 0x5d, 0xa0, 0xaf, 0x60, 0x46, 0xc8,
+ 0xe7, 0x6d, 0xce, 0x27, 0x3d, 0x55, 0x4d, 0x6a, 0xe1, 0xf8, 0x1d, 0x75, 0xc3, 0x6c, 0x13, 0x53,
+ 0x66, 0xc0, 0x6d, 0xc9, 0x71, 0xc5, 0xda, 0x93, 0x80, 0xdb, 0xf8, 0x7b, 0x78, 0x3a, 0xdf, 0xa4,
+ 0xe8, 0x3c, 0x47, 0xff, 0x40, 0xa2, 0x0b, 0x9f, 0xf6, 0x73, 0xd1, 0x33, 0x05, 0x7a, 0x0f, 0xed,
+ 0xd9, 0xb7, 0x03, 0xb4, 0x9b, 0xb5, 0x34, 0xf5, 0x48, 0xa1, 0xef, 0x15, 0x2f, 0x90, 0x29, 0x54,
+ 0x41, 0xb7, 0xd3, 0x9e, 0x3f, 0xf1, 0x20, 0x80, 0x92, 0x1b, 0x73, 0xdf, 0x1e, 0xf4, 0xfd, 0x39,
+ 0x2b, 0x62, 0xec, 0x17, 0x00, 0xaa, 0xc3, 0x47, 0x9d, 0xf4, 0x96, 0xc4, 0x1b, 0x83, 0xae, 0xe7,
+ 0x4d, 0xc5, 0x30, 0x7f, 0x82, 0x47, 0xe9, 0x06, 0x1d, 0x3d, 0x8e, 0x3b, 0x84, 0xbc, 0xa7, 0x02,
+ 0xfd, 0x49, 0xd1, 0x74, 0x12, 0x32, 0xdd, 0x33, 0x2b, 0xc8, 0xdc, 0xc6, 0x5c, 0x41, 0xe6, 0xb7,
+ 0xda, 0xb8, 0x82, 0xfe, 0x02, 0x28, 0xdb, 0xeb, 0xa2, 0xd8, 0x4f, 0x85, 0x4d, 0xb7, 0x8e, 0xe7,
+ 0x2d, 0x89, 0xe1, 0x2f, 0xa1, 0x99, 0xe8, 0x12, 0x51, 0xec, 0xb1, 0x6c, 0x03, 0xad, 0x6f, 0xe7,
+ 0xce, 0xc5, 0x48, 0xef, 0xa1, 0x3d, 0x7b, 0x11, 0xd5, 0x55, 0x2a, 0x68, 0x39, 0xd5, 0x55, 0x2a,
+ 0x6c, 0x22, 0x2b, 0xe8, 0x02, 0x40, 0x35, 0x56, 0x2a, 0xdc, 0x99, 0x2e, 0x4e, 0x85, 0x3b, 0xdb,
+ 0x87, 0xe1, 0xca, 0x4f, 0xd5, 0xd0, 0xc2, 0xd9, 0x46, 0x49, 0x59, 0x58, 0xd0, 0x91, 0x29, 0x0b,
+ 0x8b, 0x7a, 0xac, 0xe8, 0xb2, 0x67, 0x3a, 0x0f, 0x75, 0xd9, 0x8b, 0x3a, 0x2d, 0x75, 0xd9, 0x0b,
+ 0xdb, 0x16, 0x5c, 0x41, 0xc7, 0xb0, 0x14, 0x76, 0x17, 0x68, 0x2d, 0x5e, 0xac, 0xda, 0x15, 0x7d,
+ 0x3d, 0x2d, 0x8c, 0x37, 0xfd, 0x0c, 0xf5, 0x29, 0x4f, 0x47, 0x5b, 0xd3, 0x35, 0x33, 0xdd, 0x87,
+ 0xae, 0x65, 0x27, 0x62, 0x80, 0x37, 0xf0, 0x4d, 0x8a, 0x54, 0xa3, 0x9d, 0x58, 0x53, 0x0e, 0xab,
+ 0xd7, 0x1f, 0x17, 0xcc, 0x26, 0x53, 0x56, 0x91, 0x5d, 0x15, 0xc3, 0x0c, 0x15, 0x57, 0x31, 0xcc,
+ 0xe1, 0xc6, 0x22, 0x19, 0xb2, 0x7c, 0x55, 0x25, 0x43, 0x21, 0x73, 0x56, 0xc9, 0x50, 0x4c, 0x77,
+ 0xa7, 0xf0, 0xb3, 0x0c, 0x33, 0x09, 0x5f, 0xc0, 0x75, 0x93, 0xf0, 0x45, 0x04, 0x15, 0x57, 0x90,
+ 0x9b, 0x7d, 0x7a, 0x91, 0xcc, 0x10, 0x7d, 0x5f, 0x94, 0x07, 0x69, 0x8a, 0xaa, 0xff, 0xea, 0x7f,
+ 0xae, 0x8b, 0xb5, 0x5d, 0x41, 0x2b, 0xc9, 0x0c, 0xd1, 0x76, 0x7a, 0x6b, 0x8a, 0x61, 0xe9, 0x3b,
+ 0xf9, 0x93, 0x89, 0xe4, 0xf9, 0x08, 0x7a, 0x31, 0x77, 0x42, 0x3f, 0xcc, 0xb3, 0x2b, 0xad, 0xea,
+ 0xc7, 0x2f, 0x59, 0x3a, 0x55, 0x7c, 0x50, 0x0d, 0x2b, 0x54, 0x82, 0x4e, 0xaa, 0x0a, 0x95, 0xa5,
+ 0xb2, 0xaa, 0x42, 0xe5, 0xf0, 0x4f, 0x59, 0xeb, 0x14, 0x1d, 0x4b, 0xd4, 0xba, 0x0c, 0xdb, 0x4b,
+ 0xd4, 0xba, 0x2c, 0x7f, 0xc3, 0x15, 0xf4, 0x67, 0xf1, 0xb0, 0x95, 0xe6, 0x50, 0x28, 0xf9, 0xbe,
+ 0x94, 0x4b, 0xd7, 0x54, 0xc2, 0x17, 0x12, 0x30, 0xe1, 0xea, 0x5b, 0x58, 0xcd, 0x90, 0x22, 0x85,
+ 0x5e, 0xc4, 0xbf, 0x14, 0x7a, 0x21, 0xa3, 0xc2, 0x15, 0xf4, 0x7b, 0xa8, 0xc9, 0x57, 0x63, 0xb4,
+ 0x19, 0xaf, 0x4f, 0x3d, 0x46, 0xeb, 0x5b, 0x19, 0x79, 0xbc, 0xfb, 0x25, 0x34, 0x13, 0x5c, 0x09,
+ 0x25, 0x0b, 0xee, 0x0c, 0x07, 0x52, 0x1e, 0xcc, 0x21, 0x57, 0xe2, 0x94, 0x7f, 0x85, 0x9d, 0x79,
+ 0xc4, 0x05, 0x3d, 0x9b, 0x77, 0x4f, 0x66, 0xb5, 0x3d, 0xff, 0xb2, 0xc5, 0x53, 0xf5, 0x77, 0x2b,
+ 0xe2, 0x9f, 0x98, 0xe3, 0xff, 0x06, 0x00, 0x00, 0xff, 0xff, 0x04, 0x94, 0xb1, 0xc3, 0xbb, 0x19,
+ 0x00, 0x00,
}
diff --git a/vendor/vendor.json b/vendor/vendor.json
index 9dcb37b0e..ae158fcd5 100644
--- a/vendor/vendor.json
+++ b/vendor/vendor.json
@@ -201,12 +201,12 @@
"revisionTime": "2017-12-31T12:27:32Z"
},
{
- "checksumSHA1": "MVrf0BwLEpU3uGHmYOrIWVoJGCk=",
+ "checksumSHA1": "B+FjK1qIgELTplSb/hl3ZHGHjl0=",
"path": "gitlab.com/gitlab-org/gitaly-proto/go",
- "revision": "00d4006669b30b6a0fe66aaa9fa72285d18616fb",
- "revisionTime": "2018-04-03T14:25:38Z",
- "version": "v0.95.0",
- "versionExact": "v0.95.0"
+ "revision": "8680efed3cdd68f3a2d4ccd56fb6684a6761d59e",
+ "revisionTime": "2018-04-05T16:15:56Z",
+ "version": "v0.96.0",
+ "versionExact": "v0.96.0"
},
{
"checksumSHA1": "nqWNlnMmVpt628zzvyo6Yv2CX5Q=",