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:
authorPavlo Strokov <pstrokov@gitlab.com>2021-01-18 17:41:32 +0300
committerPavlo Strokov <pstrokov@gitlab.com>2021-01-18 17:41:32 +0300
commitf8428022b2d5b03010b476d14e8bcedb347e25c7 (patch)
tree1fe21f98820a8652cdcbbc46da508c7ea6825f36
parentd31f61307ff82da6d881612bb677bbd3e2f61f2f (diff)
parenta7312d8b4221fe4cffd61e4287b314c944b402b2 (diff)
Merge branch 'ps-cleanup-ruby-user-squash' into 'master'
Removal of ruby implementation of the UserSquash Closes #3258 See merge request gitlab-org/gitaly!2968
-rw-r--r--changelogs/unreleased/ps-cleanup-ruby-user-squash.yml5
-rw-r--r--internal/gitaly/service/operations/squash.go76
-rw-r--r--internal/gitaly/service/operations/squash_test.go20
-rw-r--r--internal/metadata/featureflag/feature_flags.go3
-rw-r--r--ruby/lib/gitaly_server/operations_service.rb16
-rw-r--r--ruby/lib/gitlab/git/repository.rb37
-rw-r--r--ruby/spec/lib/gitlab/git/repository_spec.rb133
7 files changed, 35 insertions, 255 deletions
diff --git a/changelogs/unreleased/ps-cleanup-ruby-user-squash.yml b/changelogs/unreleased/ps-cleanup-ruby-user-squash.yml
new file mode 100644
index 000000000..23aaaab60
--- /dev/null
+++ b/changelogs/unreleased/ps-cleanup-ruby-user-squash.yml
@@ -0,0 +1,5 @@
+---
+title: Removal of ruby implementation of the UserSquash
+merge_request: 2968
+author:
+type: removed
diff --git a/internal/gitaly/service/operations/squash.go b/internal/gitaly/service/operations/squash.go
index f90c32796..4fad71a78 100644
--- a/internal/gitaly/service/operations/squash.go
+++ b/internal/gitaly/service/operations/squash.go
@@ -13,33 +13,16 @@ import (
"strings"
"github.com/grpc-ecosystem/go-grpc-middleware/logging/logrus/ctxlogrus"
- "github.com/prometheus/client_golang/prometheus"
"gitlab.com/gitlab-org/gitaly/internal/command"
"gitlab.com/gitlab-org/gitaly/internal/git"
"gitlab.com/gitlab-org/gitaly/internal/git/alternates"
- "gitlab.com/gitlab-org/gitaly/internal/gitaly/rubyserver"
"gitlab.com/gitlab-org/gitaly/internal/helper"
"gitlab.com/gitlab-org/gitaly/internal/helper/text"
- "gitlab.com/gitlab-org/gitaly/internal/metadata/featureflag"
"gitlab.com/gitlab-org/gitaly/proto/go/gitalypb"
"google.golang.org/grpc/codes"
"google.golang.org/grpc/status"
)
-var (
- userSquashImplCounter = prometheus.NewCounterVec(
- prometheus.CounterOpts{
- Name: "gitaly_user_squash_counter",
- Help: "Number of calls to UserSquash rpc for each implementation (ruby/go)",
- },
- []string{"impl"},
- )
-)
-
-func init() {
- prometheus.MustRegister(userSquashImplCounter)
-}
-
const (
squashWorktreePrefix = "squash"
gitlabWorktreesSubDir = "gitlab-worktree"
@@ -50,24 +33,32 @@ func (s *Server) UserSquash(ctx context.Context, req *gitalypb.UserSquashRequest
return nil, status.Errorf(codes.InvalidArgument, "UserSquash: %v", err)
}
- if featureflag.IsEnabled(ctx, featureflag.GoUserSquash) {
- userSquashImplCounter.WithLabelValues("go").Inc()
- return s.userSquashGo(ctx, req)
+ if strings.Contains(req.GetSquashId(), "/") {
+ return nil, helper.ErrInvalidArgument(errors.New("worktree id can't contain slashes"))
}
- userSquashImplCounter.WithLabelValues("ruby").Inc()
-
- client, err := s.ruby.OperationServiceClient(ctx)
+ repo := req.GetRepository()
+ repoPath, err := s.locator.GetRepoPath(repo)
if err != nil {
- return nil, err
+ return nil, helper.ErrInternal(fmt.Errorf("repo path: %w", err))
}
+ env := alternates.Env(repoPath, repo.GetGitObjectDirectory(), repo.GetGitAlternateObjectDirectories())
- clientCtx, err := rubyserver.SetHeaders(ctx, s.locator, req.GetRepository())
+ sha, err := s.userSquash(ctx, req, env, repoPath)
if err != nil {
- return nil, err
+ var gitErr gitError
+ if errors.As(err, &gitErr) {
+ if gitErr.ErrMsg != "" {
+ // we log an actual error as it would be lost otherwise (it is not sent back to the client)
+ ctxlogrus.Extract(ctx).WithError(err).Error("user squash")
+ return &gitalypb.UserSquashResponse{GitError: gitErr.ErrMsg}, nil
+ }
+ }
+
+ return nil, helper.ErrInternal(err)
}
- return client.UserSquash(clientCtx, req)
+ return &gitalypb.UserSquashResponse{SquashSha: sha}, nil
}
func validateUserSquashRequest(req *gitalypb.UserSquashRequest) error {
@@ -113,36 +104,7 @@ func (er gitError) Error() string {
return er.ErrMsg + ": " + er.Err.Error()
}
-func (s *Server) userSquashGo(ctx context.Context, req *gitalypb.UserSquashRequest) (*gitalypb.UserSquashResponse, error) {
- if strings.Contains(req.GetSquashId(), "/") {
- return nil, helper.ErrInvalidArgument(errors.New("worktree id can't contain slashes"))
- }
-
- repo := req.GetRepository()
- repoPath, err := s.locator.GetRepoPath(repo)
- if err != nil {
- return nil, helper.ErrInternal(fmt.Errorf("repo path: %w", err))
- }
- env := alternates.Env(repoPath, repo.GetGitObjectDirectory(), repo.GetGitAlternateObjectDirectories())
-
- sha, err := s.runUserSquashGo(ctx, req, env, repoPath)
- if err != nil {
- var gitErr gitError
- if errors.As(err, &gitErr) {
- if gitErr.ErrMsg != "" {
- // we log an actual error as it would be lost otherwise (it is not sent back to the client)
- ctxlogrus.Extract(ctx).WithError(err).Error("user squash")
- return &gitalypb.UserSquashResponse{GitError: gitErr.ErrMsg}, nil
- }
- }
-
- return nil, helper.ErrInternal(err)
- }
-
- return &gitalypb.UserSquashResponse{SquashSha: sha}, nil
-}
-
-func (s *Server) runUserSquashGo(ctx context.Context, req *gitalypb.UserSquashRequest, env []string, repoPath string) (string, error) {
+func (s *Server) userSquash(ctx context.Context, req *gitalypb.UserSquashRequest, env []string, repoPath string) (string, error) {
sparseDiffFiles, err := s.diffFiles(ctx, env, repoPath, req)
if err != nil {
return "", fmt.Errorf("define diff files: %w", err)
diff --git a/internal/gitaly/service/operations/squash_test.go b/internal/gitaly/service/operations/squash_test.go
index 9cb734521..e27a57899 100644
--- a/internal/gitaly/service/operations/squash_test.go
+++ b/internal/gitaly/service/operations/squash_test.go
@@ -14,7 +14,6 @@ import (
"gitlab.com/gitlab-org/gitaly/internal/git/log"
"gitlab.com/gitlab-org/gitaly/internal/gitaly/config"
"gitlab.com/gitlab-org/gitaly/internal/helper/text"
- "gitlab.com/gitlab-org/gitaly/internal/metadata/featureflag"
"gitlab.com/gitlab-org/gitaly/internal/testhelper"
"gitlab.com/gitlab-org/gitaly/proto/go/gitalypb"
"google.golang.org/grpc/codes"
@@ -33,17 +32,16 @@ var (
)
func TestSuccessfulUserSquashRequest(t *testing.T) {
- testhelper.NewFeatureSets(
- []featureflag.FeatureFlag{featureflag.GoUserSquash},
- ).Run(t, func(t *testing.T, ctx context.Context) {
- t.Run("with sparse checkout", func(t *testing.T) {
- testSuccessfulUserSquashRequest(t, ctx, startSha, endSha)
- })
+ ctx, cancel := testhelper.Context()
+ defer cancel()
- t.Run("without sparse checkout", func(t *testing.T) {
- // there are no files that could be used for sparse checkout for those two commits
- testSuccessfulUserSquashRequest(t, ctx, "60ecb67744cb56576c30214ff52294f8ce2def98", "c84ff944ff4529a70788a5e9003c2b7feae29047")
- })
+ t.Run("with sparse checkout", func(t *testing.T) {
+ testSuccessfulUserSquashRequest(t, ctx, startSha, endSha)
+ })
+
+ t.Run("without sparse checkout", func(t *testing.T) {
+ // there are no files that could be used for sparse checkout for those two commits
+ testSuccessfulUserSquashRequest(t, ctx, "60ecb67744cb56576c30214ff52294f8ce2def98", "c84ff944ff4529a70788a5e9003c2b7feae29047")
})
}
diff --git a/internal/metadata/featureflag/feature_flags.go b/internal/metadata/featureflag/feature_flags.go
index 54f1b613a..f3b3bcf1b 100644
--- a/internal/metadata/featureflag/feature_flags.go
+++ b/internal/metadata/featureflag/feature_flags.go
@@ -25,8 +25,6 @@ var (
GoUserCreateBranch = FeatureFlag{Name: "go_user_create_branch", OnByDefault: false}
// GoUserDeleteBranch enables the Go implementation of UserDeleteBranch
GoUserDeleteBranch = FeatureFlag{Name: "go_user_delete_branch", OnByDefault: true}
- // GoUserSquash enables the Go implementation of UserSquash
- GoUserSquash = FeatureFlag{Name: "go_user_squash", OnByDefault: true}
// GoUserCommitFiles enables the Go implementation of UserCommitFiles
GoUserCommitFiles = FeatureFlag{Name: "go_user_commit_files", OnByDefault: false}
// GoResolveConflicts enables the Go implementation of ResolveConflicts
@@ -112,7 +110,6 @@ var All = []FeatureFlag{
GoUserFFBranch,
GoUserCreateBranch,
GoUserDeleteBranch,
- GoUserSquash,
GoUserCommitFiles,
GoResolveConflicts,
GoUserDeleteTag,
diff --git a/ruby/lib/gitaly_server/operations_service.rb b/ruby/lib/gitaly_server/operations_service.rb
index e227dadfa..74286ffd5 100644
--- a/ruby/lib/gitaly_server/operations_service.rb
+++ b/ruby/lib/gitaly_server/operations_service.rb
@@ -293,22 +293,6 @@ module GitalyServer
raise GRPC::InvalidArgument.new(e.message)
end
- def user_squash(request, call)
- repo = Gitlab::Git::Repository.from_gitaly(request.repository, call)
- user = Gitlab::Git::User.from_gitaly(request.user)
- author = Gitlab::Git::User.from_gitaly(request.author)
-
- squash_sha = repo.squash(user, request.squash_id,
- start_sha: request.start_sha,
- end_sha: request.end_sha,
- author: author,
- message: request.commit_message)
-
- Gitaly::UserSquashResponse.new(squash_sha: squash_sha)
- rescue Gitlab::Git::Repository::GitError => e
- Gitaly::UserSquashResponse.new(git_error: set_utf8!(e.message))
- end
-
def user_apply_patch(call)
stream = call.each_remote_read
first_request = stream.next
diff --git a/ruby/lib/gitlab/git/repository.rb b/ruby/lib/gitlab/git/repository.rb
index 2dc164f5f..afa2ed206 100644
--- a/ruby/lib/gitlab/git/repository.rb
+++ b/ruby/lib/gitlab/git/repository.rb
@@ -10,10 +10,9 @@ module Gitlab
include Gitlab::Utils::StrongMemoize
# In https://gitlab.com/gitlab-org/gitaly/merge_requests/698
- # We copied these two prefixes into gitaly-go, so don't change these
- # or things will break! (REBASE_WORKTREE_PREFIX and SQUASH_WORKTREE_PREFIX)
+ # We copied this prefix into gitaly-go, so don't change it
+ # or things will break! (REBASE_WORKTREE_PREFIX)
REBASE_WORKTREE_PREFIX = 'rebase'.freeze
- SQUASH_WORKTREE_PREFIX = 'squash'.freeze
AM_WORKTREE_PREFIX = 'am'.freeze
GITALY_INTERNAL_URL = 'ssh://gitaly/internal.git'.freeze
AUTOCRLF_VALUES = { 'true' => true, 'false' => false, 'input' => :input }.freeze
@@ -399,38 +398,6 @@ module Gitlab
end
end
- def squash(user, squash_id, start_sha:, end_sha:, author:, message:)
- worktree = Gitlab::Git::Worktree.new(path, SQUASH_WORKTREE_PREFIX, squash_id)
- env = git_env.merge(user.git_env).merge(
- 'GIT_AUTHOR_NAME' => author.name,
- 'GIT_AUTHOR_EMAIL' => author.email
- )
- diff_range = "#{start_sha}...#{end_sha}"
- diff_files = run_git!(
- %W[diff --name-only --diff-filter=ar --binary #{diff_range}]
- ).chomp
-
- with_worktree(worktree, start_sha, sparse_checkout_files: diff_files, env: env) do
- # Apply diff of the `diff_range` to the worktree
- diff = run_git!(%W[diff --binary #{diff_range}])
- run_git!(%w[apply --index --3way --whitespace=nowarn], chdir: worktree.path, env: env, include_stderr: true) do |stdin|
- stdin.binmode
- stdin.write(diff)
- end
-
- # Commit the `diff_range` diff
- run_git!(%W[commit --no-verify --message #{message}], chdir: worktree.path, env: env, include_stderr: true)
-
- # Return the squash sha. May print a warning for ambiguous refs, but
- # we can ignore that with `--quiet` and just take the SHA, if present.
- # HEAD here always refers to the current HEAD commit, even if there is
- # another ref called HEAD.
- run_git!(
- %w[rev-parse --quiet --verify HEAD], chdir: worktree.path, env: env
- ).chomp
- end
- end
-
def commit_patches(start_point, patches, extra_env: {})
worktree = Gitlab::Git::Worktree.new(path, AM_WORKTREE_PREFIX, SecureRandom.hex)
env = git_env.merge(extra_env)
diff --git a/ruby/spec/lib/gitlab/git/repository_spec.rb b/ruby/spec/lib/gitlab/git/repository_spec.rb
index 16c485660..862a55f04 100644
--- a/ruby/spec/lib/gitlab/git/repository_spec.rb
+++ b/ruby/spec/lib/gitlab/git/repository_spec.rb
@@ -627,139 +627,6 @@ describe Gitlab::Git::Repository do # rubocop:disable Metrics/BlockLength
end
end
- describe '#squash' do
- let(:repository) { mutable_repository }
- let(:squash_id) { '1' }
- let(:start_sha) { '4b4918a572fa86f9771e5ba40fbd48e1eb03e2c6' }
- let(:end_sha) { '12d65c8dd2b2676fa3ac47d955accc085a37a9c1' }
-
- subject do
- opts = {
- start_sha: start_sha,
- end_sha: end_sha,
- author: user,
- message: 'Squash commit message'
- }
-
- repository.squash(user, squash_id, **opts)
- end
-
- describe 'sparse checkout' do
- let(:expected_files) { %w[files files/js files/js/application.js] }
-
- it 'checks out only the files in the diff' do
- allow(repository).to receive(:with_worktree).and_wrap_original do |m, *args, **kwargs|
- m.call(*args, **kwargs) do
- worktree = args[0]
- files_pattern = File.join(worktree.path, '**', '*')
- expected = expected_files.map do |path|
- File.expand_path(path, worktree.path)
- end
-
- expect(Dir[files_pattern]).to eq(expected)
- end
- end
-
- subject
- end
-
- context 'when the diff contains a rename' do
- let(:end_sha) { new_commit_move_file(repository_rugged).oid }
-
- after do
- # Erase our commits so other tests get the original repo
- repository_rugged.references.update('refs/heads/master', SeedRepo::LastCommit::ID)
- end
-
- it 'does not include the renamed file in the sparse checkout' do
- allow(repository).to receive(:with_worktree).and_wrap_original do |m, *args, **kwargs|
- m.call(*args, **kwargs) do
- worktree = args[0]
- files_pattern = File.join(worktree.path, '**', '*')
-
- expect(Dir[files_pattern]).not_to include('CHANGELOG')
- expect(Dir[files_pattern]).not_to include('encoding/CHANGELOG')
- end
- end
-
- subject
- end
- end
-
- describe 'when worktree throws an error' do
- before do
- allow(repository).to receive(:run_git).and_return(['ok', 0])
- expect(repository).to receive(:run_git)
- .with(array_including('worktree'), chdir: anything, env: anything, nice: false, include_stderr: true, lazy_block: anything)
- .and_return(['error', 1])
- end
-
- it 'includes the stderr output' do
- expect do
- subject
- end.to raise_error(described_class::GitError, 'error')
- end
- end
- end
-
- describe 'with an ASCII-8BIT diff' do
- let(:diff) do
- <<~RAW_DIFF
- diff --git a/README.md b/README.md
- index faaf198..43c5edf 100644
- --- a/README.md
- +++ b/README.md
- @@ -1,4 +1,4 @@
- -testme
- +✓ testme
- ======
-
- Sample repo for testing gitlab features
- RAW_DIFF
- end
-
- it 'applies a ASCII-8BIT diff' do
- allow(repository).to receive(:run_git!).and_call_original
- allow(repository).to receive(:run_git!)
- .with(%W[diff --binary #{start_sha}...#{end_sha}])
- .and_return(diff.force_encoding('ASCII-8BIT'))
-
- expect(subject).to match(/\h{40}/)
- end
- end
-
- describe 'with trailing whitespace in an invalid patch' do
- let(:diff) do
- # rubocop:disable Layout/TrailingWhitespace
- <<~RAW_DIFF
- diff --git a/README.md b/README.md
- index faaf198..43c5edf 100644
- --- a/README.md
- +++ b/README.md
- @@ -1,4 +1,4 @@
- -testme
- +
- ======
-
- Sample repo for testing gitlab features
- RAW_DIFF
- # rubocop:enable Layout/TrailingWhitespace
- end
-
- it 'does not include whitespace warnings in the error' do
- allow(repository).to receive(:run_git!).and_call_original
- allow(repository).to receive(:run_git!)
- .with(%W[diff --binary #{start_sha}...#{end_sha}])
- .and_return(diff.force_encoding('ASCII-8BIT'))
-
- expect { subject }.to raise_error do |error|
- expect(error).to be_a(described_class::GitError)
- expect(error.message).not_to include('trailing whitespace')
- end
- end
- end
- end
-
describe '#cleanup' do
context 'when Rugged has been called' do
it 'calls close on Rugged::Repository' do