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:
authorAlejandro Rodríguez <alejorro70@gmail.com>2018-01-06 00:39:06 +0300
committerAlejandro Rodríguez <alejorro70@gmail.com>2018-01-19 01:47:46 +0300
commitb6d2b3cc894199755cf4b27b09587c4de18db2b9 (patch)
tree6d8b695e509a1522cff7f06eebae247020bf48e9
parent9206bc66f73bf20225f2c9c9e42accb7b81b6639 (diff)
Implement OperationService.UserCommitFiles
-rw-r--r--CHANGELOG.md2
-rw-r--r--internal/service/operations/commit_files.go77
-rw-r--r--internal/service/operations/commit_files_test.go283
-rw-r--r--internal/service/remote/fetch_internal_remote_test.go18
-rw-r--r--internal/testhelper/testhelper.go41
-rw-r--r--ruby/lib/gitaly_server/operations_service.rb65
-rw-r--r--ruby/lib/gitlab/git.rb1
7 files changed, 458 insertions, 29 deletions
diff --git a/CHANGELOG.md b/CHANGELOG.md
index 668f07f53..0857add54 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -2,6 +2,8 @@
UNRELEASED
+- Implement OperationService.UserCommitFiles RPC
+ https://gitlab.com/gitlab-org/gitaly/merge_requests/516
- Use grpc-go 1.9.1
https://gitlab.com/gitlab-org/gitaly/merge_requests/547
diff --git a/internal/service/operations/commit_files.go b/internal/service/operations/commit_files.go
index b2270487a..310f60a43 100644
--- a/internal/service/operations/commit_files.go
+++ b/internal/service/operations/commit_files.go
@@ -1,10 +1,81 @@
package operations
import (
+ "fmt"
+
pb "gitlab.com/gitlab-org/gitaly-proto/go"
- "gitlab.com/gitlab-org/gitaly/internal/helper"
+ "gitlab.com/gitlab-org/gitaly/internal/rubyserver"
+ "google.golang.org/grpc/codes"
+ "google.golang.org/grpc/status"
)
-func (*server) UserCommitFiles(pb.OperationService_UserCommitFilesServer) error {
- return helper.Unimplemented
+func (s *server) UserCommitFiles(stream pb.OperationService_UserCommitFilesServer) error {
+ firstRequest, err := stream.Recv()
+ if err != nil {
+ return err
+ }
+
+ header := firstRequest.GetHeader()
+ if header == nil {
+ return status.Errorf(codes.InvalidArgument, "UserCommitFiles: empty UserCommitFilesRequestHeader")
+ }
+
+ if err = validateUserCommitFilesHeader(header); err != nil {
+ return status.Errorf(codes.InvalidArgument, "UserCommitFiles: %v", err)
+ }
+
+ ctx := stream.Context()
+ client, err := s.OperationServiceClient(ctx)
+ if err != nil {
+ return err
+ }
+
+ clientCtx, err := rubyserver.SetHeaders(ctx, header.GetRepository())
+ if err != nil {
+ return err
+ }
+
+ rubyStream, err := client.UserCommitFiles(clientCtx)
+ if err != nil {
+ return err
+ }
+
+ if err := rubyStream.Send(firstRequest); err != nil {
+ return err
+ }
+
+ err = rubyserver.Proxy(func() error {
+ request, err := stream.Recv()
+ if err != nil {
+ return err
+ }
+ return rubyStream.Send(request)
+ })
+
+ if err != nil {
+ return err
+ }
+
+ response, err := rubyStream.CloseAndRecv()
+ if err != nil {
+ return err
+ }
+
+ return stream.SendAndClose(response)
+}
+
+func validateUserCommitFilesHeader(header *pb.UserCommitFilesRequestHeader) error {
+ if header.GetRepository() == nil {
+ return fmt.Errorf("empty Repository")
+ }
+ if header.GetUser() == nil {
+ return fmt.Errorf("empty User")
+ }
+ if len(header.GetCommitMessage()) == 0 {
+ return fmt.Errorf("empty CommitMessage")
+ }
+ if len(header.GetBranchName()) == 0 {
+ return fmt.Errorf("empty BranchName")
+ }
+ return nil
}
diff --git a/internal/service/operations/commit_files_test.go b/internal/service/operations/commit_files_test.go
new file mode 100644
index 000000000..ed6919fe7
--- /dev/null
+++ b/internal/service/operations/commit_files_test.go
@@ -0,0 +1,283 @@
+package operations_test
+
+import (
+ "io/ioutil"
+ "os"
+ "path"
+ "testing"
+
+ "google.golang.org/grpc/codes"
+
+ "github.com/stretchr/testify/require"
+ pb "gitlab.com/gitlab-org/gitaly-proto/go"
+ "gitlab.com/gitlab-org/gitaly/internal/git/log"
+ "gitlab.com/gitlab-org/gitaly/internal/service/operations"
+ "gitlab.com/gitlab-org/gitaly/internal/testhelper"
+ "google.golang.org/grpc/metadata"
+)
+
+var (
+ user = &pb.User{
+ Name: []byte("John Doe"),
+ Email: []byte("johndoe@gitlab.com"),
+ GlId: "user-1",
+ }
+ commitFilesMessage = []byte("Change files")
+)
+
+func TestSuccessfulUserCommitFilesRequest(t *testing.T) {
+ server, serverSocketPath := runFullServer(t)
+ defer server.Stop()
+
+ client, conn := operations.NewOperationClient(t, serverSocketPath)
+ defer conn.Close()
+
+ testRepo, testRepoPath, cleanupFn := testhelper.NewTestRepo(t)
+ defer cleanupFn()
+
+ ctxOuter, cancel := testhelper.Context()
+ defer cancel()
+
+ newRepo, newRepoPath, newRepoCleanupFn := testhelper.InitBareRepo(t)
+ defer newRepoCleanupFn()
+
+ md := testhelper.GitalyServersMetadata(t, serverSocketPath)
+ filePath := "my/file.txt"
+ authorName := []byte("Jane Doe")
+ authorEmail := []byte("janedoe@gitlab.com")
+ testCases := []struct {
+ desc string
+ repo *pb.Repository
+ repoPath string
+ branchName string
+ repoCreated bool
+ branchCreated bool
+ }{
+ {
+ desc: "existing repo and branch",
+ repo: testRepo,
+ repoPath: testRepoPath,
+ branchName: "feature",
+ repoCreated: false,
+ branchCreated: false,
+ },
+ {
+ desc: "existing repo, new branch",
+ repo: testRepo,
+ repoPath: testRepoPath,
+ branchName: "new-branch",
+ repoCreated: false,
+ branchCreated: true,
+ },
+ {
+ desc: "new repo",
+ repo: newRepo,
+ repoPath: newRepoPath,
+ branchName: "feature",
+ repoCreated: true,
+ branchCreated: true,
+ },
+ }
+
+ for _, tc := range testCases {
+ t.Run(tc.desc, func(t *testing.T) {
+ ctx := metadata.NewOutgoingContext(ctxOuter, md)
+ headerRequest := headerRequest(tc.repo, user, tc.branchName, commitFilesMessage, authorName, authorEmail)
+ actionsRequest1 := createFileHeaderRequest(filePath)
+ actionsRequest2 := actionContentRequest("My")
+ actionsRequest3 := actionContentRequest(" content")
+
+ stream, err := client.UserCommitFiles(ctx)
+ require.NoError(t, err)
+ require.NoError(t, stream.Send(headerRequest))
+ require.NoError(t, stream.Send(actionsRequest1))
+ require.NoError(t, stream.Send(actionsRequest2))
+ require.NoError(t, stream.Send(actionsRequest3))
+
+ r, err := stream.CloseAndRecv()
+ require.NoError(t, err)
+ require.Equal(t, tc.repoCreated, r.GetBranchUpdate().GetRepoCreated())
+ require.Equal(t, tc.branchCreated, r.GetBranchUpdate().GetBranchCreated())
+
+ headCommit, err := log.GetCommit(ctxOuter, tc.repo, tc.branchName, "")
+ require.NoError(t, err)
+ require.Equal(t, authorName, headCommit.Author.Name)
+ require.Equal(t, user.Name, headCommit.Committer.Name)
+ require.Equal(t, authorEmail, headCommit.Author.Email)
+ require.Equal(t, user.Email, headCommit.Committer.Email)
+ require.Equal(t, commitFilesMessage, headCommit.Subject)
+
+ fileContent := testhelper.MustRunCommand(t, nil, "git", "-C", tc.repoPath, "show", headCommit.GetId()+":"+filePath)
+ require.Equal(t, "My content", string(fileContent))
+ })
+ }
+}
+
+func TestFailedUserCommitFilesRequestDueToHooks(t *testing.T) {
+ server, serverSocketPath := runFullServer(t)
+ defer server.Stop()
+
+ client, conn := operations.NewOperationClient(t, serverSocketPath)
+ defer conn.Close()
+
+ testRepo, testRepoPath, cleanupFn := testhelper.NewTestRepo(t)
+ defer cleanupFn()
+
+ ctxOuter, cancel := testhelper.Context()
+ defer cancel()
+
+ branchName := "feature"
+ filePath := "my/file.txt"
+ headerRequest := headerRequest(testRepo, user, branchName, commitFilesMessage, nil, nil)
+ actionsRequest1 := createFileHeaderRequest(filePath)
+ actionsRequest2 := actionContentRequest("My content")
+ hookContent := []byte("#!/bin/sh\nprintenv | paste -sd ' ' -\nexit 1")
+
+ for _, hookName := range operations.GitlabPreHooks {
+ t.Run(hookName, func(t *testing.T) {
+ hookPath := path.Join(testRepoPath, "hooks", hookName)
+ ioutil.WriteFile(hookPath, hookContent, 0755)
+ defer os.Remove(hookPath)
+
+ md := testhelper.GitalyServersMetadata(t, serverSocketPath)
+ ctx := metadata.NewOutgoingContext(ctxOuter, md)
+ stream, err := client.UserCommitFiles(ctx)
+ require.NoError(t, err)
+ require.NoError(t, stream.Send(headerRequest))
+ require.NoError(t, stream.Send(actionsRequest1))
+ require.NoError(t, stream.Send(actionsRequest2))
+
+ r, err := stream.CloseAndRecv()
+ require.NoError(t, err)
+
+ require.Contains(t, r.PreReceiveError, "GL_ID="+user.GlId)
+ require.Contains(t, r.PreReceiveError, "GL_USERNAME="+user.GlUsername)
+ })
+ }
+}
+
+func TestFailedUserCommitFilesRequestDueToIndexError(t *testing.T) {
+ server, serverSocketPath := runFullServer(t)
+ defer server.Stop()
+
+ client, conn := operations.NewOperationClient(t, serverSocketPath)
+ defer conn.Close()
+
+ testRepo, _, cleanupFn := testhelper.NewTestRepo(t)
+ defer cleanupFn()
+
+ ctxOuter, cancel := testhelper.Context()
+ defer cancel()
+
+ md := testhelper.GitalyServersMetadata(t, serverSocketPath)
+ ctx := metadata.NewOutgoingContext(ctxOuter, md)
+ headerRequest := headerRequest(testRepo, user, "feature", commitFilesMessage, nil, nil)
+ actionsRequest1 := createFileHeaderRequest("README.md")
+ actionsRequest2 := actionContentRequest("This file already exists")
+
+ stream, err := client.UserCommitFiles(ctx)
+ require.NoError(t, err)
+ require.NoError(t, stream.Send(headerRequest))
+ require.NoError(t, stream.Send(actionsRequest1))
+ require.NoError(t, stream.Send(actionsRequest2))
+
+ r, err := stream.CloseAndRecv()
+ require.NoError(t, err)
+ require.Equal(t, r.GetIndexError(), "A file with this name already exists")
+}
+
+func TestFailedUserCommitFilesRequest(t *testing.T) {
+ server, serverSocketPath := runFullServer(t)
+ defer server.Stop()
+
+ client, conn := operations.NewOperationClient(t, serverSocketPath)
+ defer conn.Close()
+
+ testRepo, _, cleanupFn := testhelper.NewTestRepo(t)
+ defer cleanupFn()
+
+ ctxOuter, cancel := testhelper.Context()
+ defer cancel()
+
+ md := testhelper.GitalyServersMetadata(t, serverSocketPath)
+ ctx := metadata.NewOutgoingContext(ctxOuter, md)
+ branchName := "feature"
+ testCases := []struct {
+ desc string
+ req *pb.UserCommitFilesRequest
+ }{
+ {
+ desc: "empty Repository",
+ req: headerRequest(nil, user, branchName, commitFilesMessage, nil, nil),
+ },
+ {
+ desc: "empty User",
+ req: headerRequest(testRepo, nil, branchName, commitFilesMessage, nil, nil),
+ },
+ {
+ desc: "empty BranchName",
+ req: headerRequest(testRepo, user, "", commitFilesMessage, nil, nil),
+ },
+ {
+ desc: "empty CommitMessage",
+ req: headerRequest(testRepo, user, branchName, nil, nil, nil),
+ },
+ }
+
+ for _, tc := range testCases {
+ t.Run(tc.desc, func(t *testing.T) {
+ stream, err := client.UserCommitFiles(ctx)
+ require.NoError(t, err)
+
+ require.NoError(t, stream.Send(tc.req))
+
+ _, err = stream.CloseAndRecv()
+ testhelper.AssertGrpcError(t, err, codes.InvalidArgument, tc.desc)
+ })
+ }
+}
+
+func headerRequest(repo *pb.Repository, user *pb.User, branchName string, commitMessage, authorName, authorEmail []byte) *pb.UserCommitFilesRequest {
+ return &pb.UserCommitFilesRequest{
+ UserCommitFilesRequestPayload: &pb.UserCommitFilesRequest_Header{
+ Header: &pb.UserCommitFilesRequestHeader{
+ Repository: repo,
+ User: user,
+ BranchName: []byte(branchName),
+ CommitMessage: commitMessage,
+ CommitAuthorName: authorName,
+ CommitAuthorEmail: authorEmail,
+ StartBranchName: nil,
+ StartRepository: nil,
+ },
+ },
+ }
+}
+
+func createFileHeaderRequest(filePath string) *pb.UserCommitFilesRequest {
+ return actionRequest(&pb.UserCommitFilesAction{
+ UserCommitFilesActionPayload: &pb.UserCommitFilesAction_Header{
+ Header: &pb.UserCommitFilesActionHeader{
+ Action: pb.UserCommitFilesActionHeader_CREATE,
+ Base64Content: false,
+ FilePath: []byte(filePath),
+ },
+ },
+ })
+}
+
+func actionContentRequest(content string) *pb.UserCommitFilesRequest {
+ return actionRequest(&pb.UserCommitFilesAction{
+ UserCommitFilesActionPayload: &pb.UserCommitFilesAction_Content{
+ Content: []byte(content),
+ },
+ })
+}
+
+func actionRequest(action *pb.UserCommitFilesAction) *pb.UserCommitFilesRequest {
+ return &pb.UserCommitFilesRequest{
+ UserCommitFilesRequestPayload: &pb.UserCommitFilesRequest_Action{
+ Action: action,
+ },
+ }
+}
diff --git a/internal/service/remote/fetch_internal_remote_test.go b/internal/service/remote/fetch_internal_remote_test.go
index d9697a171..fbcd706e0 100644
--- a/internal/service/remote/fetch_internal_remote_test.go
+++ b/internal/service/remote/fetch_internal_remote_test.go
@@ -3,7 +3,6 @@ package remote_test
import (
"context"
"net"
- "os"
"testing"
"gitlab.com/gitlab-org/gitaly/internal/service/remote"
@@ -13,7 +12,6 @@ import (
"google.golang.org/grpc/metadata"
"github.com/stretchr/testify/require"
- "gitlab.com/gitlab-org/gitaly/internal/helper"
serverPkg "gitlab.com/gitlab-org/gitaly/internal/server"
pb "gitlab.com/gitlab-org/gitaly-proto/go"
@@ -30,7 +28,7 @@ func TestSuccessfulFetchInternalRemote(t *testing.T) {
remoteRepo, remoteRepoPath, remoteCleanupFn := testhelper.NewTestRepo(t)
defer remoteCleanupFn()
- repo, repoPath, cleanupFn := initRepo(t)
+ repo, repoPath, cleanupFn := testhelper.InitBareRepo(t)
defer cleanupFn()
ctxOuter, cancel := testhelper.Context()
@@ -60,7 +58,7 @@ func TestFailedFetchInternalRemote(t *testing.T) {
client, conn := remote.NewRemoteClient(t, serverSocketPath)
defer conn.Close()
- repo, _, cleanupFn := initRepo(t)
+ repo, _, cleanupFn := testhelper.InitBareRepo(t)
defer cleanupFn()
ctxOuter, cancel := testhelper.Context()
@@ -117,18 +115,6 @@ func TestFailedFetchInternalRemoteDueToValidations(t *testing.T) {
}
}
-func initRepo(t *testing.T) (*pb.Repository, string, func()) {
- testhelper.ConfigureTestStorage()
-
- repo := &pb.Repository{StorageName: "default", RelativePath: "repo.git"}
- repoPath, err := helper.GetPath(repo)
- require.NoError(t, err)
-
- testhelper.MustRunCommand(t, nil, "git", "init", "--bare", repoPath)
-
- return repo, repoPath, func() { os.RemoveAll(repoPath) }
-}
-
func runFullServer(t *testing.T) (*grpc.Server, string) {
server := serverPkg.New(remote.RubyServer)
serverSocketPath := testhelper.GetTemporaryGitalySocketFileName()
diff --git a/internal/testhelper/testhelper.go b/internal/testhelper/testhelper.go
index 6923ab962..ec1a6a632 100644
--- a/internal/testhelper/testhelper.go
+++ b/internal/testhelper/testhelper.go
@@ -329,6 +329,35 @@ func Context() (context.Context, func()) {
return context.WithCancel(context.Background())
}
+func createRepo(t *testing.T, storagePath string) (repo *pb.Repository, repoPath, relativePath string) {
+ repoPath, err := ioutil.TempDir(storagePath, t.Name())
+ require.NoError(t, err)
+ relativePath, err = filepath.Rel(storagePath, repoPath)
+ require.NoError(t, err)
+ repo = &pb.Repository{StorageName: "default", RelativePath: relativePath}
+
+ return repo, repoPath, relativePath
+}
+
+// InitBareRepo creates a new bare repository
+func InitBareRepo(t *testing.T) (*pb.Repository, string, func()) {
+ return initRepo(t, true)
+}
+
+func initRepo(t *testing.T, bare bool) (*pb.Repository, string, func()) {
+ ConfigureTestStorage()
+
+ repo, repoPath, _ := createRepo(t, GitlabTestStoragePath())
+ args := []string{"init"}
+ if bare {
+ args = append(args, "--bare")
+ }
+
+ MustRunCommand(t, nil, "git", append(args, repoPath)...)
+
+ return repo, repoPath, func() { os.RemoveAll(repoPath) }
+}
+
// NewTestRepo creates a bare copy of the test repository.
func NewTestRepo(t *testing.T) (repo *pb.Repository, repoPath string, cleanup func()) {
return cloneTestRepo(t, true)
@@ -341,23 +370,19 @@ func NewTestRepoWithWorktree(t *testing.T) (repo *pb.Repository, repoPath string
}
func cloneTestRepo(t *testing.T, bare bool) (repo *pb.Repository, repoPath string, cleanup func()) {
- testRepo := TestRepository()
storagePath := GitlabTestStoragePath()
+ repo, repoPath, relativePath := createRepo(t, storagePath)
+ testRepo := TestRepository()
testRepoPath := path.Join(storagePath, testRepo.RelativePath)
-
- repoPath, err := ioutil.TempDir(storagePath, t.Name())
- require.NoError(t, err)
- relativePath, err := filepath.Rel(storagePath, repoPath)
- require.NoError(t, err)
- repo = &pb.Repository{StorageName: "default", RelativePath: relativePath}
-
args := []string{"clone", "--no-hardlinks", "--dissociate"}
+
if bare {
args = append(args, "--bare")
} else {
// For non-bare repos the relative path is the .git folder inside the path
repo.RelativePath = path.Join(relativePath, ".git")
}
+
MustRunCommand(t, nil, "git", append(args, testRepoPath, repoPath)...)
return repo, repoPath, func() { os.RemoveAll(repoPath) }
diff --git a/ruby/lib/gitaly_server/operations_service.rb b/ruby/lib/gitaly_server/operations_service.rb
index a9fc66b7c..f20428f1d 100644
--- a/ruby/lib/gitaly_server/operations_service.rb
+++ b/ruby/lib/gitaly_server/operations_service.rb
@@ -207,11 +207,10 @@ module GitalyServer
def user_rebase(request, call)
bridge_exceptions do
+ begin
repo = Gitlab::Git::Repository.from_gitaly(request.repository, call)
user = Gitlab::Git::User.from_gitaly(request.user)
remote_repository = Gitlab::Git::GitalyRemoteRepository.new(request.remote_repository, call)
-
- begin
rebase_sha = repo.rebase(user, request.rebase_id,
branch: request.branch,
branch_sha: request.branch_sha,
@@ -227,8 +226,70 @@ module GitalyServer
end
end
+ def user_commit_files(call)
+ bridge_exceptions do
+ begin
+ actions = []
+ request_enum = call.each_remote_read
+ header = request_enum.next.header
+
+ loop do
+ action = request_enum.next.action
+
+ if action.header
+ actions << commit_files_action_from_gitaly_request(action.header)
+ else
+ actions.last[:content] << action.content
+ end
+ end
+
+ repo = Gitlab::Git::Repository.from_gitaly(header.repository, call)
+ user = Gitlab::Git::User.from_gitaly(header.user)
+ opts = commit_files_opts(call, header, actions)
+
+ branch_update = branch_update_result(repo.multi_action(user, opts))
+
+ Gitaly::UserCommitFilesResponse.new(branch_update: branch_update)
+ rescue Gitlab::Git::Index::IndexError => e
+ Gitaly::UserCommitFilesResponse.new(index_error: e.message)
+ rescue Gitlab::Git::HooksService::PreReceiveError => e
+ Gitaly::UserCommitFilesResponse.new(pre_receive_error: e.message)
+ end
+ end
+ end
+
private
+ def commit_files_opts(call, header, actions)
+ opts = {
+ branch_name: header.branch_name,
+ message: header.commit_message.b,
+ actions: actions
+ }
+
+ if header.start_repository
+ opts[:start_repository] = Gitlab::Git::GitalyRemoteRepository.new(header.start_repository, call)
+ end
+
+ optional_fields = {
+ start_branch_name: 'start_branch_name',
+ author_name: 'commit_author_name',
+ author_email: 'commit_author_email'
+ }.transform_values { |v| header[v].presence }
+
+ opts.merge(optional_fields)
+ end
+
+ def commit_files_action_from_gitaly_request(header)
+ {
+ action: header.action.downcase,
+ file_path: header.file_path,
+ previous_path: header.previous_path,
+ encoding: header.base64_content ? 'base64' : '',
+ content: ''
+ }
+ end
+
def branch_update_result(gitlab_update_result)
return if gitlab_update_result.nil?
diff --git a/ruby/lib/gitlab/git.rb b/ruby/lib/gitlab/git.rb
index 562553877..cf0f7fc7b 100644
--- a/ruby/lib/gitlab/git.rb
+++ b/ruby/lib/gitlab/git.rb
@@ -8,6 +8,7 @@ require 'active_support/core_ext/object/blank'
require 'active_support/core_ext/numeric/bytes'
require 'active_support/core_ext/numeric/time'
require 'active_support/core_ext/module/delegation'
+require 'active_support/core_ext/hash/transform_values'
require 'active_support/core_ext/enumerable'
# We split our mock implementation of Gitlab::GitalyClient into a separate file