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:
authorJacob Vosmaer (GitLab) <jacob@gitlab.com>2017-11-13 18:42:07 +0300
committerJacob Vosmaer (GitLab) <jacob@gitlab.com>2017-11-13 18:42:07 +0300
commitd6ffcc3fcff37653353d30877c67f6ac04f37b25 (patch)
tree89a5060ec6691b3ef085434254c86d4eae565d12
parentc757efaaa0267ec823ddc0858cf6ec930e93a074 (diff)
parent8542d85d27d3c8fb381179b235a04a10ece6a4c2 (diff)
Merge branch 'fetch-source-branch' into 'master'
Implement RepositoryService::FetchSourceBranch Closes #706 See merge request gitlab-org/gitaly!434
-rw-r--r--CHANGELOG.md5
-rwxr-xr-x_support/test-boot-time1
-rw-r--r--config.toml.example3
-rw-r--r--doc/configuration/README.md2
-rw-r--r--internal/config/config.go15
-rw-r--r--internal/rubyserver/proxy.go19
-rw-r--r--internal/rubyserver/proxy_test.go46
-rw-r--r--internal/rubyserver/rubyserver.go7
-rw-r--r--internal/rubyserver/rubyserver_test.go2
-rw-r--r--internal/rubyserver/testhelper_test.go22
-rw-r--r--internal/server/auth/auth.go (renamed from internal/server/auth.go)8
-rw-r--r--internal/server/server.go5
-rw-r--r--internal/service/repository/.gitignore1
-rw-r--r--internal/service/repository/fetch.go14
-rw-r--r--internal/service/repository/fetch_test.go236
-rw-r--r--internal/service/repository/testhelper_test.go48
-rw-r--r--ruby/lib/gitaly_server.rb6
-rw-r--r--ruby/lib/gitaly_server/client.rb30
-rw-r--r--ruby/lib/gitaly_server/repository_service.rb10
-rw-r--r--ruby/lib/gitlab/git.rb12
-rw-r--r--ruby/lib/gitlab/git/gitaly_remote_repository.rb67
-rw-r--r--ruby/lib/gitlab/gitaly_client.rb7
22 files changed, 540 insertions, 26 deletions
diff --git a/CHANGELOG.md b/CHANGELOG.md
index 9d3d71e5d..d22687244 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -1,5 +1,10 @@
# Gitaly changelog
+UNRELEASED
+
+- Implement RepositoryService::FetchSourceBranch
+ https://gitlab.com/gitlab-org/gitaly/merge_requests/434
+
v0.53.0
- Update vendored gitlab_git to f7537ce03a29b
diff --git a/_support/test-boot-time b/_support/test-boot-time
index 8b4099194..629c62a14 100755
--- a/_support/test-boot-time
+++ b/_support/test-boot-time
@@ -13,6 +13,7 @@ def main(gitaly_dir)
File.write('config.toml', <<~CONFIG
socket_path = "#{ADDR}"
+ bin_dir = "#{gitaly_dir}"
[[storage]]
name = "default"
diff --git a/config.toml.example b/config.toml.example
index 69c813c4f..2ad2fd4ef 100644
--- a/config.toml.example
+++ b/config.toml.example
@@ -2,6 +2,9 @@
socket_path = "/home/git/gitlab/tmp/sockets/private/gitaly.socket"
+# The directory where Gitaly's executables are stored
+bin_dir = "/home/git/gitaly"
+
# # Optional: listen on a TCP socket. This is insecure (no authentication)
# listen_addr = "localhost:9999"
#
diff --git a/doc/configuration/README.md b/doc/configuration/README.md
index 9803565e1..2aa1940f8 100644
--- a/doc/configuration/README.md
+++ b/doc/configuration/README.md
@@ -21,6 +21,7 @@ gitaly /path/to/config.toml
```toml
socket_path = "/path/to/gitaly.sock"
listen_addr = ":8081"
+bin_dir = "/path/to/gitaly-executables"
prometheus_listen_addr = ":9236"
[auth]
@@ -41,6 +42,7 @@ name = "my_shard"
|----|----|--------|-----|
|socket_path|string|see notes|A path which gitaly should open a Unix socket. Required unless listen_addr is set|
|listen_addr|string|see notes|TCP address for Gitaly to listen on (See #GITALY_LISTEN_ADDR). Required unless socket_path is set|
+|bin_dir|string|yes|Directory containing Gitaly's executables|
|prometheus_listen_addr|string|no|TCP listen address for Prometheus metrics. If not set, no Prometheus listener is started|
|storage|array|yes|An array of storage shards|
diff --git a/internal/config/config.go b/internal/config/config.go
index d1eac0ad5..681525230 100644
--- a/internal/config/config.go
+++ b/internal/config/config.go
@@ -6,6 +6,7 @@ import (
"os"
"os/exec"
"path"
+ "path/filepath"
log "github.com/sirupsen/logrus"
@@ -22,6 +23,7 @@ type config struct {
SocketPath string `toml:"socket_path" split_words:"true"`
ListenAddr string `toml:"listen_addr" split_words:"true"`
PrometheusListenAddr string `toml:"prometheus_listen_addr" split_words:"true"`
+ BinDir string `toml:"bin_dir"`
Git Git `toml:"git" envconfig:"git"`
Storages []Storage `toml:"storage" envconfig:"storage"`
Logging Logging `toml:"logging" envconfig:"logging"`
@@ -90,6 +92,7 @@ func Validate() error {
SetGitPath(),
validateShell(),
validateRuby(),
+ validateBinDir(),
} {
if err != nil {
return err
@@ -198,3 +201,15 @@ func StoragePath(storageName string) (string, bool) {
func GitlabShellBinPath() string {
return path.Join(Config.GitlabShell.Dir, "bin")
}
+
+func validateBinDir() error {
+ if err := validateIsDirectory(Config.BinDir, "bin_dir"); err != nil {
+ log.WithError(err).Warn("Gitaly bin directory is not configured")
+ // TODO this must become a fatal error
+ return nil
+ }
+
+ var err error
+ Config.BinDir, err = filepath.Abs(Config.BinDir)
+ return err
+}
diff --git a/internal/rubyserver/proxy.go b/internal/rubyserver/proxy.go
index 503d9ac8f..9cb8289e8 100644
--- a/internal/rubyserver/proxy.go
+++ b/internal/rubyserver/proxy.go
@@ -12,6 +12,16 @@ import (
"google.golang.org/grpc/metadata"
)
+// ProxyHeaderWhitelist is the list of http/2 headers that will be
+// forwarded as-is to gitaly-ruby.
+var ProxyHeaderWhitelist = []string{"gitaly-servers"}
+
+const (
+ repoPathHeader = "gitaly-repo-path"
+ glRepositoryHeader = "gitaly-gl-repository"
+ repoAltDirsHeader = "gitaly-repo-alt-dirs"
+)
+
// SetHeaders adds headers that tell gitaly-ruby the full path to the repository.
func SetHeaders(ctx context.Context, repo *pb.Repository) (context.Context, error) {
repoPath, err := helper.GetPath(repo)
@@ -28,6 +38,15 @@ func SetHeaders(ctx context.Context, repo *pb.Repository) (context.Context, erro
glRepositoryHeader, repo.GlRepository,
repoAltDirsHeader, repoAltDirsCombined,
)
+
+ if inMD, ok := metadata.FromIncomingContext(ctx); ok {
+ for _, header := range ProxyHeaderWhitelist {
+ for _, v := range inMD[header] {
+ md = metadata.Join(md, metadata.Pairs(header, v))
+ }
+ }
+ }
+
newCtx := metadata.NewOutgoingContext(ctx, md)
return newCtx, nil
}
diff --git a/internal/rubyserver/proxy_test.go b/internal/rubyserver/proxy_test.go
new file mode 100644
index 000000000..397cafe09
--- /dev/null
+++ b/internal/rubyserver/proxy_test.go
@@ -0,0 +1,46 @@
+package rubyserver
+
+import (
+ "testing"
+
+ "gitlab.com/gitlab-org/gitaly/internal/testhelper"
+
+ "github.com/stretchr/testify/require"
+ "google.golang.org/grpc/metadata"
+)
+
+func TestSetHeadersBlocksUnknownMetadata(t *testing.T) {
+ ctx, cancel := testhelper.Context()
+ defer cancel()
+
+ otherKey := "unknown-key"
+ otherValue := "test-value"
+ inCtx := metadata.NewIncomingContext(ctx, metadata.Pairs(otherKey, otherValue))
+
+ outCtx, err := SetHeaders(inCtx, testRepo)
+ require.NoError(t, err)
+
+ outMd, ok := metadata.FromOutgoingContext(outCtx)
+ require.True(t, ok, "outgoing context should have metadata")
+
+ _, ok = outMd[otherKey]
+ require.False(t, ok, "outgoing MD should not contain non-whitelisted key")
+}
+
+func TestSetHeadersPreservesWhitelistedMetadata(t *testing.T) {
+ ctx, cancel := testhelper.Context()
+ defer cancel()
+
+ key := "gitaly-servers"
+ require.Contains(t, ProxyHeaderWhitelist, key, "sanity check")
+ value := "test-value"
+ inCtx := metadata.NewIncomingContext(ctx, metadata.Pairs(key, value))
+
+ outCtx, err := SetHeaders(inCtx, testRepo)
+ require.NoError(t, err)
+
+ outMd, ok := metadata.FromOutgoingContext(outCtx)
+ require.True(t, ok, "outgoing context should have metadata")
+
+ require.Equal(t, []string{value}, outMd[key], "outgoing MD should contain whitelisted key")
+}
diff --git a/internal/rubyserver/rubyserver.go b/internal/rubyserver/rubyserver.go
index 9d0466115..3615ddeeb 100644
--- a/internal/rubyserver/rubyserver.go
+++ b/internal/rubyserver/rubyserver.go
@@ -23,12 +23,6 @@ import (
"google.golang.org/grpc"
)
-const (
- repoPathHeader = "gitaly-repo-path"
- glRepositoryHeader = "gitaly-gl-repository"
- repoAltDirsHeader = "gitaly-repo-alt-dirs"
-)
-
var (
socketDir string
@@ -101,6 +95,7 @@ func Start() (*Server, error) {
"GITALY_RUBY_GIT_BIN_PATH=" + command.GitPath(),
fmt.Sprintf("GITALY_RUBY_WRITE_BUFFER_SIZE=%d", streamio.WriteBufferSize),
"GITALY_RUBY_GITLAB_SHELL_PATH=" + cfg.GitlabShell.Dir,
+ "GITALY_RUBY_GITALY_BIN_DIR=" + cfg.BinDir,
}
args := []string{"bundle", "exec", "bin/gitaly-ruby", fmt.Sprintf("%d", os.Getpid()), socketPath()}
diff --git a/internal/rubyserver/rubyserver_test.go b/internal/rubyserver/rubyserver_test.go
index c39782d3b..77943731e 100644
--- a/internal/rubyserver/rubyserver_test.go
+++ b/internal/rubyserver/rubyserver_test.go
@@ -23,8 +23,6 @@ func TestStopSafe(t *testing.T) {
}
func TestSetHeaders(t *testing.T) {
- testRepo := testhelper.TestRepository()
-
testCases := []struct {
repo *pb.Repository
errType codes.Code
diff --git a/internal/rubyserver/testhelper_test.go b/internal/rubyserver/testhelper_test.go
new file mode 100644
index 000000000..cc83a7923
--- /dev/null
+++ b/internal/rubyserver/testhelper_test.go
@@ -0,0 +1,22 @@
+package rubyserver
+
+import (
+ "os"
+ "testing"
+
+ "gitlab.com/gitlab-org/gitaly/internal/testhelper"
+)
+
+var (
+ testRepo = testhelper.TestRepository()
+)
+
+func TestMain(m *testing.M) {
+ os.Exit(testMain(m))
+}
+
+func testMain(m *testing.M) int {
+ defer testhelper.MustHaveNoChildProcess()
+
+ return m.Run()
+}
diff --git a/internal/server/auth.go b/internal/server/auth/auth.go
index d59abc5bc..0498756b5 100644
--- a/internal/server/auth.go
+++ b/internal/server/auth/auth.go
@@ -1,4 +1,4 @@
-package server
+package auth
import (
"encoding/base64"
@@ -26,11 +26,13 @@ func init() {
prometheus.MustRegister(authCount)
}
-func authStreamServerInterceptor() grpc.StreamServerInterceptor {
+// StreamServerInterceptor checks for Gitaly bearer tokens.
+func StreamServerInterceptor() grpc.StreamServerInterceptor {
return grpc_auth.StreamServerInterceptor(check)
}
-func authUnaryServerInterceptor() grpc.UnaryServerInterceptor {
+// UnaryServerInterceptor checks for Gitaly bearer tokens.
+func UnaryServerInterceptor() grpc.UnaryServerInterceptor {
return grpc_auth.UnaryServerInterceptor(check)
}
diff --git a/internal/server/server.go b/internal/server/server.go
index 721b3f95e..4fa980be4 100644
--- a/internal/server/server.go
+++ b/internal/server/server.go
@@ -10,6 +10,7 @@ import (
"gitlab.com/gitlab-org/gitaly/internal/middleware/panichandler"
"gitlab.com/gitlab-org/gitaly/internal/middleware/sentryhandler"
"gitlab.com/gitlab-org/gitaly/internal/rubyserver"
+ "gitlab.com/gitlab-org/gitaly/internal/server/auth"
"gitlab.com/gitlab-org/gitaly/internal/service"
"github.com/grpc-ecosystem/go-grpc-middleware"
@@ -41,7 +42,7 @@ func New(rubyServer *rubyserver.Server) *grpc.Server {
sentryhandler.StreamLogHandler,
cancelhandler.Stream, // Should be below LogHandler
lh.StreamInterceptor(),
- authStreamServerInterceptor(),
+ auth.StreamServerInterceptor(),
// Panic handler should remain last so that application panics will be
// converted to errors and logged
panichandler.StreamPanicHandler,
@@ -54,7 +55,7 @@ func New(rubyServer *rubyserver.Server) *grpc.Server {
sentryhandler.UnaryLogHandler,
cancelhandler.Unary, // Should be below LogHandler
lh.UnaryInterceptor(),
- authUnaryServerInterceptor(),
+ auth.UnaryServerInterceptor(),
// Panic handler should remain last so that application panics will be
// converted to errors and logged
panichandler.UnaryPanicHandler,
diff --git a/internal/service/repository/.gitignore b/internal/service/repository/.gitignore
new file mode 100644
index 000000000..9cd02c87a
--- /dev/null
+++ b/internal/service/repository/.gitignore
@@ -0,0 +1 @@
+/testdata/gitaly-libexec
diff --git a/internal/service/repository/fetch.go b/internal/service/repository/fetch.go
index 7039cdb92..bd5dfb499 100644
--- a/internal/service/repository/fetch.go
+++ b/internal/service/repository/fetch.go
@@ -2,11 +2,21 @@ package repository
import (
pb "gitlab.com/gitlab-org/gitaly-proto/go"
- "gitlab.com/gitlab-org/gitaly/internal/helper"
+ "gitlab.com/gitlab-org/gitaly/internal/rubyserver"
"golang.org/x/net/context"
)
func (s *server) FetchSourceBranch(ctx context.Context, req *pb.FetchSourceBranchRequest) (*pb.FetchSourceBranchResponse, error) {
- return nil, helper.Unimplemented
+ client, err := s.RepositoryServiceClient(ctx)
+ if err != nil {
+ return nil, err
+ }
+
+ clientCtx, err := rubyserver.SetHeaders(ctx, req.GetRepository())
+ if err != nil {
+ return nil, err
+ }
+
+ return client.FetchSourceBranch(clientCtx, req)
}
diff --git a/internal/service/repository/fetch_test.go b/internal/service/repository/fetch_test.go
new file mode 100644
index 000000000..c79d3d517
--- /dev/null
+++ b/internal/service/repository/fetch_test.go
@@ -0,0 +1,236 @@
+package repository_test
+
+import (
+ "encoding/base64"
+ "encoding/json"
+ "net"
+ "os"
+ "strings"
+ "testing"
+ "time"
+
+ pb "gitlab.com/gitlab-org/gitaly-proto/go"
+ gitLog "gitlab.com/gitlab-org/gitaly/internal/git/log"
+ "gitlab.com/gitlab-org/gitaly/internal/helper"
+ serverPkg "gitlab.com/gitlab-org/gitaly/internal/server"
+ "gitlab.com/gitlab-org/gitaly/internal/service/repository"
+ "gitlab.com/gitlab-org/gitaly/internal/testhelper"
+
+ "github.com/stretchr/testify/require"
+ "google.golang.org/grpc"
+ "google.golang.org/grpc/codes"
+ healthpb "google.golang.org/grpc/health/grpc_health_v1"
+ "google.golang.org/grpc/metadata"
+)
+
+func TestFetchSourceBranchSourceRepositorySuccess(t *testing.T) {
+ server, serverSocketPath := runFullServer(t)
+ defer server.Stop()
+
+ client, conn := repository.NewRepositoryClient(t, serverSocketPath)
+ defer conn.Close()
+
+ ctxOuter, cancel := testhelper.Context()
+ defer cancel()
+
+ md, err := gitalyServersMetadata(serverSocketPath)
+ require.NoError(t, err)
+ ctx := metadata.NewOutgoingContext(ctxOuter, md)
+
+ targetRepo, _ := newTestRepo(t, "fetch-source-target.git")
+ sourceRepo, sourcePath := newTestRepo(t, "fetch-source-source.git")
+
+ sourceBranch := "fetch-source-branch-test-branch"
+ newCommitID := createCommit(t, sourcePath, sourceBranch)
+
+ targetRef := "refs/tmp/fetch-source-branch-test"
+ req := &pb.FetchSourceBranchRequest{
+ Repository: targetRepo,
+ SourceRepository: sourceRepo,
+ SourceBranch: []byte(sourceBranch),
+ TargetRef: []byte(targetRef),
+ }
+
+ resp, err := client.FetchSourceBranch(ctx, req)
+ require.NoError(t, err)
+ require.True(t, resp.Result, "response.Result should be true")
+
+ fetchedCommit, err := gitLog.GetCommit(ctx, targetRepo, targetRef, "")
+ require.NoError(t, err)
+ require.Equal(t, newCommitID, fetchedCommit.GetId())
+}
+
+func TestFetchSourceBranchSameRepositorySuccess(t *testing.T) {
+ server, serverSocketPath := runFullServer(t)
+ defer server.Stop()
+
+ client, conn := repository.NewRepositoryClient(t, serverSocketPath)
+ defer conn.Close()
+
+ ctxOuter, cancel := testhelper.Context()
+ defer cancel()
+
+ md, err := gitalyServersMetadata(serverSocketPath)
+ require.NoError(t, err)
+ ctx := metadata.NewOutgoingContext(ctxOuter, md)
+
+ repo, repoPath := newTestRepo(t, "fetch-source-source.git")
+
+ sourceBranch := "fetch-source-branch-test-branch"
+ newCommitID := createCommit(t, repoPath, sourceBranch)
+
+ targetRef := "refs/tmp/fetch-source-branch-test"
+ req := &pb.FetchSourceBranchRequest{
+ Repository: repo,
+ SourceRepository: repo,
+ SourceBranch: []byte(sourceBranch),
+ TargetRef: []byte(targetRef),
+ }
+
+ resp, err := client.FetchSourceBranch(ctx, req)
+ require.NoError(t, err)
+ require.True(t, resp.Result, "response.Result should be true")
+
+ fetchedCommit, err := gitLog.GetCommit(ctx, repo, targetRef, "")
+ require.NoError(t, err)
+ require.Equal(t, newCommitID, fetchedCommit.GetId())
+}
+
+func TestFetchSourceBranchBranchNotFound(t *testing.T) {
+ server, serverSocketPath := runFullServer(t)
+ defer server.Stop()
+
+ client, conn := repository.NewRepositoryClient(t, serverSocketPath)
+ defer conn.Close()
+
+ ctxOuter, cancel := testhelper.Context()
+ defer cancel()
+
+ md, err := gitalyServersMetadata(serverSocketPath)
+ require.NoError(t, err)
+ ctx := metadata.NewOutgoingContext(ctxOuter, md)
+
+ targetRepo, _ := newTestRepo(t, "fetch-source-target.git")
+ sourceRepo, _ := newTestRepo(t, "fetch-source-source.git")
+
+ sourceBranch := "does-not-exist"
+ targetRef := "refs/tmp/fetch-source-branch-test"
+
+ testCases := []struct {
+ req *pb.FetchSourceBranchRequest
+ desc string
+ }{
+ {
+ desc: "target different from source",
+ req: &pb.FetchSourceBranchRequest{
+ Repository: targetRepo,
+ SourceRepository: sourceRepo,
+ SourceBranch: []byte(sourceBranch),
+ TargetRef: []byte(targetRef),
+ },
+ },
+ {
+ desc: "target same as source",
+ req: &pb.FetchSourceBranchRequest{
+ Repository: sourceRepo,
+ SourceRepository: sourceRepo,
+ SourceBranch: []byte(sourceBranch),
+ TargetRef: []byte(targetRef),
+ },
+ },
+ }
+
+ for _, tc := range testCases {
+ t.Run(tc.desc, func(t *testing.T) {
+ resp, err := client.FetchSourceBranch(ctx, tc.req)
+ require.NoError(t, err)
+ require.False(t, resp.Result, "response.Result should be false")
+ })
+ }
+}
+
+func TestFetchFullServerRequiresAuthentication(t *testing.T) {
+ // The purpose of this test is to ensure that the server started by
+ // 'runFullServer' requires authentication. The RPC under test in this
+ // file (FetchSourceBranch) makes calls to a "remote" Gitaly server and
+ // we want to be sure that authentication is handled correctly. If the
+ // tests in this file were using a server without authentication we could
+ // not be confident that authentication is done right.
+ server, serverSocketPath := runFullServer(t)
+ defer server.Stop()
+
+ connOpts := []grpc.DialOption{
+ grpc.WithInsecure(),
+ grpc.WithDialer(func(addr string, timeout time.Duration) (net.Conn, error) {
+ return net.DialTimeout("unix", addr, timeout)
+ }),
+ }
+
+ conn, err := grpc.Dial(serverSocketPath, connOpts...)
+ require.NoError(t, err)
+ defer conn.Close()
+
+ ctx, cancel := testhelper.Context()
+ defer cancel()
+
+ client := healthpb.NewHealthClient(conn)
+ _, err = client.Check(ctx, &healthpb.HealthCheckRequest{})
+ testhelper.AssertGrpcError(t, err, codes.Unauthenticated, "")
+}
+
+func gitalyServersMetadata(serverSocketPath string) (metadata.MD, error) {
+ gitalyServers := map[string]map[string]string{
+ "default": {
+ "address": "unix:" + serverSocketPath,
+ "token": repository.AuthToken,
+ },
+ }
+
+ gitalyServersJSON, err := json.Marshal(gitalyServers)
+ if err != nil {
+ return nil, err
+ }
+
+ return metadata.Pairs("gitaly-servers", base64.StdEncoding.EncodeToString(gitalyServersJSON)), nil
+}
+
+func createCommit(t *testing.T, repoPath string, branchName string) string {
+ // The ID of an arbitrary commit known to exist in the test repository.
+ parentID := "1a0b36b3cdad1d2ee32457c102a8c0b7056fa863"
+
+ // Use 'commit-tree' instead of 'commit' because we are in a bare
+ // repository. What we do here is the same as "commit -m message
+ // --allow-empty".
+ commitArgs := []string{"-C", repoPath, "commit-tree", "-m", "message", "-p", parentID, parentID + "^{tree}"}
+ newCommit := testhelper.MustRunCommand(t, nil, "git", commitArgs...)
+ newCommitID := strings.TrimSpace(string(newCommit))
+
+ testhelper.MustRunCommand(t, nil, "git", "-C", repoPath, "update-ref", "refs/heads/"+branchName, newCommitID)
+ return newCommitID
+}
+
+func newTestRepo(t *testing.T, relativePath string) (*pb.Repository, string) {
+ repo := &pb.Repository{StorageName: "default", RelativePath: relativePath}
+
+ repoPath, err := helper.GetPath(repo)
+ require.NoError(t, err)
+
+ require.NoError(t, os.RemoveAll(repoPath))
+ testhelper.MustRunCommand(t, nil, "git", "clone", "--bare", repository.TestRepoPath, repoPath)
+
+ return repo, repoPath
+}
+
+func runFullServer(t *testing.T) (*grpc.Server, string) {
+ server := serverPkg.New(repository.RubyServer)
+ serverSocketPath := testhelper.GetTemporaryGitalySocketFileName()
+
+ listener, err := net.Listen("unix", serverSocketPath)
+ if err != nil {
+ t.Fatal(err)
+ }
+
+ go server.Serve(listener)
+
+ return server, serverSocketPath
+}
diff --git a/internal/service/repository/testhelper_test.go b/internal/service/repository/testhelper_test.go
index 0851f9e8c..e4e9d7f87 100644
--- a/internal/service/repository/testhelper_test.go
+++ b/internal/service/repository/testhelper_test.go
@@ -4,15 +4,20 @@ import (
"log"
"net"
"os"
+ "path"
"path/filepath"
"testing"
"time"
- "github.com/stretchr/testify/assert"
pb "gitlab.com/gitlab-org/gitaly-proto/go"
+ gitalyauth "gitlab.com/gitlab-org/gitaly/auth"
"gitlab.com/gitlab-org/gitaly/internal/config"
+ "gitlab.com/gitlab-org/gitaly/internal/helper"
"gitlab.com/gitlab-org/gitaly/internal/rubyserver"
+ "gitlab.com/gitlab-org/gitaly/internal/server/auth"
"gitlab.com/gitlab-org/gitaly/internal/testhelper"
+
+ "github.com/stretchr/testify/assert"
"google.golang.org/grpc"
"google.golang.org/grpc/reflection"
)
@@ -21,9 +26,11 @@ import (
const testTimeString = "200601021504.05"
var (
- testTime = time.Date(2006, 1, 2, 15, 4, 5, 0, time.UTC)
- testRepo = testhelper.TestRepository()
- rubyServer *rubyserver.Server
+ testTime = time.Date(2006, 1, 2, 15, 4, 5, 0, time.UTC)
+ testRepo = testhelper.TestRepository()
+ TestRepoPath string
+ RubyServer *rubyserver.Server
+ AuthToken = "the-secret-token"
)
func newRepositoryClient(t *testing.T, serverSocketPath string) (pb.RepositoryServiceClient, *grpc.ClientConn) {
@@ -32,6 +39,7 @@ func newRepositoryClient(t *testing.T, serverSocketPath string) (pb.RepositorySe
grpc.WithDialer(func(addr string, timeout time.Duration) (net.Conn, error) {
return net.DialTimeout("unix", addr, timeout)
}),
+ grpc.WithPerRPCCredentials(gitalyauth.RPCCredentials(AuthToken)),
}
conn, err := grpc.Dial(serverSocketPath, connOpts...)
if err != nil {
@@ -41,8 +49,13 @@ func newRepositoryClient(t *testing.T, serverSocketPath string) (pb.RepositorySe
return pb.NewRepositoryServiceClient(conn), conn
}
+var NewRepositoryClient = newRepositoryClient
+
func runRepoServer(t *testing.T) (*grpc.Server, string) {
- server := testhelper.NewTestGrpcServer(t, nil, nil)
+ streamInt := []grpc.StreamServerInterceptor{auth.StreamServerInterceptor()}
+ unaryInt := []grpc.UnaryServerInterceptor{auth.UnaryServerInterceptor()}
+
+ server := testhelper.NewTestGrpcServer(t, streamInt, unaryInt)
serverSocketPath := testhelper.GetTemporaryGitalySocketFileName()
listener, err := net.Listen("unix", serverSocketPath)
@@ -50,7 +63,7 @@ func runRepoServer(t *testing.T) (*grpc.Server, string) {
t.Fatal(err)
}
- pb.RegisterRepositoryServiceServer(server, NewServer(rubyServer))
+ pb.RegisterRepositoryServiceServer(server, NewServer(RubyServer))
reflection.Register(server)
go server.Serve(listener)
@@ -81,6 +94,7 @@ func testMain(m *testing.M) int {
defer testhelper.MustHaveNoChildProcess()
testhelper.ConfigureRuby()
+ config.Config.Auth = config.Auth{Token: config.Token(AuthToken)}
var err error
config.Config.GitlabShell.Dir, err = filepath.Abs("testdata/gitlab-shell")
@@ -88,11 +102,29 @@ func testMain(m *testing.M) int {
log.Fatal(err)
}
- rubyServer, err = rubyserver.Start()
+ config.Config.BinDir, err = filepath.Abs("testdata/gitaly-libexec")
+ if err != nil {
+ log.Fatal(err)
+ }
+
+ goBuildArgs := []string{
+ "build",
+ "-o",
+ path.Join(config.Config.BinDir, "gitaly-ssh"),
+ "gitlab.com/gitlab-org/gitaly/cmd/gitaly-ssh",
+ }
+ testhelper.MustRunCommand(nil, nil, "go", goBuildArgs...)
+
+ RubyServer, err = rubyserver.Start()
+ if err != nil {
+ log.Fatal(err)
+ }
+ defer RubyServer.Stop()
+
+ TestRepoPath, err = helper.GetPath(testRepo)
if err != nil {
log.Fatal(err)
}
- defer rubyServer.Stop()
return m.Run()
}
diff --git a/ruby/lib/gitaly_server.rb b/ruby/lib/gitaly_server.rb
index 32e47e4c1..5579024cc 100644
--- a/ruby/lib/gitaly_server.rb
+++ b/ruby/lib/gitaly_server.rb
@@ -2,6 +2,7 @@ require 'gitaly'
require_relative 'gitlab/git.rb'
+require_relative 'gitaly_server/client.rb'
require_relative 'gitaly_server/utils.rb'
require_relative 'gitaly_server/commit_service.rb'
require_relative 'gitaly_server/diff_service.rb'
@@ -14,6 +15,7 @@ module GitalyServer
REPO_PATH_HEADER = 'gitaly-repo-path'.freeze
GL_REPOSITORY_HEADER = 'gitaly-gl-repository'.freeze
REPO_ALT_DIRS_HEADER = 'gitaly-repo-alt-dirs'.freeze
+ GITALY_SERVERS_HEADER = 'gitaly-servers'.freeze
def self.repo_path(call)
call.metadata.fetch(REPO_PATH_HEADER)
@@ -27,6 +29,10 @@ module GitalyServer
call.metadata.fetch(REPO_ALT_DIRS_HEADER)
end
+ def self.client(call)
+ Client.new(call.metadata[GITALY_SERVERS_HEADER])
+ end
+
def self.register_handlers(server)
server.handle(CommitService.new)
server.handle(DiffService.new)
diff --git a/ruby/lib/gitaly_server/client.rb b/ruby/lib/gitaly_server/client.rb
new file mode 100644
index 000000000..1f5e4ff62
--- /dev/null
+++ b/ruby/lib/gitaly_server/client.rb
@@ -0,0 +1,30 @@
+require 'base64'
+require 'json'
+
+module GitalyServer
+ class Client
+ ServerLookupError = Class.new(StandardError)
+
+ def initialize(encoded_servers)
+ @servers = encoded_servers.present? ? JSON.parse(Base64.strict_decode64(encoded_servers)) : {}
+ end
+
+ def token(storage)
+ server(storage)['token']
+ end
+
+ def address(storage)
+ server(storage)['address']
+ end
+
+ private
+
+ def server(storage)
+ unless @servers.has_key?(storage)
+ raise ServerLookupError.new("cannot find gitaly address for storage #{storage.inspect}")
+ end
+
+ @servers[storage]
+ end
+ end
+end
diff --git a/ruby/lib/gitaly_server/repository_service.rb b/ruby/lib/gitaly_server/repository_service.rb
index 41ca6bd10..725714350 100644
--- a/ruby/lib/gitaly_server/repository_service.rb
+++ b/ruby/lib/gitaly_server/repository_service.rb
@@ -17,5 +17,15 @@ module GitalyServer
Gitaly::HasLocalBranchesResponse.new(value: repo.has_local_branches?)
end
+
+ def fetch_source_branch(request, call)
+ bridge_exceptions do
+ source_repository = Gitlab::Git::GitalyRemoteRepository.new(request.source_repository, call)
+ repository = Gitlab::Git::Repository.from_gitaly(request.repository, call)
+ result = repository.fetch_source_branch!(source_repository, request.source_branch, request.target_ref)
+
+ Gitaly::FetchSourceBranchResponse.new(result: result)
+ end
+ end
end
end
diff --git a/ruby/lib/gitlab/git.rb b/ruby/lib/gitlab/git.rb
index b44771c54..d1d3df5e7 100644
--- a/ruby/lib/gitlab/git.rb
+++ b/ruby/lib/gitlab/git.rb
@@ -33,6 +33,8 @@ Dir["#{dir}/git/**/*.rb"].sort.each do |ruby_file|
require_relative ruby_file.sub(dir, File.join(vendor_gitlab_git, 'lib/gitlab/')).sub(%r{^/*}, '')
end
+require_relative 'git/gitaly_remote_repository.rb'
+
module Gitlab
# Config lets Gitlab::Git do mock config lookups.
class Config
@@ -56,6 +58,12 @@ module Gitlab
end
end
+ class Gitaly
+ def client_path
+ ENV['GITALY_RUBY_GITALY_BIN_DIR']
+ end
+ end
+
def git
Git.new
end
@@ -63,6 +71,10 @@ module Gitlab
def gitlab_shell
GitlabShell.new
end
+
+ def gitaly
+ Gitaly.new
+ end
end
def self.config
diff --git a/ruby/lib/gitlab/git/gitaly_remote_repository.rb b/ruby/lib/gitlab/git/gitaly_remote_repository.rb
new file mode 100644
index 000000000..5fb8f2371
--- /dev/null
+++ b/ruby/lib/gitlab/git/gitaly_remote_repository.rb
@@ -0,0 +1,67 @@
+module Gitlab
+ module Git
+ class GitalyRemoteRepository < RemoteRepository
+ CLIENT_NAME = 'gitaly-ruby'.freeze
+
+ attr_reader :gitaly_client
+
+ def initialize(gitaly_repository, call)
+ @gitaly_repository = gitaly_repository
+ @storage = gitaly_repository.storage_name
+ @gitaly_client = GitalyServer.client(call)
+ end
+
+ def path
+ raise 'gitaly-ruby cannot access remote repositories by path'
+ end
+
+ def empty_repo?
+ !exists? || !has_visible_content?
+ end
+
+ def commit_id(revision)
+ request = Gitaly::FindCommitRequest.new(repository: @gitaly_repository, revision: revision.b)
+ stub = Gitaly::CommitService::Stub.new(address, credentials)
+ stub.find_commit(request, request_kwargs)&.commit&.id.presence
+ end
+
+ private
+
+ def exists?
+ request = Gitaly::RepositoryExistsRequest.new(repository: @gitaly_repository)
+ stub = Gitaly::RepositoryService::Stub.new(address, credentials)
+ stub.repository_exists(request, request_kwargs).exists
+ end
+
+ def has_visible_content?
+ request = Gitaly::HasLocalBranchesRequest.new(repository: @gitaly_repository)
+ stub = Gitaly::RepositoryService::Stub.new(address, credentials)
+ stub.has_local_branches(request, request_kwargs).value
+ end
+
+ def address
+ gitaly_client.address(storage)
+ end
+
+ def credentials
+ :this_channel_is_insecure
+ end
+
+ def token
+ gitaly_client.token(storage)
+ end
+
+ def request_kwargs
+ @request_kwargs ||= begin
+ encoded_token = Base64.strict_encode64(token.to_s)
+ metadata = {
+ 'authorization' => "Bearer #{encoded_token}",
+ 'client_name' => CLIENT_NAME
+ }
+
+ { metadata: metadata }
+ end
+ end
+ end
+ end
+end
diff --git a/ruby/lib/gitlab/gitaly_client.rb b/ruby/lib/gitlab/gitaly_client.rb
index fbc8e4114..5e7b71559 100644
--- a/ruby/lib/gitlab/gitaly_client.rb
+++ b/ruby/lib/gitlab/gitaly_client.rb
@@ -5,10 +5,11 @@ module Gitlab
end
class << self
- # In case we hit a method that tries to do a Gitaly RPC, prevent this.
- # We also don't want to instrument the block.
+ # In case we hit a method that tries to do a Gitaly RPC, we want to
+ # prevent this most of the time.
def migrate(*args)
- yield false # 'false' means 'don't use gitaly for this block'
+ whitelist = [:fetch_ref]
+ yield whitelist.include?(args.first)
end
end
end