Commit 92249f1a authored by Douwe Maan's avatar Douwe Maan

Merge branch 'github-importer-refactor' into 'master'

Rewrite the GitHub importer to perform work in parallel and greatly improve performance

Closes #33135, #38621, and #39361

See merge request gitlab-org/gitlab-ce!14731
parents 3c369ba1 6e242e82
......@@ -43,7 +43,7 @@ class Import::GithubController < Import::BaseController
@target_namespace = find_or_create_namespace(namespace_path, current_user.namespace_path)
if can?(current_user, :create_projects, @target_namespace)
@project = Gitlab::GithubImport::ProjectCreator.new(repo, @project_name, @target_namespace, current_user, access_params, type: provider).execute
@project = Gitlab::LegacyGithubImport::ProjectCreator.new(repo, @project_name, @target_namespace, current_user, access_params, type: provider).execute
else
render 'unauthorized'
end
......@@ -52,7 +52,7 @@ class Import::GithubController < Import::BaseController
private
def client
@client ||= Gitlab::GithubImport::Client.new(session[access_token_key], client_options)
@client ||= Gitlab::LegacyGithubImport::Client.new(session[access_token_key], client_options)
end
def verify_import_enabled
......
......@@ -365,6 +365,7 @@ class Project < ActiveRecord::Base
scope :abandoned, -> { where('projects.last_activity_at < ?', 6.months.ago) }
scope :excluding_project, ->(project) { where.not(id: project) }
scope :import_started, -> { where(import_status: 'started') }
state_machine :import_status, initial: :none do
event :import_schedule do
......@@ -1190,6 +1191,10 @@ class Project < ActiveRecord::Base
!!repository.exists?
end
def wiki_repository_exists?
wiki.repository_exists?
end
# update visibility_level of forks
def update_forks_visibility_level
return unless visibility_level < visibility_level_was
......@@ -1433,6 +1438,31 @@ class Project < ActiveRecord::Base
reload_repository!
end
def after_import
repository.after_import
import_finish
remove_import_jid
update_project_counter_caches
end
def update_project_counter_caches
classes = [
Projects::OpenIssuesCountService,
Projects::OpenMergeRequestsCountService
]
classes.each do |klass|
klass.new(self).refresh_cache
end
end
def remove_import_jid
return unless import_jid
Gitlab::SidekiqStatus.unset(import_jid)
update_column(:import_jid, nil)
end
def running_or_pending_build_count(force: false)
Rails.cache.fetch(['projects', id, 'running_or_pending_build_count'], force: force) do
builds.running_or_pending.count(:all)
......@@ -1690,6 +1720,17 @@ class Project < ActiveRecord::Base
Gitlab::ReferenceCounter.new(gl_repository(is_wiki: wiki))
end
# Refreshes the expiration time of the associated import job ID.
#
# This method can be used by asynchronous importers to refresh the status,
# preventing the StuckImportJobsWorker from marking the import as failed.
def refresh_import_jid_expiration
return unless import_jid
Gitlab::SidekiqStatus
.set(import_jid, StuckImportJobsWorker::IMPORT_JOBS_EXPIRATION)
end
private
def storage
......
......@@ -973,6 +973,10 @@ class Repository
raw_repository.fetch_source_branch!(source_repository.raw_repository, source_branch, local_ref)
end
def remote_exists?(name)
raw_repository.remote_exists?(name)
end
def compare_source_branch(target_branch_name, source_repository, source_branch_name, straight:)
raw_repository.compare_source_branch(target_branch_name, source_repository.raw_repository, source_branch_name, straight: straight)
end
......
......@@ -267,18 +267,23 @@ class User < ActiveRecord::Base
end
end
def for_github_id(id)
joins(:identities)
.where(identities: { provider: :github, extern_uid: id.to_s })
end
# Find a User by their primary email or any associated secondary email
def find_by_any_email(email)
sql = 'SELECT *
FROM users
WHERE id IN (
SELECT id FROM users WHERE email = :email
UNION
SELECT emails.user_id FROM emails WHERE email = :email
)
LIMIT 1;'
by_any_email(email).take
end
# Returns a relation containing all the users for the given Email address
def by_any_email(email)
users = where(email: email)
emails = joins(:emails).where(emails: { email: email })
union = Gitlab::SQL::Union.new([users, emails])
User.find_by_sql([sql, { email: email }]).first
from("(#{union.to_sql}) #{table_name}")
end
def filter(filter_name)
......
......@@ -4,6 +4,18 @@ module Projects
Error = Class.new(StandardError)
# Returns true if this importer is supposed to perform its work in the
# background.
#
# This method will only return `true` if async importing is explicitly
# supported by an importer class (`Gitlab::GithubImport::ParallelImporter`
# for example).
def async?
return false unless has_importer?
!!importer_class.try(:async?)
end
def execute
add_repository_to_project unless project.gitlab_project_import?
......@@ -75,12 +87,16 @@ module Projects
end
end
def importer_class
Gitlab::ImportSources.importer(project.import_type)
end
def has_importer?
Gitlab::ImportSources.importer_names.include?(project.import_type)
end
def importer
Gitlab::ImportSources.importer(project.import_type).new(project)
importer_class.new(project)
end
def unknown_url?
......
# frozen_string_literal: true
module Gitlab
module GithubImport
# NotifyUponDeath can be included into a GitHub worker class if it should
# notify any JobWaiter instances upon being moved to the Sidekiq dead queue.
#
# Note that this will only notify the waiter upon graceful termination, a
# SIGKILL will still result in the waiter _not_ being notified.
#
# Workers including this module must have jobs passed where the last
# argument is the key to notify, as a String.
module NotifyUponDeath
extend ActiveSupport::Concern
included do
# If a job is being exhausted we still want to notify the
# AdvanceStageWorker. This prevents the entire import from getting stuck
# just because 1 job threw too many errors.
sidekiq_retries_exhausted do |job|
args = job['args']
jid = job['jid']
if args.length == 3 && (key = args.last) && key.is_a?(String)
JobWaiter.notify(key, jid)
end
end
end
end
end
end
# frozen_string_literal: true
module Gitlab
module GithubImport
# ObjectImporter defines the base behaviour for every Sidekiq worker that
# imports a single resource such as a note or pull request.
module ObjectImporter
extend ActiveSupport::Concern
included do
include Sidekiq::Worker
include GithubImport::Queue
include ReschedulingMethods
include NotifyUponDeath
end
# project - An instance of `Project` to import the data into.
# client - An instance of `Gitlab::GithubImport::Client`
# hash - A Hash containing the details of the object to import.
def import(project, client, hash)
object = representation_class.from_json_hash(hash)
importer_class.new(object, project, client).execute
counter.increment(project: project.path_with_namespace)
end
def counter
@counter ||= Gitlab::Metrics.counter(counter_name, counter_description)
end
# Returns the representation class to use for the object. This class must
# define the class method `from_json_hash`.
def representation_class
raise NotImplementedError
end
# Returns the class to use for importing the object.
def importer_class
raise NotImplementedError
end
# Returns the name (as a Symbol) of the Prometheus counter.
def counter_name
raise NotImplementedError
end
# Returns the description (as a String) of the Prometheus counter.
def counter_description
raise NotImplementedError
end
end
end
end
module Gitlab
module GithubImport
module Queue
extend ActiveSupport::Concern
included do
# If a job produces an error it may block a stage from advancing
# forever. To prevent this from happening we prevent jobs from going to
# the dead queue. This does mean some resources may not be imported, but
# this is better than a project being stuck in the "import" state
# forever.
sidekiq_options queue: 'github_importer', dead: false, retry: 5
end
end
end
end
# frozen_string_literal: true
module Gitlab
module GithubImport
# Module that provides methods shared by the various workers used for
# importing GitHub projects.
module ReschedulingMethods
# project_id - The ID of the GitLab project to import the note into.
# hash - A Hash containing the details of the GitHub object to imoprt.
# notify_key - The Redis key to notify upon completion, if any.
def perform(project_id, hash, notify_key = nil)
project = Project.find_by(id: project_id)
return notify_waiter(notify_key) unless project
client = GithubImport.new_client_for(project, parallel: true)
if try_import(project, client, hash)
notify_waiter(notify_key)
else
# In the event of hitting the rate limit we want to reschedule the job
# so its retried after our rate limit has been reset.
self.class
.perform_in(client.rate_limit_resets_in, project.id, hash, notify_key)
end
end
def try_import(*args)
import(*args)
true
rescue RateLimitError
false
end
def notify_waiter(key = nil)
JobWaiter.notify(key, jid) if key
end
end
end
end
# frozen_string_literal: true
module Gitlab
module GithubImport
module StageMethods
# project_id - The ID of the GitLab project to import the data into.
def perform(project_id)
return unless (project = find_project(project_id))
client = GithubImport.new_client_for(project)
try_import(client, project)
end
# client - An instance of Gitlab::GithubImport::Client.
# project - An instance of Project.
def try_import(client, project)
import(client, project)
rescue RateLimitError
self.class.perform_in(client.rate_limit_resets_in, project.id)
end
def find_project(id)
# If the project has been marked as failed we want to bail out
# automatically.
Project.import_started.find_by(id: id)
end
end
end
end
# frozen_string_literal: true
module Gitlab
module GithubImport
# AdvanceStageWorker is a worker used by the GitHub importer to wait for a
# number of jobs to complete, without blocking a thread. Once all jobs have
# been completed this worker will advance the import process to the next
# stage.
class AdvanceStageWorker
include Sidekiq::Worker
sidekiq_options queue: 'github_importer_advance_stage', dead: false
INTERVAL = 30.seconds.to_i
# The number of seconds to wait (while blocking the thread) before
# continueing to the next waiter.
BLOCKING_WAIT_TIME = 5
# The known importer stages and their corresponding Sidekiq workers.
STAGES = {
issues_and_diff_notes: Stage::ImportIssuesAndDiffNotesWorker,
notes: Stage::ImportNotesWorker,
finish: Stage::FinishImportWorker
}.freeze
# project_id - The ID of the project being imported.
# waiters - A Hash mapping Gitlab::JobWaiter keys to the number of
# remaining jobs.
# next_stage - The name of the next stage to start when all jobs have been
# completed.
def perform(project_id, waiters, next_stage)
return unless (project = find_project(project_id))
new_waiters = wait_for_jobs(waiters)
if new_waiters.empty?
# We refresh the import JID here so workers importing individual
# resources (e.g. notes) don't have to do this all the time, reducing
# the pressure on Redis. We _only_ do this once all jobs are done so
# we don't get stuck forever if one or more jobs failed to notify the
# JobWaiter.
project.refresh_import_jid_expiration
STAGES.fetch(next_stage.to_sym).perform_async(project_id)
else
self.class.perform_in(INTERVAL, project_id, new_waiters, next_stage)
end
end
def wait_for_jobs(waiters)
waiters.each_with_object({}) do |(key, remaining), new_waiters|
waiter = JobWaiter.new(remaining, key)
# We wait for a brief moment of time so we don't reschedule if we can
# complete the work fast enough.
waiter.wait(BLOCKING_WAIT_TIME)
next unless waiter.jobs_remaining.positive?
new_waiters[waiter.key] = waiter.jobs_remaining
end
end
def find_project(id)
# We only care about the import JID so we can refresh it. We also only
# want the project if it hasn't been marked as failed yet. It's possible
# the import gets marked as stuck when jobs of the current stage failed
# somehow.
Project.select(:import_jid).import_started.find_by(id: id)
end
end
end
end
# frozen_string_literal: true
module Gitlab
module GithubImport
class ImportDiffNoteWorker
include ObjectImporter
def representation_class
Representation::DiffNote
end
def importer_class
Importer::DiffNoteImporter
end
def counter_name
:github_importer_imported_diff_notes
end
def counter_description
'The number of imported GitHub pull request review comments'
end
end
end
end
# frozen_string_literal: true
module Gitlab
module GithubImport
class ImportIssueWorker
include ObjectImporter
def representation_class
Representation::Issue
end
def importer_class
Importer::IssueAndLabelLinksImporter
end
def counter_name
:github_importer_imported_issues
end
def counter_description
'The number of imported GitHub issues'
end
end
end
end
# frozen_string_literal: true
module Gitlab
module GithubImport
class ImportNoteWorker
include ObjectImporter
def representation_class
Representation::Note
end
def importer_class
Importer::NoteImporter
end
def counter_name
:github_importer_imported_notes
end
def counter_description
'The number of imported GitHub comments'
end
end
end
end
# frozen_string_literal: true
module Gitlab
module GithubImport
class ImportPullRequestWorker
include ObjectImporter
def representation_class
Representation::PullRequest
end
def importer_class
Importer::PullRequestImporter
end
def counter_name
:github_importer_imported_pull_requests
end
def counter_description
'The number of imported GitHub pull requests'
end
end
end
end
# frozen_string_literal: true
module Gitlab
module GithubImport
class RefreshImportJidWorker
include Sidekiq::Worker
include GithubImport::Queue
# The interval to schedule new instances of this job at.
INTERVAL = 1.minute.to_i
def self.perform_in_the_future(*args)
perform_in(INTERVAL, *args)
end
# project_id - The ID of the project that is being imported.
# check_job_id - The ID of the job for which to check the status.
def perform(project_id, check_job_id)
return unless (project = find_project(project_id))
if SidekiqStatus.running?(check_job_id)
# As long as the repository is being cloned we want to keep refreshing
# the import JID status.
project.refresh_import_jid_expiration
self.class.perform_in_the_future(project_id, check_job_id)
end
# If the job is no longer running there's nothing else we need to do. If
# the clone job completed successfully it will have scheduled the next
# stage, if it died there's nothing we can do anyway.
end
def find_project(id)
Project.select(:import_jid).import_started.find_by(id: id)
end
end
end
end
# frozen_string_literal: true
module Gitlab
module GithubImport
module Stage
class FinishImportWorker
include Sidekiq::Worker
include GithubImport::Queue
include StageMethods
# project - An instance of Project.
def import(_, project)
project.after_import
report_import_time(project)
end
def report_import_time(project)
duration = Time.zone.now - project.created_at
path = project.path_with_namespace
histogram.observe({ project: path }, duration)
counter.increment
logger.info("GitHub importer finished for #{path} in #{duration.round(2)} seconds")
end
def histogram
@histogram ||= Gitlab::Metrics.histogram(
:github_importer_total_duration_seconds,
'Total time spent importing GitHub projects, in seconds'
)
end
def counter
@counter ||= Gitlab::Metrics.counter(
:github_importer_imported_projects,
'The number of imported GitHub projects'
)
end
end
end
end
end
# frozen_string_literal: true
module Gitlab
module GithubImport
module Stage
class ImportBaseDataWorker
include Sidekiq::Worker
include GithubImport::Queue
include StageMethods
# These importers are fast enough that we can just run them in the same
# thread.
IMPORTERS = [
Importer::LabelsImporter,
Importer::MilestonesImporter,
Importer::ReleasesImporter
].freeze
# client - An instance of Gitlab::GithubImport::Client.
# project - An instance of Project.
def import(client, project)
IMPORTERS.each do |klass|
klass.new(project, client).execute
end
project.refresh_import_jid_expiration
ImportPullRequestsWorker.perform_async(project.id)
end
end
end
end
end
# frozen_string_literal: true
module Gitlab
module GithubImport
module Stage
class ImportIssuesAndDiffNotesWorker
include Sidekiq::Worker
include GithubImport::Queue
include StageMethods
# The importers to run in this stage. Issues can't be imported earlier
# on as we also use these to enrich pull requests with assigned labels.
IMPORTERS = [
Importer::IssuesImporter,
Importer::DiffNotesImporter
].freeze
# client - An instance of Gitlab::GithubImport::Client.
# project - An instance of Project.
def import(client, project)
waiters = IMPORTERS.each_with_object({}) do |klass, hash|
waiter = klass.new(project, client).execute
hash[waiter.key] = waiter.jobs_remaining
end
AdvanceStageWorker.perform_async(project.id, waiters, :notes)
end
end
end
end
end
# frozen_string_literal: true
module Gitlab
module GithubImport
module Stage
class ImportNotesWorker
include Sidekiq::Worker
include GithubImport::Queue
include StageMethods
# client - An instance of Gitlab::GithubImport::Client.
# project - An instance of Project.
def import(client, project)
waiter = Importer::NotesImporter
.new(project, client)
.execute
AdvanceStageWorker.perform_async(
project.id,
{ waiter.key => waiter.jobs_remaining },
:finish
)
end
end
end
end
end
# frozen_string_literal: true
module Gitlab
module GithubImport
module Stage
class ImportPullRequestsWorker
include Sidekiq::Worker
include GithubImport::Queue
include StageMethods
# client - An instance of Gitlab::GithubImport::Client.
# project - An instance of Project.
def import(client, project)
waiter = Importer::PullRequestsImporter
.new(project, client)
.execute
project.refresh_import_jid_expiration
AdvanceStageWorker.perform_async(
project.id,
{ waiter.key => waiter.jobs_remaining },
:issues_and_diff_notes
)
end
end
end
end
end
# frozen_string_literal: true
module Gitlab
module GithubImport
module Stage
class ImportRepositoryWorker
include Sidekiq::Worker
include GithubImport::Queue
include StageMethods
# client - An instance of Gitlab::GithubImport::Client.
# project - An instance of Project.
def import(client, project)
# In extreme cases it's possible for a clone to take more than the
# import job expiration time. To work around this we schedule a
# separate job that will periodically run and refresh the import
# expiration time.
RefreshImportJidWorker.perform_in_the_future(project.id, jid)
importer = Importer::RepositoryImporter.new(project, client)
return unless importer.execute
counter.increment
ImportBaseDataWorker.perform_async(project.id)
end
def counter
Gitlab::Metrics.counter(
:github_importer_imported_repositories,
'The number of imported GitHub repositories'
)
end
end
end
end
end
......@@ -17,11 +17,16 @@ class RepositoryImportWorker
import_url: project.import_url,
path: project.full_path)
result = Projects::ImportService.new(project, project.creator).execute
service = Projects::ImportService.new(project, project.creator)
result = service.execute
# Some importers may perform their work asynchronously. In this case it's up
# to those importers to mark the import process as complete.
return if service.async?
raise ImportError, result[:message] if result[:status] == :error
project.repository.after_import
project.import_finish
project.after_import
rescue ImportError => ex
fail_import(project, ex.message)
raise
......
......@@ -40,6 +40,8 @@
- [upload_checksum, 1]
- [repository_fork, 1]
- [repository_import, 1]
- [github_importer, 1]
- [github_importer_advance_stage, 1]
- [project_service, 1]
- [delete_user, 1]
- [delete_merged_branches, 1]
......
......@@ -171,6 +171,7 @@ have access to GitLab administration tools and settings.
- [GitLab performance monitoring with InfluxDB](administration/monitoring/performance/introduction.md): Configure GitLab and InfluxDB for measuring performance metrics.
- [GitLab performance monitoring with Prometheus](administration/monitoring/prometheus/index.md): Configure GitLab and Prometheus for measuring performance metrics.
- [Monitoring uptime](user/admin_area/monitoring/health_check.md): Check the server status using the health check endpoint.
- [Monitoring GitHub imports](administration/monitoring/github_imports.md)
### Performance
......
# Monitoring GitHub imports
>**Note:**
Available since [GitLab 10.2][14731].
The GitHub importer exposes various Prometheus metrics that you can use to
monitor the health and progress of the importer.
## Import Duration Times
| Name | Type |
|------------------------------------------|-----------|
| `github_importer_total_duration_seconds` | histogram |
This metric tracks the total time spent (in seconds) importing a project (from
project creation until the import process finishes), for every imported project.
The name of the project is stored in the `project` label in the format
`namespace/name` (e.g. `gitlab-org/gitlab-ce`).
## Number of imported projects
| Name | Type |
|-------------------------------------|---------|
| `github_importer_imported_projects` | counter |
This metric tracks the total number of projects imported over time. This metric
does not expose any labels.
## Number of GitHub API calls
| Name | Type |
|---------------------------------|---------|
| `github_importer_request_count` | counter |
This metric tracks the total number of GitHub API calls performed over time, for
all projects. This metric does not expose any labels.
## Rate limit errors
| Name | Type |
|-----------------------------------|---------|
| `github_importer_rate_limit_hits` | counter |
This metric tracks the number of times we hit the GitHub rate limit, for all
projects. This metric does not expose any labels.
## Number of imported issues
| Name | Type |
|-----------------------------------|---------|
| `github_importer_imported_issues` | counter |
This metric tracks the number of imported issues across all projects.
The name of the project is stored in the `project` label in the format
`namespace/name` (e.g. `gitlab-org/gitlab-ce`).
## Number of imported pull requests
| Name | Type |
|------------------------------------------|---------|
| `github_importer_imported_pull_requests` | counter |
This metric tracks the number of imported pull requests across all projects.
The name of the project is stored in the `project` label in the format
`namespace/name` (e.g. `gitlab-org/gitlab-ce`).
## Number of imported comments
| Name | Type |
|----------------------------------|---------|
| `github_importer_imported_notes` | counter |
This metric tracks the number of imported comments across all projects.
The name of the project is stored in the `project` label in the format
`namespace/name` (e.g. `gitlab-org/gitlab-ce`).
## Number of imported pull request review comments
| Name | Type |
|---------------------------------------|---------|
| `github_importer_imported_diff_notes` | counter |
This metric tracks the number of imported comments across all projects.
The name of the project is stored in the `project` label in the format
`namespace/name` (e.g. `gitlab-org/gitlab-ce`).
## Number of imported repositories
| Name | Type |
|-----------------------------------------|---------|
| `github_importer_imported_repositories` | counter |
This metric tracks the number of imported repositories across all projects. This
metric does not expose any labels.
[14731]: https://gitlab.com/gitlab-org/gitlab-ce/merge_requests/14731
......@@ -37,6 +37,7 @@ comments: false
- [Gotchas](gotchas.md) to avoid
- [Issue and merge requests state models](object_state_models.md)
- [How to dump production data to staging](db_dump.md)
- [Working with the GitHub importer](github_importer.md)
## Performance guides
......
# Working with the GitHub importer
In GitLab 10.2 a new version of the GitHub importer was introduced. This new
importer performs its work in parallel using Sidekiq, greatly reducing the time
necessary to import GitHub projects into a GitLab instance.
The GitHub importer offers two different types of importers: a sequential
importer and a parallel importer. The Rake task `import:github` uses the
sequential importer, while everything else uses the parallel importer. The
difference between these two importers is quite simple: the sequential importer
does all work in a single thread, making it more useful for debugging purposes
or Rake tasks. The parallel importer on the other hand uses Sidekiq.
## Requirements
* GitLab CE 10.2.0 or newer.
* Sidekiq workers that process the `github_importer` and
`github_importer_advance_stage` queues (this is enabled by default).
* Octokit (used for interacting with the GitHub API)
## Code structure
The importer's codebase is broken up into the following directories:
* `lib/gitlab/github_import`: this directory contains most of the code such as
the classes used for importing resources.
* `app/workers/gitlab/github_import`: this directory contains the Sidekiq
workers.
* `app/workers/concerns/gitlab/github_import`: this directory contains a few
modules reused by the various Sidekiq workers.
## Architecture overview
When a GitHub project is imported we schedule and execute a job for the
`RepositoryImportworker` worker as all other importers. However, unlike other
importers we don't immediately perform the work necessary. Instead work is
divided into separate stages, with each stage consisting out of a set of Sidekiq
jobs that are executed. Between every stage a job is scheduled that periodically
checks if all work of the current stage is completed, advancing the import
process to the next stage when this is the case. The worker handling this is
called `Gitlab::GithubImport::AdvanceStageWorker`.
## Stages
### 1. RepositoryImportWorker
This worker will kick off the import process by simply scheduling a job for the
next worker.
### 2. Stage::ImportRepositoryWorker
This worker will import the repository and wiki, scheduling the next stage when
done.
### 3. Stage::ImportBaseDataWorker
This worker will import base data such as labels, milestones, and releases. This
work is done in a single thread since it can be performed fast enough that we
don't need to perform this work in parallel.
### 4. Stage::ImportPullRequestsWorker
This worker will import all pull requests. For every pull request a job for the
`Gitlab::GithubImport::ImportPullRequestWorker` worker is scheduled.
### 5. Stage::ImportIssuesAndDiffNotesWorker
This worker will import all issues and pull request comments. For every issue we
schedule a job for the `Gitlab::GithubImport::ImportIssueWorker` worker. For
pull request comments we instead schedule jobs for the
`Gitlab::GithubImport::DiffNoteImporter` worker.
This worker processes both issues and diff notes in parallel so we don't need to
schedule a separate stage and wait for the previous one to complete.
Issues are imported separately from pull requests because only the "issues" API
includes labels for both issue and pull requests. Importing issues and setting
label links in the same worker removes the need for performing a separate crawl
through the API data, reducing the number of API calls necessary to import a
project.
### 6. Stage::ImportNotesWorker
This worker imports regular comments for both issues and pull requests. For
every comment we schedule a job for the
`Gitlab::GithubImport::ImportNoteWorker` worker.
Regular comments have to be imported at the end since the GitHub API used
returns comments for both issues and pull requests. This means we have to wait
for all issues and pull requests to be imported before we can import regular
comments.
### 7. Stage::FinishImportWorker
This worker will wrap up the import process by performing some housekeeping
(such as flushing any caches) and by marking the import as completed.
## Advancing stages
Advancing stages is done in one of two ways:
1. Scheduling the worker for the next stage directly.
2. Scheduling a job for `Gitlab::GithubImport::AdvanceStageWorker` which will
advance the stage when all work of the current stage has been completed.
The first approach should only be used by workers that perform all their work in
a single thread, while `AdvanceStageWorker` should be used for everything else.
The way `AdvanceStageWorker` works is fairly simple. When scheduling a job it
will be given a project ID, a list of Redis keys, and the name of the next
stage. The Redis keys (produced by `Gitlab::JobWaiter`) are used to check if the
currently running stage has been completed or not. If the stage has not yet been
completed `AdvanceStageWorker` will reschedule itself. Once a stage finishes
`AdvanceStageworker` will refresh the import JID (more on this below) and
schedule the worker of the next stage.
To reduce the number of `AdvanceStageWorker` jobs scheduled this worker will
briefly wait for jobs to complete before deciding what the next action should
be. For small projects this may slow down the import process a bit, but it will
also reduce pressure on the system as a whole.
## Refreshing import JIDs
GitLab includes a worker called `StuckImportJobsWorker` that will periodically
run and mark project imports as failed if they have been running for more than
15 hours. For GitHub projects this poses a bit of a problem: importing large
projects could take several hours depending on how often we hit the GitHub rate
limit (more on this below), but we don't want `StuckImportJobsWorker` to mark
our import as failed because of this.
To prevent this from happening we periodically refresh the expiration time of
the import process. This works by storing the JID of the import job in the
database, then refreshing this JID's TTL at various stages throughout the import
process. This is done by calling `Project#refresh_import_jid_expiration`. By
refreshing this TTL we can ensure our import does not get marked as failed so
long we're still performing work.
## GitHub rate limit
GitHub has a rate limit of 5 000 API calls per hour. The number of requests
necessary to import a project is largely dominated by the number of unique users
involved in a project (e.g. issue authors). Other data such as issue pages
and comments typically only requires a few dozen requests to import. This is
because we need the Email address of users in order to map them to GitLab users.
We handle this by doing the following:
1. Once we hit the rate limit all jobs will automatically reschedule themselves
in such a way that they are not executed until the rate limit has been reset.
2. We cache the mapping of GitHub users to GitLab users in Redis.
More information on user caching can be found below.
## Caching user lookups
When mapping GitHub users to GitLab users we need to (in the worst case)
perform:
1. One API call to get the user's Email address.
2. Two database queries to see if a corresponding GitLab user exists. One query
will try to find the user based on the GitHub user ID, while the second query
is used to find the user using their GitHub Email address.
Because this process is quite expensive we cache the result of these lookups in
Redis. For every user looked up we store three keys:
1. A Redis key mapping GitHub usernames to their Email addresses.
2. A Redis key mapping a GitHub Email addresses to a GitLab user ID.
3. A Redis key mapping a GitHub user ID to GitLab user ID.
There are two types of lookups we cache:
1. A positive lookup, meaning we found a GitLab user ID.
2. A negative lookup, meaning we didn't find a GitLab user ID. Caching this
prevents us from performing the same work for users that we know don't exist
in our GitLab database.
The expiration time of these keys is 24 hours. When retrieving the cache of a
positive lookups we refresh the TTL automatically. The TTL of false lookups is
never refreshed.
Because of this caching layer it's possible newly registered GitLab accounts
won't be linked to their corresponding GitHub accounts. This however will sort
itself out once the cached keys expire.
The user cache lookup is shared across projects. This means that the more
projects get imported the fewer GitHub API calls will be needed.
The code for this resides in:
* `lib/gitlab/github_import/user_finder.rb`
* `lib/gitlab/github_import/caching.rb`
## Mapping labels and milestones
To reduce pressure on the database we do not query it when setting labels and
milestones on issues and merge requests. Instead we cache this data when we
import labels and milestones, then we reuse this cache when assigning them to
issues/merge requests. Similar to the user lookups these cache keys are expired
automatically after 24 hours of not being used.
Unlike the user lookup caches these label and milestone caches are scoped to the
project that is being imported.
The code for this resides in:
* `lib/gitlab/github_import/label_finder.rb`
* `lib/gitlab/github_import/milestone_finder.rb`
* `lib/gitlab/github_import/caching.rb`
......@@ -24,6 +24,8 @@ constrains of a Sidekiq worker.
- the milestones (GitLab 8.7+)
- the labels (GitLab 8.7+)
- the release note descriptions (GitLab 8.12+)
- the pull request review comments (GitLab 10.2+)
- the regular issue and pull request comments
- References to pull requests and issues are preserved (GitLab 8.7+)
- Repository public access is retained. If a repository is private in GitHub
it will be created as private in GitLab as well.
......@@ -43,10 +45,13 @@ the case the namespace is taken, the repository will be imported under the user'
namespace that started the import process.
The importer will also import branches on forks of projects related to open pull
requests. These branches will be imported with a naming scheume similar to
requests. These branches will be imported with a naming scheme similar to
GH-SHA-Username/Pull-Request-number/fork-name/branch. This may lead to a discrepency
in branches compared to the GitHub Repository.
For a more technical description and an overview of the architecture you can
refer to [Working with the GitHub importer][gh-import-dev-docs].
## Importing your GitHub repositories
The importer page is visible when you create a new project.
......@@ -121,7 +126,29 @@ If you want, you can import all your GitHub projects in one go by hitting
You can also choose a different name for the project and a different namespace,
if you have the privileges to do so.
## Making the import process go faster
For large projects it may take a while to import all data. To reduce the time
necessary you can increase the number of Sidekiq workers that process the
following queues:
* `github_importer`
* `github_importer_advance_stage`
For an optimal experience we recommend having at least 4 Sidekiq processes (each
running a number of threads equal to the number of CPU cores) that _only_
process these queues. We also recommend that these processes run on separate
servers. For 4 servers with 8 cores this means you can import up to 32 objects
(e.g. issues) in parallel.
Reducing the time spent in cloning a repository can be done by increasing
network throughput, CPU capacity, and disk performance (e.g. by using high
performance SSDs) of the disks that store the Git repositories (for your GitLab
instance). Increasing the number of Sidekiq workers will _not_ reduce the time
spent cloning repositories.
[gh-import]: ../../../integration/github.md "GitHub integration"
[gh-rake]: ../../../administration/raketasks/github_import.md "GitHub rake task"
[gh-integration]: #authorize-access-to-your-repositories-using-the-github-integration
[gh-token]: #authorize-access-to-your-repositories-using-a-personal-access-token
[gh-import-dev-docs]: ../../../development/github_importer.md "Working with the GitHub importer"
......@@ -5,6 +5,10 @@ class Feature
class FlipperFeature < Flipper::Adapters::ActiveRecord::Feature
# Using `self.table_name` won't work. ActiveRecord bug?
superclass.table_name = 'features'
def self.feature_names
pluck(:key)
end
end
class FlipperGate < Flipper::Adapters::ActiveRecord::Gate
......@@ -22,11 +26,19 @@ class Feature
flipper.feature(key)
end
def persisted_names
if RequestStore.active?
RequestStore[:flipper_persisted_names] ||= FlipperFeature.feature_names
else
FlipperFeature.feature_names
end
end
def persisted?(feature)
# Flipper creates on-memory features when asked for a not-yet-created one.
# If we want to check if a feature has been actually set, we look for it
# on the persisted features list.
all.map(&:name).include?(feature.name)
persisted_names.include?(feature.name)
end
def enabled?(key, thing = nil)
......
module Github
class Client
TIMEOUT = 60
DEFAULT_PER_PAGE = 100
attr_reader :connection, :rate_limit
def initialize(options)
@connection = Faraday.new(url: options.fetch(:url, root_endpoint)) do |faraday|
faraday.options.open_timeout = options.fetch(:timeout, TIMEOUT)
faraday.options.timeout = options.fetch(:timeout, TIMEOUT)
faraday.authorization 'token', options.fetch(:token)
faraday.adapter :net_http
faraday.ssl.verify = verify_ssl
end
@rate_limit = RateLimit.new(connection)
end
def get(url, query = {})
exceed, reset_in = rate_limit.get
sleep reset_in if exceed
Github::Response.new(connection.get(url, { per_page: DEFAULT_PER_PAGE }.merge(query)))
end
private
def root_endpoint
custom_endpoint || github_endpoint
end
def custom_endpoint
github_omniauth_provider.dig('args', 'client_options', 'site')
end
def verify_ssl
# If there is no config, we're connecting to github.com
# and we should verify ssl.
github_omniauth_provider.fetch('verify_ssl', true)
end
def github_endpoint
OmniAuth::Strategies::GitHub.default_options[:client_options][:site]
end
def github_omniauth_provider
@github_omniauth_provider ||=
Gitlab.config.omniauth.providers
.find { |provider| provider.name == 'github' }
.to_h
end
end
end
module Github
class Collection
attr_reader :options
def initialize(options)
@options = options
end
def fetch(url, query = {})
return [] if url.blank?
Enumerator.new do |yielder|
loop do
response = client.get(url, query)
response.body.each { |item| yielder << item }
raise StopIteration unless response.rels.key?(:next)
url = response.rels[:next]
end
end.lazy
end
private
def client
@client ||= Github::Client.new(options)
end
end
end
module Github
RepositoryFetchError = Class.new(StandardError)
end
require_relative 'error'
require_relative 'import/issue'
require_relative 'import/legacy_diff_note'
require_relative 'import/merge_request'
require_relative 'import/note'
module Github
class Import
include Gitlab::ShellAdapter
attr_reader :project, :repository, :repo, :repo_url, :wiki_url,
:options, :errors, :cached, :verbose, :last_fetched_at
def initialize(project, options = {})
@project = project
@repository = project.repository
@repo = project.import_source
@repo_url = project.import_url
@wiki_url = project.import_url.sub(/\.git\z/, '.wiki.git')
@options = options.reverse_merge(token: project.import_data&.credentials&.fetch(:user))
@verbose = options.fetch(:verbose, false)
@cached = Hash.new { |hash, key| hash[key] = Hash.new }
@errors = []
@last_fetched_at = nil
end
# rubocop: disable Rails/Output
def execute
puts 'Fetching repository...'.color(:aqua) if verbose
setup_and_fetch_repository
puts 'Fetching labels...'.color(:aqua) if verbose
fetch_labels
puts 'Fetching milestones...'.color(:aqua) if verbose
fetch_milestones
puts 'Fetching pull requests...'.color(:aqua) if verbose
fetch_pull_requests
puts 'Fetching issues...'.color(:aqua) if verbose
fetch_issues
puts 'Fetching releases...'.color(:aqua) if verbose
fetch_releases
puts 'Cloning wiki repository...'.color(:aqua) if verbose
fetch_wiki_repository
puts 'Expiring repository cache...'.color(:aqua) if verbose
expire_repository_cache
errors.empty?
rescue Github::RepositoryFetchError
expire_repository_cache
false
ensure
keep_track_of_errors
end
private
def setup_and_fetch_repository
begin
project.ensure_repository
project.repository.add_remote('github', repo_url)
project.repository.set_import_remote_as_mirror('github')
project.repository.add_remote_fetch_config('github', '+refs/pull/*/head:refs/merge-requests/*/head')
fetch_remote(forced: true)
rescue Gitlab::Git::Repository::NoRepository,
Gitlab::Git::RepositoryMirroring::RemoteError,
Gitlab::Shell::Error => e
error(:project, repo_url, e.message)
raise Github::RepositoryFetchError
end
end
def fetch_remote(forced: false)
@last_fetched_at = Time.now
project.repository.fetch_remote('github', forced: forced)
end
def fetch_wiki_repository
return if project.wiki.repository_exists?
wiki_path = project.wiki.disk_path
gitlab_shell.import_repository(project.repository_storage_path, wiki_path, wiki_url)
rescue Gitlab::Shell::Error => e
# GitHub error message when the wiki repo has not been created,
# this means that repo has wiki enabled, but have no pages. So,
# we can skip the import.
if e.message !~ /repository not exported/
error(:wiki, wiki_url, e.message)
end
end
def fetch_labels
url = "/repos/#{repo}/labels"
while url
response = Github::Client.new(options).get(url)
response.body.each do |raw|
begin
representation = Github::Representation::Label.new(raw)
label = project.labels.find_or_create_by!(title: representation.title) do |label|
label.color = representation.color
end
cached[:label_ids][representation.title] = label.id
rescue => e
error(:label, representation.url, e.message)
end
end
url = response.rels[:next]
end
end
def fetch_milestones
url = "/repos/#{repo}/milestones"
while url
response = Github::Client.new(options).get(url, state: :all)
response.body.each do |raw|
begin
milestone = Github::Representation::Milestone.new(raw)
next if project.milestones.where(iid: milestone.iid).exists?
project.milestones.create!(
iid: milestone.iid,
title: milestone.title,
description: milestone.description,
due_date: milestone.due_date,
state: milestone.state,
created_at: milestone.created_at,
updated_at: milestone.updated_at
)
rescue => e
error(:milestone, milestone.url, e.message)
end
end
url = response.rels[:next]
end
end
def fetch_pull_requests
url = "/repos/#{repo}/pulls"
while url
response = Github::Client.new(options).get(url, state: :all, sort: :created, direction: :asc)
response.body.each do |raw|
pull_request = Github::Representation::PullRequest.new(raw, options.merge(project: project))
merge_request = MergeRequest.find_or_initialize_by(iid: pull_request.iid, source_project_id: project.id)
next unless merge_request.new_record? && pull_request.valid?
begin
# If the PR has been created/updated after we last fetched the
# remote, we fetch again to get the up-to-date refs.
fetch_remote if pull_request.updated_at > last_fetched_at
author_id = user_id(pull_request.author, project.creator_id)
description = format_description(pull_request.description, pull_request.author)
merge_request.attributes = {
iid: pull_request.iid,
title: pull_request.title,
description: description,
source_project: pull_request.source_project,
source_branch: pull_request.source_branch_name,
source_branch_sha: pull_request.source_branch_sha,
target_project: pull_request.target_project,
target_branch: pull_request.target_branch_name,
target_branch_sha: pull_request.target_branch_sha,
state: pull_request.state,
milestone_id: milestone_id(pull_request.milestone),
author_id: author_id,
assignee_id: user_id(pull_request.assignee),
created_at: pull_request.created_at,
updated_at: pull_request.updated_at
}
merge_request.save!(validate: false)
merge_request.merge_request_diffs.create
review_comments_url = "/repos/#{repo}/pulls/#{pull_request.iid}/comments"
fetch_comments(merge_request, :review_comment, review_comments_url, LegacyDiffNote)
rescue => e
error(:pull_request, pull_request.url, e.message)
end
end
url = response.rels[:next]
end
end
def fetch_issues
url = "/repos/#{repo}/issues"
while url
response = Github::Client.new(options).get(url, state: :all, sort: :created, direction: :asc)
response.body.each { |raw| populate_issue(raw) }
url = response.rels[:next]
end
end
def populate_issue(raw)
representation = Github::Representation::Issue.new(raw, options)
begin
# Every pull request is an issue, but not every issue
# is a pull request. For this reason, "shared" actions
# for both features, like manipulating assignees, labels
# and milestones, are provided within the Issues API.
if representation.pull_request?
return unless representation.labels? || representation.comments?
merge_request = MergeRequest.find_by!(target_project_id: project.id, iid: representation.iid)
if representation.labels?
merge_request.update_attribute(:label_ids, label_ids(representation.labels))
end
fetch_comments_conditionally(merge_request, representation)
else
return if Issue.exists?(iid: representation.iid, project_id: project.id)
author_id = user_id(representation.author, project.creator_id)
issue = Issue.new
issue.iid = representation.iid
issue.project_id = project.id
issue.title = representation.title
issue.description = format_description(representation.description, representation.author)
issue.state = representation.state
issue.milestone_id = milestone_id(representation.milestone)
issue.author_id = author_id
issue.created_at = representation.created_at
issue.updated_at = representation.updated_at
issue.save!(validate: false)
issue.update(
label_ids: label_ids(representation.labels),
assignee_ids: assignee_ids(representation.assignees))
fetch_comments_conditionally(issue, representation)
end
rescue => e
error(:issue, representation.url, e.message)
end
end
def fetch_comments_conditionally(issuable, representation)
if representation.comments?
comments_url = "/repos/#{repo}/issues/#{issuable.iid}/comments"
fetch_comments(issuable, :comment, comments_url)
end
end
def fetch_comments(noteable, type, url, klass = Note)
while url
comments = Github::Client.new(options).get(url)
ActiveRecord::Base.no_touching do
comments.body.each do |raw|
begin
representation = Github::Representation::Comment.new(raw, options)
author_id = user_id(representation.author, project.creator_id)
note = klass.new
note.project_id = project.id
note.noteable = noteable
note.note = format_description(representation.note, representation.author)
note.commit_id = representation.commit_id
note.line_code = representation.line_code
note.author_id = author_id
note.created_at = representation.created_at
note.updated_at = representation.updated_at
note.save!(validate: false)
rescue => e
error(type, representation.url, e.message)
end
end
end
url = comments.rels[:next]
end
end
def fetch_releases
url = "/repos/#{repo}/releases"
while url
response = Github::Client.new(options).get(url)
response.body.each do |raw|
representation = Github::Representation::Release.new(raw)
next unless representation.valid?
release = ::Release.find_or_initialize_by(project_id: project.id, tag: representation.tag)
next unless release.new_record?
begin
release.description = representation.description
release.created_at = representation.created_at
release.updated_at = representation.updated_at
release.save!(validate: false)
rescue => e
error(:release, representation.url, e.message)
end
end
url = response.rels[:next]
end
end
def label_ids(labels)
labels.map { |label| cached[:label_ids][label.title] }.compact
end
def assignee_ids(assignees)
assignees.map { |assignee| user_id(assignee) }.compact
end
def milestone_id(milestone)
return unless milestone.present?
project.milestones.select(:id).find_by(iid: milestone.iid)&.id
end
def user_id(user, fallback_id = nil)
return unless user.present?
return cached[:user_ids][user.id] if cached[:user_ids][user.id].present?
gitlab_user_id = user_id_by_external_uid(user.id) || user_id_by_email(user.email)
cached[:gitlab_user_ids][user.id] = gitlab_user_id.present?
cached[:user_ids][user.id] = gitlab_user_id || fallback_id
end
def user_id_by_email(email)
return nil unless email
::User.find_by_any_email(email)&.id
end
def user_id_by_external_uid(id)
return nil unless id
::User.select(:id)
.joins(:identities)
.merge(::Identity.where(provider: :github, extern_uid: id))
.first&.id
end
def format_description(body, author)
return body if cached[:gitlab_user_ids][author.id]
"*Created by: #{author.username}*\n\n#{body}"
end
def expire_repository_cache
repository.expire_content_cache if project.repository_exists?
end
def keep_track_of_errors
return unless errors.any?
project.update_column(:import_error, {
message: 'The remote data could not be fully imported.',
errors: errors
}.to_json)
end
def error(type, url, message)
errors << { type: type, url: Gitlab::UrlSanitizer.sanitize(url), error: message }
end
end
end
module Github
class Import
class Issue < ::Issue
self.table_name = 'issues'
self.reset_callbacks :save
self.reset_callbacks :create
self.reset_callbacks :commit
self.reset_callbacks :update
self.reset_callbacks :validate
end
end
end
module Github
class Import
class LegacyDiffNote < ::LegacyDiffNote
self.table_name = 'notes'
self.store_full_sti_class = false
self.reset_callbacks :commit
self.reset_callbacks :update
self.reset_callbacks :validate
end
end
end
module Github
class Import
class MergeRequest < ::MergeRequest
self.table_name = 'merge_requests'
self.reset_callbacks :create
self.reset_callbacks :save
self.reset_callbacks :commit
self.reset_callbacks :update
self.reset_callbacks :validate
end
end
end
module Github
class Import
class Note < ::Note
self.table_name = 'notes'
self.store_full_sti_class = false
self.reset_callbacks :save
self.reset_callbacks :commit
self.reset_callbacks :update
self.reset_callbacks :validate
end
end
end
module Github
class RateLimit
SAFE_REMAINING_REQUESTS = 100
SAFE_RESET_TIME = 500
RATE_LIMIT_URL = '/rate_limit'.freeze
attr_reader :connection
def initialize(connection)
@connection = connection
end
def get
response = connection.get(RATE_LIMIT_URL)
# GitHub Rate Limit API returns 404 when the rate limit is disabled
return false unless response.status != 404
body = Oj.load(response.body, class_cache: false, mode: :compat)
remaining = body.dig('rate', 'remaining').to_i
reset_in = body.dig('rate', 'reset').to_i
exceed = remaining <= SAFE_REMAINING_REQUESTS
[exceed, reset_in]
end
end
end
module Github
class Repositories
attr_reader :options
def initialize(options)
@options = options
end
def fetch
Collection.new(options).fetch(repos_url)
end
private
def repos_url
'/user/repos'
end
end
end
module Github
module Representation
class Base
def initialize(raw, options = {})
@raw = raw
@options = options
end
def id
raw['id']
end
def url
raw['url']
end
def created_at
raw['created_at']
end
def updated_at
raw['updated_at']
end
private
attr_reader :raw, :options
end
end
end
module Github
module Representation
class Branch < Representation::Base
attr_reader :repository
def user
raw.dig('user', 'login') || 'unknown'
end
def repo?
raw['repo'].present?
end
def repo
return unless repo?
@repo ||= Github::Representation::Repo.new(raw['repo'])
end
def ref
raw['ref']
end
def sha
raw['sha']
end
def short_sha
Commit.truncate_sha(sha)
end
def valid?
sha.present? && ref.present?
end
def restore!(name)
repository.create_branch(name, sha)
rescue Gitlab::Git::Repository::InvalidRef => e
Rails.logger.error("#{self.class.name}: Could not restore branch #{name}: #{e}")
end
def remove!(name)
repository.delete_branch(name)
rescue Gitlab::Git::Repository::DeleteBranchError => e
Rails.logger.error("#{self.class.name}: Could not remove branch #{name}: #{e}")
end
private
def repository
@repository ||= options.fetch(:repository)
end
end
end
end
module Github
module Representation
class Comment < Representation::Base
def note
raw['body'] || ''
end
def author
@author ||= Github::Representation::User.new(raw['user'], options)
end
def commit_id
raw['commit_id']
end
def line_code
return unless on_diff?
parsed_lines = Gitlab::Diff::Parser.new.parse(diff_hunk.lines)
generate_line_code(parsed_lines.to_a.last)
end
private
def generate_line_code(line)
Gitlab::Git.diff_line_code(file_path, line.new_pos, line.old_pos)
end
def on_diff?
diff_hunk.present?
end
def diff_hunk
raw['diff_hunk']
end
def file_path
raw['path']
end
end
end
end
module Github
module Representation
class Issuable < Representation::Base
def iid
raw['number']
end
def title
raw['title']
end
def description
raw['body'] || ''
end
def milestone
return unless raw['milestone'].present?
@milestone ||= Github::Representation::Milestone.new(raw['milestone'])
end
def author
@author ||= Github::Representation::User.new(raw['user'], options)
end
def labels?
raw['labels'].any?
end
def labels
@labels ||= Array(raw['labels']).map do |label|
Github::Representation::Label.new(label, options)
end
end
end
end
end
module Github
module Representation
class Issue < Representation::Issuable
def state
raw['state'] == 'closed' ? 'closed' : 'opened'
end
def comments?
raw['comments'] > 0
end
def pull_request?
raw['pull_request'].present?
end
def assigned?
raw['assignees'].present?
end
def assignees
@assignees ||= Array(raw['assignees']).map do |user|
Github::Representation::User.new(user, options)
end
end
end
end
end
module Github
module Representation
class Label < Representation::Base
def color
"##{raw['color']}"
end
def title
raw['name']
end
end
end
end
module Github
module Representation
class Milestone < Representation::Base
def iid
raw['number']
end
def title
raw['title']
end
def description
raw['description']
end
def due_date
raw['due_on']
end
def state
raw['state'] == 'closed' ? 'closed' : 'active'
end
end
end
end
module Github
module Representation
class PullRequest < Representation::Issuable
delegate :sha, to: :source_branch, prefix: true
delegate :sha, to: :target_branch, prefix: true
def source_project
project
end
def source_branch_name
# Mimic the "user:branch" displayed in the MR widget,
# i.e. "Request to merge rymai:add-external-mounts into master"
cross_project? ? "#{source_branch.user}:#{source_branch.ref}" : source_branch.ref
end
def target_project
project
end
def target_branch_name
target_branch.ref
end
def state
return 'merged' if raw['state'] == 'closed' && raw['merged_at'].present?
return 'closed' if raw['state'] == 'closed'
'opened'
end
def opened?
state == 'opened'
end
def valid?
source_branch.valid? && target_branch.valid?
end
def assigned?
raw['assignee'].present?
end
def assignee
return unless assigned?
@assignee ||= Github::Representation::User.new(raw['assignee'], options)
end
private
def project
@project ||= options.fetch(:project)
end
def source_branch
@source_branch ||= Representation::Branch.new(raw['head'], repository: project.repository)
end
def target_branch
@target_branch ||= Representation::Branch.new(raw['base'], repository: project.repository)
end
def cross_project?
return true unless source_branch.repo?
source_branch.repo.id != target_branch.repo.id
end
end
end
end
module Github
module Representation
class Release < Representation::Base
def description
raw['body']
end
def tag
raw['tag_name']
end
def valid?
!raw['draft']
end
end
end
end
module Github
module Representation
class Repo < Representation::Base
end
end
end
module Github
module Representation
class User < Representation::Base
def email
return @email if defined?(@email)
@email = Github::User.new(username, options).get.fetch('email', nil)
end
def username
raw['login']
end
end
end
end
module Github
class Response
attr_reader :raw, :headers, :status
def initialize(response)
@raw = response
@headers = response.headers
@status = response.status
end
def body
Oj.load(raw.body, class_cache: false, mode: :compat)
end
def rels
links = headers['Link'].to_s.split(', ').map do |link|
href, name = link.match(/<(.*?)>; rel="(\w+)"/).captures
[name.to_sym, href]
end
Hash[*links.flatten]
end
end
end
module Github
class User
attr_reader :username, :options
def initialize(username, options)
@username = username
@options = options
end
def get
client.get(user_url).body
end
private
def client
@client ||= Github::Client.new(options)
end
def user_url
"/users/#{username}"
end
end
end
......@@ -108,20 +108,41 @@ module Gitlab
end
end
def self.bulk_insert(table, rows)
# Bulk inserts a number of rows into a table, optionally returning their
# IDs.
#
# table - The name of the table to insert the rows into.
# rows - An Array of Hash instances, each mapping the columns to their
# values.
# return_ids - When set to true the return value will be an Array of IDs of
# the inserted rows, this only works on PostgreSQL.
def self.bulk_insert(table, rows, return_ids: false)
return if rows.empty?
keys = rows.first.keys
columns = keys.map { |key| connection.quote_column_name(key) }
return_ids = false if mysql?
tuples = rows.map do |row|
row.values_at(*keys).map { |value| connection.quote(value) }
end
connection.execute <<-EOF
sql = <<-EOF
INSERT INTO #{table} (#{columns.join(', ')})
VALUES #{tuples.map { |tuple| "(#{tuple.join(', ')})" }.join(', ')}
EOF
if return_ids
sql << 'RETURNING id'
end
result = connection.execute(sql)
if return_ids
result.values.map { |tuple| tuple[0].to_i }
else
[]
end
end
def self.sanitize_timestamp(timestamp)
......
......@@ -920,6 +920,11 @@ module Gitlab
false
end
# Returns true if a remote exists.
def remote_exists?(name)
rugged.remotes[name].present?
end
# Update the specified remote using the values in the +options+ hash
#
# Example
......
module Gitlab
module GithubImport
def self.new_client_for(project, token: nil, parallel: true)
token_to_use = token || project.import_data&.credentials&.fetch(:user)
Client.new(token_to_use, parallel: parallel)
end
# Inserts a raw row and returns the ID of the inserted row.
#
# attributes - The attributes/columns to set.
# relation - An ActiveRecord::Relation to use for finding the ID of the row
# when using MySQL.
def self.insert_and_return_id(attributes, relation)
# We use bulk_insert here so we can bypass any queries executed by
# callbacks or validation rules, as doing this wouldn't scale when
# importing very large projects.
result = Gitlab::Database
.bulk_insert(relation.table_name, [attributes], return_ids: true)
# MySQL doesn't support returning the IDs of a bulk insert in a way that
# is not a pain, so in this case we'll issue an extra query instead.
result.first ||
relation.where(iid: attributes[:iid]).limit(1).pluck(:id).first
end
# Returns the ID of the ghost user.
def self.ghost_user_id
key = 'github-import/ghost-user-id'
Caching.read_integer(key) || Caching.write(key, User.select(:id).ghost.id)
end
end
end
# frozen_string_literal: true
module Gitlab
module GithubImport
module BulkImporting
# Builds and returns an Array of objects to bulk insert into the
# database.
#
# enum - An Enumerable that returns the objects to turn into database
# rows.
def build_database_rows(enum)
enum.each_with_object([]) do |(object, _), rows|
rows << build(object) unless already_imported?(object)
end
end
# Bulk inserts the given rows into the database.
def bulk_insert(model, rows, batch_size: 100)
rows.each_slice(batch_size) do |slice|
Gitlab::Database.bulk_insert(model.table_name, slice)
end
end
end
end
end
# frozen_string_literal: true
module Gitlab
module GithubImport
module Caching
# The default timeout of the cache keys.
TIMEOUT = 24.hours.to_i
WRITE_IF_GREATER_SCRIPT = <<-EOF.strip_heredoc.freeze
local key, value, ttl = KEYS[1], tonumber(ARGV[1]), ARGV[2]
local existing = tonumber(redis.call("get", key))
if existing == nil or value > existing then
redis.call("set", key, value)
redis.call("expire", key, ttl)
return true
else
return false
end
EOF
# Reads a cache key.
#
# If the key exists and has a non-empty value its TTL is refreshed
# automatically.
#
# raw_key - The cache key to read.
# timeout - The new timeout of the key if the key is to be refreshed.
def self.read(raw_key, timeout: TIMEOUT)
key = cache_key_for(raw_key)
value = Redis::Cache.with { |redis| redis.get(key) }
if value.present?
# We refresh the expiration time so frequently used keys stick
# around, removing the need for querying the database as much as
# possible.
#
# A key may be empty when we looked up a GitHub user (for example) but
# did not find a matching GitLab user. In that case we _don't_ want to
# refresh the TTL so we automatically pick up the right data when said
# user were to register themselves on the GitLab instance.
Redis::Cache.with { |redis| redis.expire(key, timeout) }
end
value
end
# Reads an integer from the cache, or returns nil if no value was found.
#
# See Caching.read for more information.
def self.read_integer(raw_key, timeout: TIMEOUT)
value = read(raw_key, timeout: timeout)
value.to_i if value.present?
end
# Sets a cache key to the given value.
#
# key - The cache key to write.
# value - The value to set.
# timeout - The time after which the cache key should expire.
def self.write(raw_key, value, timeout: TIMEOUT)
key = cache_key_for(raw_key)
Redis::Cache.with do |redis|
redis.set(key, value, ex: timeout)
end
value
end
# Adds a value to a set.
#
# raw_key - The key of the set to add the value to.
# value - The value to add to the set.
# timeout - The new timeout of the key.
def self.set_add(raw_key, value, timeout: TIMEOUT)
key = cache_key_for(raw_key)
Redis::Cache.with do |redis|
redis.multi do |m|
m.sadd(key, value)
m.expire(key, timeout)
end
end
end
# Returns true if the given value is present in the set.
#
# raw_key - The key of the set to check.
# value - The value to check for.
def self.set_includes?(raw_key, value)
key = cache_key_for(raw_key)
Redis::Cache.with do |redis|
redis.sismember(key, value)
end
end
# Sets multiple keys to a given value.
#
# mapping - A Hash mapping the cache keys to their values.
# timeout - The time after which the cache key should expire.
def self.write_multiple(mapping, timeout: TIMEOUT)
Redis::Cache.with do |redis|
redis.multi do |multi|
mapping.each do |raw_key, value|
multi.set(cache_key_for(raw_key), value, ex: timeout)
end
end
end
end
# Sets the expiration time of a key.
#
# raw_key - The key for which to change the timeout.
# timeout - The new timeout.
def self.expire(raw_key, timeout)
key = cache_key_for(raw_key)
Redis::Cache.with do |redis|
redis.expire(key, timeout)
end
end
# Sets a key to the given integer but only if the existing value is
# smaller than the given value.
#
# This method uses a Lua script to ensure the read and write are atomic.
#
# raw_key - The key to set.
# value - The new value for the key.
# timeout - The key timeout in seconds.
#
# Returns true when the key was overwritten, false otherwise.
def self.write_if_greater(raw_key, value, timeout: TIMEOUT)
key = cache_key_for(raw_key)
val = Redis::Cache.with do |redis|
redis
.eval(WRITE_IF_GREATER_SCRIPT, keys: [key], argv: [value, timeout])
end
val ? true : false
end
def self.cache_key_for(raw_key)
"#{Redis::Cache::CACHE_NAMESPACE}:#{raw_key}"
end
end
end
end
# frozen_string_literal: true
module Gitlab
module GithubImport
# HTTP client for interacting with the GitHub API.
#
# This class is basically a fancy wrapped around Octokit while adding some
# functionality to deal with rate limiting and parallel imports. Usage is
# mostly the same as Octokit, for example:
#
# client = GithubImport::Client.new('hunter2')
#
# client.labels.each do |label|
# puts label.name
# end
class Client
GITHUB_SAFE_REMAINING_REQUESTS = 100
GITHUB_SAFE_SLEEP_TIME = 500
attr_reader :octokit
attr_reader :access_token, :host, :api_version
# A single page of data and the corresponding page number.
Page = Struct.new(:objects, :number)
def initialize(access_token, host: nil, api_version: 'v3')
@access_token = access_token
@host = host.to_s.sub(%r{/+\z}, '')
@api_version = api_version
@users = {}
# The minimum number of requests we want to keep available.
#
# We don't use a value of 0 as multiple threads may be using the same
# token in parallel. This could result in all of them hitting the GitHub
# rate limit at once. The threshold is put in place to not hit the limit
# in most cases.
RATE_LIMIT_THRESHOLD = 50
if access_token
::Octokit.auto_paginate = false
end
# token - The GitHub API token to use.
#
# per_page - The number of objects that should be displayed per page.
#
# parallel - When set to true hitting the rate limit will result in a
# dedicated error being raised. When set to `false` we will
# instead just `sleep()` until the rate limit is reset. Setting
# this value to `true` for parallel importing is crucial as
# otherwise hitting the rate limit will result in a thread
# being blocked in a `sleep()` call for up to an hour.
def initialize(token, per_page: 100, parallel: true)
@octokit = Octokit::Client.new(access_token: token, per_page: per_page)
@parallel = parallel
end
def api
@api ||= ::Octokit::Client.new(
access_token: access_token,
api_endpoint: api_endpoint,
# If there is no config, we're connecting to github.com and we
# should verify ssl.
connection_options: {
ssl: { verify: config ? config['verify_ssl'] : true }
}
)
def parallel?
@parallel
end
def client
unless config
raise Projects::ImportService::Error,
'OAuth configuration for GitHub missing.'
end
@client ||= ::OAuth2::Client.new(
config.app_id,
config.app_secret,
github_options.merge(ssl: { verify: config['verify_ssl'] })
)
# Returns the details of a GitHub user.
#
# username - The username of the user.
def user(username)
with_rate_limit { octokit.user(username) }
end
def authorize_url(redirect_uri)
client.auth_code.authorize_url({
redirect_uri: redirect_uri,
scope: "repo, user, user:email"
})
# Returns the details of a GitHub repository.
#
# name - The path (in the form `owner/repository`) of the repository.
def repository(name)
with_rate_limit { octokit.repo(name) }
end
def get_token(code)
client.auth_code.get_token(code).token
def labels(*args)
each_object(:labels, *args)
end
def method_missing(method, *args, &block)
if api.respond_to?(method)
request(method, *args, &block)
else
super(method, *args, &block)
end
def milestones(*args)
each_object(:milestones, *args)
end
def respond_to?(method)
api.respond_to?(method) || super
def releases(*args)
each_object(:releases, *args)
end
def user(login)
return nil unless login.present?
return @users[login] if @users.key?(login)
# Fetches data from the GitHub API and yields a Page object for every page
# of data, without loading all of them into memory.
#
# method - The Octokit method to use for getting the data.
# args - Arguments to pass to the Octokit method.
#
# rubocop: disable GitlabSecurity/PublicSend
def each_page(method, *args, &block)
return to_enum(__method__, method, *args) unless block_given?
@users[login] = api.user(login)
end
page =
if args.last.is_a?(Hash) && args.last[:page]
args.last[:page]
else
1
end
private
collection = with_rate_limit { octokit.public_send(method, *args) }
next_url = octokit.last_response.rels[:next]
def api_endpoint
if host.present? && api_version.present?
"#{host}/api/#{api_version}"
else
github_options[:site]
yield Page.new(collection, page)
while next_url
response = with_rate_limit { next_url.get }
next_url = response.rels[:next]
yield Page.new(response.data, page += 1)
end
end
def config
Gitlab.config.omniauth.providers.find { |provider| provider.name == "github" }
end
# Iterates over all of the objects for the given method (e.g. `:labels`).
#
# method - The method to send to Octokit for querying data.
# args - Any arguments to pass to the Octokit method.
def each_object(method, *args, &block)
return to_enum(__method__, method, *args) unless block_given?
def github_options
if config
config["args"]["client_options"].deep_symbolize_keys
else
OmniAuth::Strategies::GitHub.default_options[:client_options].symbolize_keys
each_page(method, *args) do |page|
page.objects.each do |object|
yield object
end
end
end
def rate_limit
api.rate_limit!
# GitHub Rate Limit API returns 404 when the rate limit is
# disabled. In this case we just want to return gracefully
# instead of spitting out an error.
rescue Octokit::NotFound
nil
end
# Yields the supplied block, responding to any rate limit errors.
#
# The exact strategy used for handling rate limiting errors depends on
# whether we are running in parallel mode or not. For more information see
# `#rate_or_wait_for_rate_limit`.
def with_rate_limit
request_count_counter.increment
def has_rate_limit?
return @has_rate_limit if defined?(@has_rate_limit)
raise_or_wait_for_rate_limit unless requests_remaining?
@has_rate_limit = rate_limit.present?
end
begin
yield
rescue Octokit::TooManyRequests
raise_or_wait_for_rate_limit
def rate_limit_exceed?
has_rate_limit? && rate_limit.remaining <= GITHUB_SAFE_REMAINING_REQUESTS
# This retry will only happen when running in sequential mode as we'll
# raise an error in parallel mode.
retry
end
end
def rate_limit_sleep_time
rate_limit.resets_in + GITHUB_SAFE_SLEEP_TIME
# Returns `true` if we're still allowed to perform API calls.
def requests_remaining?
remaining_requests > RATE_LIMIT_THRESHOLD
end
def request(method, *args, &block)
sleep rate_limit_sleep_time if rate_limit_exceed?
data = api.__send__(method, *args) # rubocop:disable GitlabSecurity/PublicSend
return data unless data.is_a?(Array)
def remaining_requests
octokit.rate_limit.remaining
end
last_response = api.last_response
def raise_or_wait_for_rate_limit
rate_limit_counter.increment
if block_given?
yield data
# api.last_response could change while we're yielding (e.g. fetching labels for each PR)
# so we cache our own last response
each_response_page(last_response, &block)
if parallel?
raise RateLimitError
else
each_response_page(last_response) { |page| data.concat(page) }
data
sleep(rate_limit_resets_in)
end
end
def each_response_page(last_response)
while last_response.rels[:next]
sleep rate_limit_sleep_time if rate_limit_exceed?
last_response = last_response.rels[:next].get
yield last_response.data if last_response.data.is_a?(Array)
end
def rate_limit_resets_in
# We add a few seconds to the rate limit so we don't _immediately_
# resume when the rate limit resets as this may result in us performing
# a request before GitHub has a chance to reset the limit.
octokit.rate_limit.resets_in + 5
end
def respond_to_missing?(method, include_private = false)
octokit.respond_to?(method, include_private)
end
def rate_limit_counter
@rate_limit_counter ||= Gitlab::Metrics.counter(
:github_importer_rate_limit_hits,
'The number of times we hit the GitHub rate limit when importing projects'
)
end
def request_count_counter
@request_counter ||= Gitlab::Metrics.counter(
:github_importer_request_count,
'The number of GitHub API calls performed when importing projects'
)
end
end
end
......
# frozen_string_literal: true
module Gitlab
module GithubImport
module Importer
class DiffNoteImporter
attr_reader :note, :project, :client, :user_finder
# note - An instance of `Gitlab::GithubImport::Representation::DiffNote`.
# project - An instance of `Project`.
# client - An instance of `Gitlab::GithubImport::Client`.
def initialize(note, project, client)
@note = note
@project = project
@client = client
@user_finder = UserFinder.new(project, client)
end
def execute
return unless (mr_id = find_merge_request_id)
author_id, author_found = user_finder.author_id_for(note)
note_body =
MarkdownText.format(note.note, note.author, author_found)
attributes = {
noteable_type: 'MergeRequest',
noteable_id: mr_id,
project_id: project.id,
author_id: author_id,
note: note_body,
system: false,
commit_id: note.commit_id,
line_code: note.line_code,
type: 'LegacyDiffNote',
created_at: note.created_at,
updated_at: note.updated_at,
st_diff: note.diff_hash.to_yaml
}
# It's possible that during an import we'll insert tens of thousands
# of diff notes. If we were to use the Note/LegacyDiffNote model here
# we'd also have to run additional queries for both validations and
# callbacks, putting a lot of pressure on the database.
#
# To work around this we're using bulk_insert with a single row. This
# allows us to efficiently insert data (even if it's just 1 row)
# without having to use all sorts of hacks to disable callbacks.
Gitlab::Database.bulk_insert(LegacyDiffNote.table_name, [attributes])
rescue ActiveRecord::InvalidForeignKey
# It's possible the project and the issue have been deleted since
# scheduling this job. In this case we'll just skip creating the note.
end
# Returns the ID of the merge request this note belongs to.
def find_merge_request_id
GithubImport::IssuableFinder.new(project, note).database_id
end
end
end
end
end
# frozen_string_literal: true
module Gitlab
module GithubImport
module Importer
class DiffNotesImporter
include ParallelScheduling
def representation_class
Representation::DiffNote
end
def importer_class
DiffNoteImporter
end
def sidekiq_worker_class
ImportDiffNoteWorker
end
def collection_method
:pull_requests_comments
end
def id_for_already_imported_cache(note)
note.id
end
end
end
end
end
# frozen_string_literal: true
module Gitlab
module GithubImport
module Importer
class IssueAndLabelLinksImporter
attr_reader :issue, :project, :client
# issue - An instance of `Gitlab::GithubImport::Representation::Issue`.
# project - An instance of `Project`
# client - An instance of `Gitlab::GithubImport::Client`
def initialize(issue, project, client)
@issue = issue
@project = project
@client = client
end
def execute
IssueImporter.import_if_issue(issue, project, client)
LabelLinksImporter.new(issue, project, client).execute
end
end
end
end
end
# frozen_string_literal: true
module Gitlab
module GithubImport
module Importer
class IssueImporter
attr_reader :project, :issue, :client, :user_finder, :milestone_finder,
:issuable_finder
# Imports an issue if it's a regular issue and not a pull request.
def self.import_if_issue(issue, project, client)
new(issue, project, client).execute unless issue.pull_request?
end
# issue - An instance of `Gitlab::GithubImport::Representation::Issue`.
# project - An instance of `Project`
# client - An instance of `Gitlab::GithubImport::Client`
def initialize(issue, project, client)
@issue = issue
@project = project
@client = client
@user_finder = UserFinder.new(project, client)
@milestone_finder = MilestoneFinder.new(project)
@issuable_finder = GithubImport::IssuableFinder.new(project, issue)
end
def execute
Issue.transaction do
if (issue_id = create_issue)
create_assignees(issue_id)
issuable_finder.cache_database_id(issue_id)
end
end
end
# Creates a new GitLab issue for the current GitHub issue.
#
# Returns the ID of the created issue as an Integer. If the issue
# couldn't be created this method will return `nil` instead.
def create_issue
author_id, author_found = user_finder.author_id_for(issue)
description =
MarkdownText.format(issue.description, issue.author, author_found)
attributes = {
iid: issue.iid,
title: issue.truncated_title,
author_id: author_id,
project_id: project.id,
description: description,
milestone_id: milestone_finder.id_for(issue),
state: issue.state,
created_at: issue.created_at,
updated_at: issue.updated_at
}
GithubImport.insert_and_return_id(attributes, project.issues)
rescue ActiveRecord::InvalidForeignKey
# It's possible the project has been deleted since scheduling this
# job. In this case we'll just skip creating the issue.
end
# Stores all issue assignees in the database.
#
# issue_id - The ID of the created issue.
def create_assignees(issue_id)
assignees = []
issue.assignees.each do |assignee|
if (user_id = user_finder.user_id_for(assignee))
assignees << { issue_id: issue_id, user_id: user_id }
end
end
Gitlab::Database.bulk_insert(IssueAssignee.table_name, assignees)
end
end
end
end
end
# frozen_string_literal: true
module Gitlab
module GithubImport
module Importer
class IssuesImporter
include ParallelScheduling
def importer_class
IssueAndLabelLinksImporter
end
def representation_class
Representation::Issue
end
def sidekiq_worker_class
ImportIssueWorker
end
def collection_method
:issues
end
def id_for_already_imported_cache(issue)
issue.number
end
def collection_options
{ state: 'all', sort: 'created', direction: 'asc' }
end
end
end
end
end
# frozen_string_literal: true
module Gitlab
module GithubImport
module Importer
class LabelLinksImporter
attr_reader :issue, :project, :client, :label_finder
# issue - An instance of `Gitlab::GithubImport::Representation::Issue`
# project - An instance of `Project`
# client - An instance of `Gitlab::GithubImport::Client`
def initialize(issue, project, client)
@issue = issue
@project = project
@client = client
@label_finder = LabelFinder.new(project)
end
def execute
create_labels
end
def create_labels
time = Time.zone.now
rows = []
target_id = find_target_id
issue.label_names.each do |label_name|
# Although unlikely it's technically possible for an issue to be
# given a label that was created and assigned after we imported all
# the project's labels.
next unless (label_id = label_finder.id_for(label_name))
rows << {
label_id: label_id,
target_id: target_id,
target_type: issue.issuable_type,
created_at: time,
updated_at: time
}
end
Gitlab::Database.bulk_insert(LabelLink.table_name, rows)
end
def find_target_id
GithubImport::IssuableFinder.new(project, issue).database_id
end
end
end
end
end
# frozen_string_literal: true
module Gitlab
module GithubImport
module Importer
class LabelsImporter
include BulkImporting
attr_reader :project, :client, :existing_labels
# project - An instance of `Project`.
# client - An instance of `Gitlab::GithubImport::Client`.
def initialize(project, client)
@project = project
@client = client
@existing_labels = project.labels.pluck(:title).to_set
end
def execute
bulk_insert(Label, build_labels)
build_labels_cache
end
def build_labels
build_database_rows(each_label)
end
def already_imported?(label)
existing_labels.include?(label.name)
end
def build_labels_cache
LabelFinder.new(project).build_cache
end
def build(label)
time = Time.zone.now
{
title: label.name,
color: '#' + label.color,
project_id: project.id,
type: 'ProjectLabel',
created_at: time,
updated_at: time
}
end
def each_label
client.labels(project.import_source)
end
end
end
end
end
# frozen_string_literal: true
module Gitlab
module GithubImport
module Importer
class MilestonesImporter
include BulkImporting
attr_reader :project, :client, :existing_milestones
# project - An instance of `Project`
# client - An instance of `Gitlab::GithubImport::Client`
def initialize(project, client)
@project = project
@client = client
@existing_milestones = project.milestones.pluck(:iid).to_set
end
def execute
bulk_insert(Milestone, build_milestones)
build_milestones_cache
end
def build_milestones
build_database_rows(each_milestone)
end
def already_imported?(milestone)
existing_milestones.include?(milestone.number)
end
def build_milestones_cache
MilestoneFinder.new(project).build_cache
end
def build(milestone)
{
iid: milestone.number,
title: milestone.title,
description: milestone.description,
project_id: project.id,
state: state_for(milestone),
created_at: milestone.created_at,
updated_at: milestone.updated_at
}
end
def state_for(milestone)
milestone.state == 'open' ? :active : :closed
end
def each_milestone
client.milestones(project.import_source, state: 'all')
end
end
end
end
end
# frozen_string_literal: true
module Gitlab
module GithubImport
module Importer
class NoteImporter
attr_reader :note, :project, :client, :user_finder
# note - An instance of `Gitlab::GithubImport::Representation::Note`.
# project - An instance of `Project`.
# client - An instance of `Gitlab::GithubImport::Client`.
def initialize(note, project, client)
@note = note
@project = project
@client = client
@user_finder = UserFinder.new(project, client)
end
def execute
return unless (noteable_id = find_noteable_id)
author_id, author_found = user_finder.author_id_for(note)
note_body =
MarkdownText.format(note.note, note.author, author_found)
attributes = {
noteable_type: note.noteable_type,
noteable_id: noteable_id,
project_id: project.id,
author_id: author_id,
note: note_body,
system: false,
created_at: note.created_at,
updated_at: note.updated_at
}
# We're using bulk_insert here so we can bypass any validations and
# callbacks. Running these would result in a lot of unnecessary SQL
# queries being executed when importing large projects.
Gitlab::Database.bulk_insert(Note.table_name, [attributes])
rescue ActiveRecord::InvalidForeignKey
# It's possible the project and the issue have been deleted since
# scheduling this job. In this case we'll just skip creating the note.
end
# Returns the ID of the issue or merge request to create the note for.
def find_noteable_id
GithubImport::IssuableFinder.new(project, note).database_id
end
end
end
end
end
# frozen_string_literal: true
module Gitlab
module GithubImport
module Importer
class NotesImporter
include ParallelScheduling
def importer_class
NoteImporter
end
def representation_class
Representation::Note
end
def sidekiq_worker_class
ImportNoteWorker
end
def collection_method
:issues_comments
end
def id_for_already_imported_cache(note)
note.id
end
end
end
end
end
# frozen_string_literal: true
module Gitlab
module GithubImport
module Importer
class PullRequestImporter
attr_reader :pull_request, :project, :client, :user_finder,
:milestone_finder, :issuable_finder
# pull_request - An instance of
# `Gitlab::GithubImport::Representation::PullRequest`.
# project - An instance of `Project`
# client - An instance of `Gitlab::GithubImport::Client`
def initialize(pull_request, project, client)
@pull_request = pull_request
@project = project
@client = client
@user_finder = UserFinder.new(project, client)
@milestone_finder = MilestoneFinder.new(project)
@issuable_finder =
GithubImport::IssuableFinder.new(project, pull_request)
end
def execute
if (mr_id = create_merge_request)
issuable_finder.cache_database_id(mr_id)
end
end
# Creates the merge request and returns its ID.
#
# This method will return `nil` if the merge request could not be
# created.
def create_merge_request
author_id, author_found = user_finder.author_id_for(pull_request)
description = MarkdownText
.format(pull_request.description, pull_request.author, author_found)
# This work must be wrapped in a transaction as otherwise we can leave
# behind incomplete data in the event of an error. This can then lead
# to duplicate key errors when jobs are retried.
MergeRequest.transaction do
attributes = {
iid: pull_request.iid,
title: pull_request.truncated_title,
description: description,
source_project_id: project.id,
target_project_id: project.id,
source_branch: pull_request.formatted_source_branch,
target_branch: pull_request.target_branch,
state: pull_request.state,
milestone_id: milestone_finder.id_for(pull_request),
author_id: author_id,
assignee_id: user_finder.assignee_id_for(pull_request),
created_at: pull_request.created_at,
updated_at: pull_request.updated_at
}
# When creating merge requests there are a lot of hooks that may
# run, for many different reasons. Many of these hooks (e.g. the
# ones used for rendering Markdown) are completely unnecessary and
# may even lead to transaction timeouts.
#
# To ensure importing pull requests has a minimal impact and can
# complete in a reasonable time we bypass all the hooks by inserting
# the row and then retrieving it. We then only perform the
# additional work that is strictly necessary.
merge_request_id = GithubImport
.insert_and_return_id(attributes, project.merge_requests)
merge_request = project.merge_requests.find(merge_request_id)
# These fields are set so we can create the correct merge request
# diffs.
merge_request.source_branch_sha = pull_request.source_branch_sha
merge_request.target_branch_sha = pull_request.target_branch_sha
merge_request.keep_around_commit
merge_request.merge_request_diffs.create
merge_request.id
end
rescue ActiveRecord::InvalidForeignKey
# It's possible the project has been deleted since scheduling this
# job. In this case we'll just skip creating the merge request.
end
end
end
end
end
# frozen_string_literal: true
module Gitlab
module GithubImport
module Importer
class PullRequestsImporter
include ParallelScheduling
def importer_class
PullRequestImporter
end
def representation_class
Representation::PullRequest
end
def sidekiq_worker_class
ImportPullRequestWorker
end
def id_for_already_imported_cache(pr)
pr.number
end
def each_object_to_import
super do |pr|
update_repository if update_repository?(pr)
yield pr
end
end
def update_repository
# We set this column _before_ fetching the repository, and this is
# deliberate. If we were to update this column after the fetch we may
# miss out on changes pushed during the fetch or between the fetch and
# updating the timestamp.
project.update_column(:last_repository_updated_at, Time.zone.now)
project.repository.fetch_remote('github', forced: false)
pname = project.path_with_namespace
Rails.logger
.info("GitHub importer finished updating repository for #{pname}")
repository_updates_counter.increment(project: pname)
end
def update_repository?(pr)
last_update = project.last_repository_updated_at || project.created_at
return false if pr.updated_at < last_update
# PRs may be updated without there actually being new commits, thus we
# check to make sure we only re-fetch if truly necessary.
!(commit_exists?(pr.head.sha) && commit_exists?(pr.base.sha))
end
def commit_exists?(sha)
project.repository.lookup(sha)
true
rescue Rugged::Error
false
end
def collection_method
:pull_requests
end
def collection_options
{ state: 'all', sort: 'created', direction: 'asc' }
end
def repository_updates_counter
@repository_updates_counter ||= Gitlab::Metrics.counter(
:github_importer_repository_updates,
'The number of times repositories have to be updated again'
)
end
end
end
end
end
# frozen_string_literal: true
module Gitlab
module GithubImport
module Importer
class ReleasesImporter
include BulkImporting
attr_reader :project, :client, :existing_tags
# project - An instance of `Project`
# client - An instance of `Gitlab::GithubImport::Client`
def initialize(project, client)
@project = project
@client = client
@existing_tags = project.releases.pluck(:tag).to_set
end
def execute
bulk_insert(Release, build_releases)
end
def build_releases
build_database_rows(each_release)
end
def already_imported?(release)
existing_tags.include?(release.tag_name)
end
def build(release)
{
tag: release.tag_name,
description: description_for(release),
created_at: release.created_at,
updated_at: release.updated_at,
project_id: project.id
}
end
def each_release
client.releases(project.import_source)
end
def description_for(release)
if release.body.present?
release.body
else
"Release for tag #{release.tag_name}"
end
end
end
end
end
end
# frozen_string_literal: true
module Gitlab
module GithubImport
module Importer
class RepositoryImporter
include Gitlab::ShellAdapter
attr_reader :project, :client
def initialize(project, client)
@project = project
@client = client
end
# Returns true if we should import the wiki for the project.
def import_wiki?
client.repository(project.import_source)&.has_wiki &&
!project.wiki_repository_exists?
end
# Imports the repository data.
#
# This method will return true if the data was imported successfully or
# the repository had already been imported before.
def execute
imported =
# It's possible a repository has already been imported when running
# this code, e.g. because we had to retry this job after
# `import_wiki?` raised a rate limit error. In this case we'll skip
# re-importing the main repository.
if project.repository.empty_repo?
import_repository
else
true
end
update_clone_time if imported
imported = import_wiki_repository if import_wiki? && imported
imported
end
def import_repository
project.ensure_repository
configure_repository_remote
project.repository.fetch_remote('github', forced: true)
true
rescue Gitlab::Git::Repository::NoRepository, Gitlab::Shell::Error => e
fail_import("Failed to import the repository: #{e.message}")
end
def configure_repository_remote
return if project.repository.remote_exists?('github')
project.repository.add_remote('github', project.import_url)
project.repository.set_import_remote_as_mirror('github')
project.repository.add_remote_fetch_config(
'github',
'+refs/pull/*/head:refs/merge-requests/*/head'
)
end
def import_wiki_repository
wiki_path = "#{project.disk_path}.wiki"
wiki_url = project.import_url.sub(/\.git\z/, '.wiki.git')
storage_path = project.repository_storage_path
gitlab_shell.import_repository(storage_path, wiki_path, wiki_url)
true
rescue Gitlab::Shell::Error => e
if e.message !~ /repository not exported/
fail_import("Failed to import the wiki: #{e.message}")
else
true
end
end
def update_clone_time
project.update_column(:last_repository_updated_at, Time.zone.now)
end
def fail_import(message)
project.mark_import_as_failed(message)
false
end
end
end
end
end
# frozen_string_literal: true
module Gitlab
module GithubImport
# IssuableFinder can be used for caching and retrieving database IDs for
# issuable objects such as issues and pull requests. By caching these IDs we
# remove the need for running a lot of database queries when importing
# GitHub projects.
class IssuableFinder
attr_reader :project, :object
# The base cache key to use for storing/retrieving issuable IDs.
CACHE_KEY = 'github-import/issuable-finder/%{project}/%{type}/%{iid}'.freeze
# project - An instance of `Project`.
# object - The object to look up or set a database ID for.
def initialize(project, object)
@project = project
@object = object
end
# Returns the database ID for the object.
#
# This method will return `nil` if no ID could be found.
def database_id
val = Caching.read(cache_key)
val.to_i if val.present?
end
# Associates the given database ID with the current object.
#
# database_id - The ID of the corresponding database row.
def cache_database_id(database_id)
Caching.write(cache_key, database_id)
end
private
def cache_key
CACHE_KEY % {
project: project.id,
type: cache_key_type,
iid: cache_key_iid
}
end
# Returns the identifier to use for cache keys.
#
# For issues and pull requests this will be "Issue" or "MergeRequest"
# respectively. For diff notes this will return "MergeRequest", for
# regular notes it will either return "Issue" or "MergeRequest" depending
# on what type of object the note belongs to.
def cache_key_type
if object.respond_to?(:issuable_type)
object.issuable_type
elsif object.respond_to?(:noteable_type)
object.noteable_type
else
raise(
TypeError,
"Instances of #{object.class} are not supported"
)
end
end
def cache_key_iid
if object.respond_to?(:noteable_id)
object.noteable_id
elsif object.respond_to?(:iid)
object.iid
else
raise(
TypeError,
"Instances of #{object.class} are not supported"
)
end
end
end
end
end
# frozen_string_literal: true
module Gitlab
module GithubImport
class LabelFinder
attr_reader :project
# The base cache key to use for storing/retrieving label IDs.
CACHE_KEY = 'github-import/label-finder/%{project}/%{name}'.freeze
# project - An instance of `Project`.
def initialize(project)
@project = project
end
# Returns the label ID for the given name.
def id_for(name)
Caching.read_integer(cache_key_for(name))
end
def build_cache
mapping = @project
.labels
.pluck(:id, :name)
.each_with_object({}) do |(id, name), hash|
hash[cache_key_for(name)] = id
end
Caching.write_multiple(mapping)
end
def cache_key_for(name)
CACHE_KEY % { project: project.id, name: name }
end
end
end
end
# frozen_string_literal: true
module Gitlab
module GithubImport
class MarkdownText
attr_reader :text, :author, :exists
def self.format(*args)
new(*args).to_s
end
# text - The Markdown text as a String.
# author - An instance of `Gitlab::GithubImport::Representation::User`
# exists - Boolean that indicates the user exists in the GitLab database.
def initialize(text, author, exists = false)
@text = text
@author = author
@exists = exists
end
def to_s
if exists
text
else
"*Created by: #{author.login}*\n\n#{text}"
end
end
end
end
end
# frozen_string_literal: true
module Gitlab
module GithubImport
class MilestoneFinder
attr_reader :project
# The base cache key to use for storing/retrieving milestone IDs.
CACHE_KEY = 'github-import/milestone-finder/%{project}/%{iid}'.freeze
# project - An instance of `Project`
def initialize(project)
@project = project
end
# issuable - An instance of `Gitlab::GithubImport::Representation::Issue`
# or `Gitlab::GithubImport::Representation::PullRequest`.
def id_for(issuable)
return unless issuable.milestone_number
Caching.read_integer(cache_key_for(issuable.milestone_number))
end
def build_cache
mapping = @project
.milestones
.pluck(:id, :iid)
.each_with_object({}) do |(id, iid), hash|
hash[cache_key_for(iid)] = id
end
Caching.write_multiple(mapping)
end
def cache_key_for(iid)
CACHE_KEY % { project: project.id, iid: iid }
end
end
end
end
# frozen_string_literal: true
module Gitlab
module GithubImport
# PageCounter can be used to keep track of the last imported page of a
# collection, allowing workers to resume where they left off in the event of
# an error.
class PageCounter
attr_reader :cache_key
# The base cache key to use for storing the last page number.
CACHE_KEY = 'github-importer/page-counter/%{project}/%{collection}'.freeze
def initialize(project, collection)
@cache_key = CACHE_KEY % { project: project.id, collection: collection }
end
# Sets the page number to the given value.
#
# Returns true if the page number was overwritten, false otherwise.
def set(page)
Caching.write_if_greater(cache_key, page)
end
# Returns the current value from the cache.
def current
Caching.read_integer(cache_key) || 1
end
end
end
end
# frozen_string_literal: true
module Gitlab
module GithubImport
# The ParallelImporter schedules the importing of a GitHub project using
# Sidekiq.
class ParallelImporter
attr_reader :project
def self.async?
true
end
def initialize(project)
@project = project
end
def execute
jid = generate_jid
# The original import JID is the JID of the RepositoryImportWorker job,
# which will be removed once that job completes. Reusing that JID could
# result in StuckImportJobsWorker marking the job as stuck before we get
# to running Stage::ImportRepositoryWorker.
#
# We work around this by setting the JID to a custom generated one, then
# refreshing it in the various stages whenever necessary.
Gitlab::SidekiqStatus
.set(jid, StuckImportJobsWorker::IMPORT_JOBS_EXPIRATION)
project.update_column(:import_jid, jid)
Stage::ImportRepositoryWorker
.perform_async(project.id)
true
end
def generate_jid
"github-importer/#{project.id}"
end
end
end
end
# frozen_string_literal: true
module Gitlab
module GithubImport
module ParallelScheduling
attr_reader :project, :client, :page_counter, :already_imported_cache_key
# The base cache key to use for tracking already imported objects.
ALREADY_IMPORTED_CACHE_KEY =
'github-importer/already-imported/%{project}/%{collection}'.freeze
# project - An instance of `Project`.
# client - An instance of `Gitlab::GithubImport::Client`.
# parallel - When set to true the objects will be imported in parallel.
def initialize(project, client, parallel: true)
@project = project
@client = client
@parallel = parallel
@page_counter = PageCounter.new(project, collection_method)
@already_imported_cache_key = ALREADY_IMPORTED_CACHE_KEY %
{ project: project.id, collection: collection_method }
end
def parallel?
@parallel
end
def execute
retval =
if parallel?
parallel_import
else
sequential_import
end
# Once we have completed all work we can remove our "already exists"
# cache so we don't put too much pressure on Redis.
#
# We don't immediately remove it since it's technically possible for
# other instances of this job to still run, instead we set the
# expiration time to a lower value. This prevents the other jobs from
# still scheduling duplicates while. Since all work has already been
# completed those jobs will just cycle through any remaining pages while
# not scheduling anything.
Caching.expire(already_imported_cache_key, 15.minutes.to_i)
retval
end
# Imports all the objects in sequence in the current thread.
def sequential_import
each_object_to_import do |object|
repr = representation_class.from_api_response(object)
importer_class.new(repr, project, client).execute
end
end
# Imports all objects in parallel by scheduling a Sidekiq job for every
# individual object.
def parallel_import
waiter = JobWaiter.new
each_object_to_import do |object|
repr = representation_class.from_api_response(object)
sidekiq_worker_class
.perform_async(project.id, repr.to_hash, waiter.key)
waiter.jobs_remaining += 1
end
waiter
end
# The method that will be called for traversing through all the objects to
# import, yielding them to the supplied block.
def each_object_to_import
repo = project.import_source
# We inject the page number here to make sure that all importers always
# start where they left off. Simply starting over wouldn't work for
# repositories with a lot of data (e.g. tens of thousands of comments).
options = collection_options.merge(page: page_counter.current)
client.each_page(collection_method, repo, options) do |page|
# Technically it's possible that the same work is performed multiple
# times, as Sidekiq doesn't guarantee there will ever only be one
# instance of a job. In such a scenario it's possible for one job to
# have a lower page number (e.g. 5) compared to another (e.g. 10). In
# this case we skip over all the objects until we have caught up,
# reducing the number of duplicate jobs scheduled by the provided
# block.
next unless page_counter.set(page.number)
page.objects.each do |object|
next if already_imported?(object)
yield object
# We mark the object as imported immediately so we don't end up
# scheduling it multiple times.
mark_as_imported(object)
end
end
end
# Returns true if the given object has already been imported, false
# otherwise.
#
# object - The object to check.
def already_imported?(object)
id = id_for_already_imported_cache(object)
Caching.set_includes?(already_imported_cache_key, id)
end
# Marks the given object as "already imported".
def mark_as_imported(object)
id = id_for_already_imported_cache(object)
Caching.set_add(already_imported_cache_key, id)
end
# Returns the ID to use for the cache used for checking if an object has
# already been imported or not.
#
# object - The object we may want to import.
def id_for_already_imported_cache(object)
raise NotImplementedError
end
# The class used for converting API responses to Hashes when performing
# the import.
def representation_class
raise NotImplementedError
end
# The class to use for importing objects when importing them sequentially.
def importer_class
raise NotImplementedError
end
# The Sidekiq worker class used for scheduling the importing of objects in
# parallel.
def sidekiq_worker_class
raise NotImplementedError
end
# The name of the method to call to retrieve the data to import.
def collection_method
raise NotImplementedError
end
# Any options to be passed to the method used for retrieving the data to
# import.
def collection_options
{}
end
end
end
end
# frozen_string_literal: true
module Gitlab
module GithubImport
# Error that will be raised when we're about to reach (or have reached) the
# GitHub API's rate limit.
RateLimitError = Class.new(StandardError)
end
end
# frozen_string_literal: true
module Gitlab
module GithubImport
module Representation
TIMESTAMP_KEYS = %i[created_at updated_at merged_at].freeze
# Converts a Hash with String based keys to one that can be used by the
# various Representation classes.
#
# Example:
#
# Representation.symbolize_hash('number' => 10) # => { number: 10 }
def self.symbolize_hash(raw_hash = nil)
hash = raw_hash.deep_symbolize_keys
TIMESTAMP_KEYS.each do |key|
hash[key] = Time.parse(hash[key]) if hash[key].is_a?(String)
end
hash
end
end
end
end
# frozen_string_literal: true
module Gitlab
module GithubImport
module Representation
class DiffNote
include ToHash
include ExposeAttribute
attr_reader :attributes
expose_attribute :noteable_type, :noteable_id, :commit_id, :file_path,
:diff_hunk, :author, :note, :created_at, :updated_at,
:github_id
NOTEABLE_ID_REGEX = /\/pull\/(?<iid>\d+)/i
# Builds a diff note from a GitHub API response.
#
# note - An instance of `Sawyer::Resource` containing the note details.
def self.from_api_response(note)
matches = note.html_url.match(NOTEABLE_ID_REGEX)
unless matches
raise(
ArgumentError,
"The note URL #{note.html_url.inspect} is not supported"
)
end
user = Representation::User.from_api_response(note.user) if note.user
hash = {
noteable_type: 'MergeRequest',
noteable_id: matches[:iid].to_i,
file_path: note.path,
commit_id: note.commit_id,
diff_hunk: note.diff_hunk,
author: user,
note: note.body,
created_at: note.created_at,
updated_at: note.updated_at,
github_id: note.id
}
new(hash)
end
# Builds a new note using a Hash that was built from a JSON payload.
def self.from_json_hash(raw_hash)
hash = Representation.symbolize_hash(raw_hash)
hash[:author] &&= Representation::User.from_json_hash(hash[:author])
new(hash)
end
# attributes - A Hash containing the raw note details. The keys of this
# Hash must be Symbols.
def initialize(attributes)
@attributes = attributes
end
def line_code
diff_line = Gitlab::Diff::Parser.new.parse(diff_hunk.lines).to_a.last
Gitlab::Git
.diff_line_code(file_path, diff_line.new_pos, diff_line.old_pos)
end
# Returns a Hash that can be used to populate `notes.st_diff`, removing
# the need for requesting Git data for every diff note.
def diff_hash
{
diff: diff_hunk,
new_path: file_path,
old_path: file_path,
# These fields are not displayed for LegacyDiffNote notes, so it
# doesn't really matter what we set them to.
a_mode: '100644',
b_mode: '100644',
new_file: false
}
end
end
end
end
end
# frozen_string_literal: true
module Gitlab
module GithubImport
module Representation
module ExposeAttribute
extend ActiveSupport::Concern
module ClassMethods
# Defines getter methods for the given attribute names.
#
# Example:
#
# expose_attribute :iid, :title
def expose_attribute(*names)
names.each do |name|
name = name.to_sym
define_method(name) { attributes[name] }
end
end
end
end
end
end
end
# frozen_string_literal: true
module Gitlab
module GithubImport
module Representation
class Issue
include ToHash
include ExposeAttribute
attr_reader :attributes
expose_attribute :iid, :title, :description, :milestone_number,
:created_at, :updated_at, :state, :assignees,
:label_names, :author
# Builds an issue from a GitHub API response.
#
# issue - An instance of `Sawyer::Resource` containing the issue
# details.
def self.from_api_response(issue)
user =
if issue.user
Representation::User.from_api_response(issue.user)
end
hash = {
iid: issue.number,
title: issue.title,
description: issue.body,
milestone_number: issue.milestone&.number,
state: issue.state == 'open' ? :opened : :closed,
assignees: issue.assignees.map do |u|
Representation::User.from_api_response(u)
end,
label_names: issue.labels.map(&:name),
author: user,
created_at: issue.created_at,
updated_at: issue.updated_at,
pull_request: issue.pull_request ? true : false
}
new(hash)
end
# Builds a new issue using a Hash that was built from a JSON payload.
def self.from_json_hash(raw_hash)
hash = Representation.symbolize_hash(raw_hash)
hash[:state] = hash[:state].to_sym
hash[:assignees].map! { |u| Representation::User.from_json_hash(u) }
hash[:author] &&= Representation::User.from_json_hash(hash[:author])
new(hash)
end
# attributes - A hash containing the raw issue details. The keys of this
# Hash (and any nested hashes) must be symbols.
def initialize(attributes)
@attributes = attributes
end
def truncated_title
title.truncate(255)
end
def labels?
label_names && label_names.any?
end
def pull_request?
attributes[:pull_request]
end
def issuable_type
pull_request? ? 'MergeRequest' : 'Issue'
end
end
end
end
end
# frozen_string_literal: true
module Gitlab
module GithubImport
module Representation
class Note
include ToHash
include ExposeAttribute
attr_reader :attributes
expose_attribute :noteable_id, :noteable_type, :author, :note,
:created_at, :updated_at, :github_id
NOTEABLE_TYPE_REGEX = /\/(?<type>(pull|issues))\/(?<iid>\d+)/i
# Builds a note from a GitHub API response.
#
# note - An instance of `Sawyer::Resource` containing the note details.
def self.from_api_response(note)
matches = note.html_url.match(NOTEABLE_TYPE_REGEX)
if !matches || !matches[:type]
raise(
ArgumentError,
"The note URL #{note.html_url.inspect} is not supported"
)
end
noteable_type =
if matches[:type] == 'pull'
'MergeRequest'
else
'Issue'
end
user = Representation::User.from_api_response(note.user) if note.user
hash = {
noteable_type: noteable_type,
noteable_id: matches[:iid].to_i,
author: user,
note: note.body,
created_at: note.created_at,
updated_at: note.updated_at,
github_id: note.id
}
new(hash)
end
# Builds a new note using a Hash that was built from a JSON payload.
def self.from_json_hash(raw_hash)
hash = Representation.symbolize_hash(raw_hash)
hash[:author] &&= Representation::User.from_json_hash(hash[:author])
new(hash)
end
# attributes - A Hash containing the raw note details. The keys of this
# Hash must be Symbols.
def initialize(attributes)
@attributes = attributes
end
alias_method :issuable_type, :noteable_type
end
end
end
end
# frozen_string_literal: true
module Gitlab
module GithubImport
module Representation
class PullRequest
include ToHash
include ExposeAttribute
attr_reader :attributes
expose_attribute :iid, :title, :description, :source_branch,
:source_branch_sha, :target_branch, :target_branch_sha,
:milestone_number, :author, :assignee, :created_at,
:updated_at, :merged_at, :source_repository_id,
:target_repository_id, :source_repository_owner
# Builds a PR from a GitHub API response.
#
# issue - An instance of `Sawyer::Resource` containing the PR details.
def self.from_api_response(pr)
assignee =
if pr.assignee
Representation::User.from_api_response(pr.assignee)
end
user = Representation::User.from_api_response(pr.user) if pr.user
hash = {
iid: pr.number,
title: pr.title,
description: pr.body,
source_branch: pr.head.ref,
target_branch: pr.base.ref,
source_branch_sha: pr.head.sha,
target_branch_sha: pr.base.sha,
source_repository_id: pr.head&.repo&.id,
target_repository_id: pr.base&.repo&.id,
source_repository_owner: pr.head&.user&.login,
state: pr.state == 'open' ? :opened : :closed,
milestone_number: pr.milestone&.number,
author: user,
assignee: assignee,
created_at: pr.created_at,
updated_at: pr.updated_at,
merged_at: pr.merged_at
}
new(hash)
end
# Builds a new PR using a Hash that was built from a JSON payload.
def self.from_json_hash(raw_hash)
hash = Representation.symbolize_hash(raw_hash)
hash[:state] = hash[:state].to_sym
hash[:author] &&= Representation::User.from_json_hash(hash[:author])
# Assignees are optional so we only convert it from a Hash if one was
# set.
hash[:assignee] &&= Representation::User
.from_json_hash(hash[:assignee])
new(hash)
end
# attributes - A Hash containing the raw PR details. The keys of this
# Hash (and any nested hashes) must be symbols.
def initialize(attributes)
@attributes = attributes
end
def truncated_title
title.truncate(255)
end
# Returns a formatted source branch.
#
# For cross-project pull requests the branch name will be in the format
# `owner-name:branch-name`.
def formatted_source_branch
if cross_project? && source_repository_owner
"#{source_repository_owner}:#{source_branch}"
elsif source_branch == target_branch
# Sometimes the source and target branch are the same, but GitLab
# doesn't support this. This can happen when both the user and
# source repository have been deleted, and the PR was submitted from
# the fork's master branch.
"#{source_branch}-#{iid}"
else
source_branch
end
end
def state
if merged_at
:merged
else
attributes[:state]
end
end
def cross_project?
return true unless source_repository_id
source_repository_id != target_repository_id
end
def issuable_type
'MergeRequest'
end
end
end
end
end
# frozen_string_literal: true
module Gitlab
module GithubImport
module Representation
module ToHash
# Converts the current representation to a Hash. The keys of this Hash
# will be Symbols.
def to_hash
hash = {}
attributes.each do |key, value|
hash[key] = convert_value_for_to_hash(value)
end
hash
end
def convert_value_for_to_hash(value)
if value.is_a?(Array)
value.map { |v| convert_value_for_to_hash(v) }
elsif value.respond_to?(:to_hash)
value.to_hash
else
value
end
end
end
end
end
end
# frozen_string_literal: true
module Gitlab
module GithubImport
module Representation
class User
include ToHash
include ExposeAttribute
attr_reader :attributes
expose_attribute :id, :login
# Builds a user from a GitHub API response.
#
# user - An instance of `Sawyer::Resource` containing the user details.
def self.from_api_response(user)
new(id: user.id, login: user.login)
end
# Builds a user using a Hash that was built from a JSON payload.
def self.from_json_hash(raw_hash)
new(Representation.symbolize_hash(raw_hash))
end
# attributes - A Hash containing the user details. The keys of this
# Hash (and any nested hashes) must be symbols.
def initialize(attributes)
@attributes = attributes
end
end
end
end
end
# frozen_string_literal: true
module Gitlab
module GithubImport
# The SequentialImporter imports a GitHub project in a single thread,
# without using Sidekiq. This makes it useful for testing purposes as well
# as Rake tasks, but it should be avoided for anything else in favour of the
# parallel importer.
class SequentialImporter
attr_reader :project, :client
SEQUENTIAL_IMPORTERS = [
Importer::LabelsImporter,
Importer::MilestonesImporter,
Importer::ReleasesImporter
].freeze
PARALLEL_IMPORTERS = [
Importer::PullRequestsImporter,
Importer::IssuesImporter,
Importer::DiffNotesImporter,
Importer::NotesImporter
].freeze
# project - The project to import the data into.
# token - The token to use for the GitHub API.
def initialize(project, token: nil)
@project = project
@client = GithubImport
.new_client_for(project, token: token, parallel: false)
end
def execute
Importer::RepositoryImporter.new(project, client).execute
SEQUENTIAL_IMPORTERS.each do |klass|
klass.new(project, client).execute
end
PARALLEL_IMPORTERS.each do |klass|
klass.new(project, client, parallel: false).execute
end
project.repository.after_import
true
end
end
end
end
# frozen_string_literal: true
module Gitlab
module GithubImport
# Class that can be used for finding a GitLab user ID based on a GitHub user
# ID or username.
#
# Any found user IDs are cached in Redis to reduce the number of SQL queries
# executed over time. Valid keys are refreshed upon access so frequently
# used keys stick around.
#
# Lookups are cached even if no ID was found to remove the need for querying
# the database when most queries are not going to return results anyway.
class UserFinder
attr_reader :project, :client
# The base cache key to use for caching user IDs for a given GitHub user
# ID.
ID_CACHE_KEY = 'github-import/user-finder/user-id/%s'.freeze
# The base cache key to use for caching user IDs for a given GitHub email
# address.
ID_FOR_EMAIL_CACHE_KEY =
'github-import/user-finder/id-for-email/%s'.freeze
# The base cache key to use for caching the Email addresses of GitHub
# usernames.
EMAIL_FOR_USERNAME_CACHE_KEY =
'github-import/user-finder/email-for-username/%s'.freeze
# project - An instance of `Project`
# client - An instance of `Gitlab::GithubImport::Client`
def initialize(project, client)
@project = project
@client = client
end
# Returns the GitLab user ID of an object's author.
#
# If the object has no author ID we'll use the ID of the GitLab ghost
# user.
def author_id_for(object)
id =
if object&.author
user_id_for(object.author)
else
GithubImport.ghost_user_id
end
if id
[id, true]
else
[project.creator_id, false]
end
end
# Returns the GitLab user ID of an issuable's assignee.
def assignee_id_for(issuable)
user_id_for(issuable.assignee) if issuable.assignee
end
# Returns the GitLab user ID for a GitHub user.
#
# user - An instance of `Gitlab::GithubImport::Representation::User`.
def user_id_for(user)
find(user.id, user.login)
end
# Returns the GitLab ID for the given GitHub ID or username.
#
# id - The ID of the GitHub user.
# username - The username of the GitHub user.
def find(id, username)
email = email_for_github_username(username)
cached, found_id = find_from_cache(id, email)
return found_id if found_id
# We only want to query the database if necessary. If previous lookups
# didn't yield a user ID we won't query the database again until the
# keys expire.
find_id_from_database(id, email) unless cached
end
# Finds a user ID from the cache for a given GitHub ID or Email.
def find_from_cache(id, email = nil)
id_exists, id_for_github_id = cached_id_for_github_id(id)
return [id_exists, id_for_github_id] if id_for_github_id
# Just in case no Email address could be retrieved (for whatever reason)
return [false] unless email
cached_id_for_github_email(email)
end
# Finds a GitLab user ID from the database for a given GitHub user ID or
# Email.
def find_id_from_database(id, email)
id_for_github_id(id) || id_for_github_email(email)
end
def email_for_github_username(username)
cache_key = EMAIL_FOR_USERNAME_CACHE_KEY % username
email = Caching.read(cache_key)
unless email
user = client.user(username)
email = Caching.write(cache_key, user.email) if user
end
email
end
def cached_id_for_github_id(id)
read_id_from_cache(ID_CACHE_KEY % id)
end
def cached_id_for_github_email(email)
read_id_from_cache(ID_FOR_EMAIL_CACHE_KEY % email)
end
# Queries and caches the GitLab user ID for a GitHub user ID, if one was
# found.
def id_for_github_id(id)
gitlab_id = query_id_for_github_id(id) || nil
Caching.write(ID_CACHE_KEY % id, gitlab_id)
end
# Queries and caches the GitLab user ID for a GitHub email, if one was
# found.
def id_for_github_email(email)
gitlab_id = query_id_for_github_email(email) || nil
Caching.write(ID_FOR_EMAIL_CACHE_KEY % email, gitlab_id)
end
def query_id_for_github_id(id)
User.for_github_id(id).pluck(:id).first
end
def query_id_for_github_email(email)
User.by_any_email(email).pluck(:id).first
end
# Reads an ID from the cache.
#
# The return value is an Array with two values:
#
# 1. A boolean indicating if the key was present or not.
# 2. The ID as an Integer, or nil in case no ID could be found.
def read_id_from_cache(key)
value = Caching.read(key)
exists = !value.nil?
number = value.to_i
# The cache key may be empty to indicate a previously looked up user for
# which we couldn't find an ID.
[exists, number.positive? ? number : nil]
end
end
end
end
......@@ -8,14 +8,14 @@ module Gitlab
ImportSource = Struct.new(:name, :title, :importer)
ImportTable = [
ImportSource.new('github', 'GitHub', Github::Import),
ImportSource.new('github', 'GitHub', Gitlab::GithubImport::ParallelImporter),
ImportSource.new('bitbucket', 'Bitbucket', Gitlab::BitbucketImport::Importer),
ImportSource.new('gitlab', 'GitLab.com', Gitlab::GitlabImport::Importer),
ImportSource.new('google_code', 'Google Code', Gitlab::GoogleCodeImport::Importer),
ImportSource.new('fogbugz', 'FogBugz', Gitlab::FogbugzImport::Importer),
ImportSource.new('git', 'Repo by URL', nil),
ImportSource.new('gitlab_project', 'GitLab export', Gitlab::ImportExport::Importer),
ImportSource.new('gitea', 'Gitea', Gitlab::GithubImport::Importer)
ImportSource.new('gitea', 'Gitea', Gitlab::LegacyGithubImport::Importer)
].freeze
class << self
......
......@@ -19,11 +19,13 @@ module Gitlab
Gitlab::Redis::SharedState.with { |redis| redis.lpush(key, jid) }
end
attr_reader :key, :jobs_remaining, :finished
attr_reader :key, :finished
attr_accessor :jobs_remaining
# jobs_remaining - the number of jobs left to wait for
def initialize(jobs_remaining)
@key = "gitlab:job_waiter:#{SecureRandom.uuid}"
# key - The key of this waiter.
def initialize(jobs_remaining = 0, key = "gitlab:job_waiter:#{SecureRandom.uuid}")
@key = key
@jobs_remaining = jobs_remaining
@finished = []
end
......
module Gitlab
module GithubImport
module LegacyGithubImport
class BaseFormatter
attr_reader :client, :formatter, :project, :raw_data
......
module Gitlab
module GithubImport
module LegacyGithubImport
class BranchFormatter < BaseFormatter
delegate :repo, :sha, :ref, to: :raw_data
......
module Gitlab
module LegacyGithubImport
class Client
GITHUB_SAFE_REMAINING_REQUESTS = 100
GITHUB_SAFE_SLEEP_TIME = 500
attr_reader :access_token, :host, :api_version
def initialize(access_token, host: nil, api_version: 'v3')
@access_token = access_token
@host = host.to_s.sub(%r{/+\z}, '')
@api_version = api_version
@users = {}
if access_token
::Octokit.auto_paginate = false
end
end
def api
@api ||= ::Octokit::Client.new(
access_token: access_token,
api_endpoint: api_endpoint,
# If there is no config, we're connecting to github.com and we
# should verify ssl.
connection_options: {
ssl: { verify: config ? config['verify_ssl'] : true }
}
)
end
def client
unless config
raise Projects::ImportService::Error,
'OAuth configuration for GitHub missing.'
end
@client ||= ::OAuth2::Client.new(
config.app_id,
config.app_secret,
github_options.merge(ssl: { verify: config['verify_ssl'] })
)
end
def authorize_url(redirect_uri)
client.auth_code.authorize_url({
redirect_uri: redirect_uri,
scope: "repo, user, user:email"
})
end
def get_token(code)
client.auth_code.get_token(code).token
end
def method_missing(method, *args, &block)
if api.respond_to?(method)
request(method, *args, &block)
else
super(method, *args, &block)
end
end
def respond_to?(method)
api.respond_to?(method) || super
end
def user(login)
return nil unless login.present?
return @users[login] if @users.key?(login)
@users[login] = api.user(login)
end
private
def api_endpoint
if host.present? && api_version.present?
"#{host}/api/#{api_version}"
else
github_options[:site]
end
end
def config
Gitlab.config.omniauth.providers.find { |provider| provider.name == "github" }
end
def github_options
if config
config["args"]["client_options"].deep_symbolize_keys
else
OmniAuth::Strategies::GitHub.default_options[:client_options].symbolize_keys
end
end
def rate_limit
api.rate_limit!
# GitHub Rate Limit API returns 404 when the rate limit is
# disabled. In this case we just want to return gracefully
# instead of spitting out an error.
rescue Octokit::NotFound
nil
end
def has_rate_limit?
return @has_rate_limit if defined?(@has_rate_limit)
@has_rate_limit = rate_limit.present?
end
def rate_limit_exceed?
has_rate_limit? && rate_limit.remaining <= GITHUB_SAFE_REMAINING_REQUESTS
end
def rate_limit_sleep_time
rate_limit.resets_in + GITHUB_SAFE_SLEEP_TIME
end
def request(method, *args, &block)
sleep rate_limit_sleep_time if rate_limit_exceed?
data = api.__send__(method, *args) # rubocop:disable GitlabSecurity/PublicSend
return data unless data.is_a?(Array)
last_response = api.last_response
if block_given?
yield data
# api.last_response could change while we're yielding (e.g. fetching labels for each PR)
# so we cache our own last response
each_response_page(last_response, &block)
else
each_response_page(last_response) { |page| data.concat(page) }
data
end
end
def each_response_page(last_response)
while last_response.rels[:next]
sleep rate_limit_sleep_time if rate_limit_exceed?
last_response = last_response.rels[:next].get
yield last_response.data if last_response.data.is_a?(Array)
end
end
end
end
end
module Gitlab
module GithubImport
module LegacyGithubImport
class CommentFormatter < BaseFormatter
attr_writer :author_id
......
module Gitlab
module GithubImport
module LegacyGithubImport
class Importer
include Gitlab::ShellAdapter
......
module Gitlab
module GithubImport
module LegacyGithubImport
class IssuableFormatter < BaseFormatter
attr_writer :assignee_id, :author_id
......
module Gitlab
module GithubImport
module LegacyGithubImport
class IssueFormatter < IssuableFormatter
def attributes
{
......
module Gitlab
module GithubImport
module LegacyGithubImport
class LabelFormatter < BaseFormatter
def attributes
{
......
module Gitlab
module GithubImport
module LegacyGithubImport
class MilestoneFormatter < BaseFormatter
def attributes
{
......
module Gitlab
module GithubImport
module LegacyGithubImport
class ProjectCreator
include Gitlab::CurrentSettings
......
module Gitlab
module GithubImport
module LegacyGithubImport
class PullRequestFormatter < IssuableFormatter
delegate :user, :project, :ref, :repo, :sha, to: :source_branch, prefix: true
delegate :user, :exists?, :project, :ref, :repo, :sha, :short_sha, to: :target_branch, prefix: true
......
module Gitlab
module GithubImport
module LegacyGithubImport
class ReleaseFormatter < BaseFormatter
def attributes
{
......
module Gitlab
module GithubImport
module LegacyGithubImport
class UserFormatter
attr_reader :client, :raw
......
module Gitlab
module GithubImport
module LegacyGithubImport
class WikiFormatter
attr_reader :project
......
......@@ -7,7 +7,7 @@ class GithubImport
end
def initialize(token, gitlab_username, project_path, extras)
@options = { token: token, verbose: true }
@options = { token: token }
@project_path = project_path
@current_user = User.find_by_username(gitlab_username)
@github_repo = extras.empty? ? nil : extras.first
......@@ -42,7 +42,9 @@ class GithubImport
import_success = false
timings = Benchmark.measure do
import_success = Github::Import.new(@project, @options).execute
import_success = Gitlab::GithubImport::SequentialImporter
.new(@project, token: @options[:token])
.execute
end
if import_success
......
......@@ -21,9 +21,9 @@ describe Import::GithubController do
describe "GET callback" do
it "updates access token" do
token = "asdasd12345"
allow_any_instance_of(Gitlab::GithubImport::Client)
allow_any_instance_of(Gitlab::LegacyGithubImport::Client)
.to receive(:get_token).and_return(token)
allow_any_instance_of(Gitlab::GithubImport::Client)
allow_any_instance_of(Gitlab::LegacyGithubImport::Client)
.to receive(:github_options).and_return({})
stub_omniauth_provider('github')
......
......@@ -13,6 +13,47 @@ describe Feature do
end
end
describe '.persisted_names' do
it 'returns the names of the persisted features' do
Feature::FlipperFeature.create!(key: 'foo')
expect(described_class.persisted_names).to eq(%w[foo])
end
it 'returns an empty Array when no features are presisted' do
expect(described_class.persisted_names).to be_empty
end
it 'caches the feature names when request store is active', :request_store do
Feature::FlipperFeature.create!(key: 'foo')
expect(Feature::FlipperFeature)
.to receive(:feature_names)
.once
.and_call_original
2.times do
expect(described_class.persisted_names).to eq(%w[foo])
end
end
end
describe '.persisted?' do
it 'returns true for a persisted feature' do
Feature::FlipperFeature.create!(key: 'foo')
feature = double(:feature, name: 'foo')
expect(described_class.persisted?(feature)).to eq(true)
end
it 'returns false for a feature that is not persisted' do
feature = double(:feature, name: 'foo')
expect(described_class.persisted?(feature)).to eq(false)
end
end
describe '.all' do
let(:features) { Set.new }
......
require 'spec_helper'
describe Github::Client do
let(:connection) { spy }
let(:rate_limit) { double(get: [false, 1]) }
let(:client) { described_class.new({}) }
let(:results) { double }
let(:response) { double }
before do
allow(Faraday).to receive(:new).and_return(connection)
allow(Github::RateLimit).to receive(:new).with(connection).and_return(rate_limit)
end
describe '#get' do
before do
allow(Github::Response).to receive(:new).with(results).and_return(response)
end
it 'uses a default per_page param' do
expect(connection).to receive(:get).with('/foo', per_page: 100).and_return(results)
expect(client.get('/foo')).to eq(response)
end
context 'with per_page given' do
it 'overwrites the default per_page' do
expect(connection).to receive(:get).with('/foo', per_page: 30).and_return(results)
expect(client.get('/foo', per_page: 30)).to eq(response)
end
end
end
end
require 'spec_helper'
describe Github::Import::LegacyDiffNote do
describe '#type' do
it 'returns the original note type' do
expect(described_class.new.type).to eq('LegacyDiffNote')
end
end
end
require 'spec_helper'
describe Github::Import::Note do
describe '#type' do
it 'returns the original note type' do
expect(described_class.new.type).to eq('Note')
end
end
end
......@@ -202,6 +202,26 @@ describe Gitlab::Database do
it 'handles non-UTF-8 data' do
expect { described_class.bulk_insert('test', [{ a: "\255" }]) }.not_to raise_error
end
context 'when using PostgreSQL' do
before do
allow(described_class).to receive(:mysql?).and_return(false)
end
it 'allows the returning of the IDs of the inserted rows' do
result = double(:result, values: [['10']])
expect(connection)
.to receive(:execute)
.with(/RETURNING id/)
.and_return(result)
ids = described_class
.bulk_insert('test', [{ number: 10 }], return_ids: true)
expect(ids).to eq([10])
end
end
end
describe '.create_connection_pool' do
......
......@@ -648,6 +648,21 @@ describe Gitlab::Git::Repository, seed_helper: true do
end
end
describe '#remote_exists?' do
before(:all) do
@repo = Gitlab::Git::Repository.new('default', TEST_MUTABLE_REPO_PATH, '')
@repo.add_remote("new_remote", SeedHelper::GITLAB_GIT_TEST_REPO_URL)
end
it 'returns true for an existing remote' do
expect(@repo.remote_exists?('new_remote')).to eq(true)
end
it 'returns false for a non-existing remote' do
expect(@repo.remote_exists?('foo')).to eq(false)
end
end
describe "#log" do
let(:commit_with_old_name) do
Gitlab::Git::Commit.decorate(repository, @commit_with_old_name_id)
......
require 'spec_helper'
describe Gitlab::GithubImport::BulkImporting do
let(:importer) do
Class.new { include(Gitlab::GithubImport::BulkImporting) }.new
end
describe '#build_database_rows' do
it 'returns an Array containing the rows to insert' do
object = double(:object, title: 'Foo')
expect(importer)
.to receive(:build)
.with(object)
.and_return({ title: 'Foo' })
expect(importer)
.to receive(:already_imported?)
.with(object)
.and_return(false)
enum = [[object, 1]].to_enum
expect(importer.build_database_rows(enum)).to eq([{ title: 'Foo' }])
end
it 'does not import objects that have already been imported' do
object = double(:object, title: 'Foo')
expect(importer)
.not_to receive(:build)
expect(importer)
.to receive(:already_imported?)
.with(object)
.and_return(true)
enum = [[object, 1]].to_enum
expect(importer.build_database_rows(enum)).to be_empty
end
end
describe '#bulk_insert' do
it 'bulk inserts rows into the database' do
rows = [{ title: 'Foo' }] * 10
model = double(:model, table_name: 'kittens')
expect(Gitlab::Database)
.to receive(:bulk_insert)
.ordered
.with('kittens', rows.first(5))
expect(Gitlab::Database)
.to receive(:bulk_insert)
.ordered
.with('kittens', rows.last(5))
importer.bulk_insert(model, rows, batch_size: 5)
end
end
end
require 'spec_helper'
describe Gitlab::GithubImport::Caching, :clean_gitlab_redis_cache do
describe '.read' do
it 'reads a value from the cache' do
described_class.write('foo', 'bar')
expect(described_class.read('foo')).to eq('bar')
end
it 'returns nil if the cache key does not exist' do
expect(described_class.read('foo')).to be_nil
end
it 'refreshes the cache key if a value is present' do
described_class.write('foo', 'bar')
redis = double(:redis)
expect(redis).to receive(:get).with(/foo/).and_return('bar')
expect(redis).to receive(:expire).with(/foo/, described_class::TIMEOUT)
expect(Gitlab::Redis::Cache).to receive(:with).twice.and_yield(redis)
described_class.read('foo')
end
it 'does not refresh the cache key if a value is empty' do
described_class.write('foo', nil)
redis = double(:redis)
expect(redis).to receive(:get).with(/foo/).and_return('')
expect(redis).not_to receive(:expire)
expect(Gitlab::Redis::Cache).to receive(:with).and_yield(redis)
described_class.read('foo')
end
end
describe '.read_integer' do
it 'returns an Integer' do
described_class.write('foo', '10')
expect(described_class.read_integer('foo')).to eq(10)
end
it 'returns nil if no value was found' do
expect(described_class.read_integer('foo')).to be_nil
end
end
describe '.write' do
it 'writes a value to the cache and returns the written value' do
expect(described_class.write('foo', 10)).to eq(10)
expect(described_class.read('foo')).to eq('10')
end
end
describe '.set_add' do
it 'adds a value to a set' do
described_class.set_add('foo', 10)
described_class.set_add('foo', 10)
key = described_class.cache_key_for('foo')
values = Gitlab::Redis::Cache.with { |r| r.smembers(key) }
expect(values).to eq(['10'])
end
end
describe '.set_includes?' do
it 'returns false when the key does not exist' do
expect(described_class.set_includes?('foo', 10)).to eq(false)
end
it 'returns false when the value is not present in the set' do
described_class.set_add('foo', 10)
expect(described_class.set_includes?('foo', 20)).to eq(false)
end
it 'returns true when the set includes the given value' do
described_class.set_add('foo', 10)
expect(described_class.set_includes?('foo', 10)).to eq(true)
end
end
describe '.write_multiple' do
it 'sets multiple keys' do
mapping = { 'foo' => 10, 'bar' => 20 }
described_class.write_multiple(mapping)
mapping.each do |key, value|
full_key = described_class.cache_key_for(key)
found = Gitlab::Redis::Cache.with { |r| r.get(full_key) }
expect(found).to eq(value.to_s)
end
end
end
describe '.expire' do
it 'sets the expiration time of a key' do
timeout = 1.hour.to_i
described_class.write('foo', 'bar', timeout: 2.hours.to_i)
described_class.expire('foo', timeout)
key = described_class.cache_key_for('foo')
found_ttl = Gitlab::Redis::Cache.with { |r| r.ttl(key) }
expect(found_ttl).to be <= timeout
end
end
end
require 'spec_helper'
describe Gitlab::GithubImport::Client do
let(:token) { '123456' }
let(:github_provider) { Settingslogic.new('app_id' => 'asd123', 'app_secret' => 'asd123', 'name' => 'github', 'args' => { 'client_options' => {} }) }
describe '#parallel?' do
it 'returns true when the client is running in parallel mode' do
client = described_class.new('foo', parallel: true)
subject(:client) { described_class.new(token) }
expect(client).to be_parallel
end
it 'returns false when the client is running in sequential mode' do
client = described_class.new('foo', parallel: false)
before do
allow(Gitlab.config.omniauth).to receive(:providers).and_return([github_provider])
expect(client).not_to be_parallel
end
end
it 'convert OAuth2 client options to symbols' do
client.client.options.keys.each do |key|
expect(key).to be_kind_of(Symbol)
describe '#user' do
it 'returns the details for the given username' do
client = described_class.new('foo')
expect(client.octokit).to receive(:user).with('foo')
expect(client).to receive(:with_rate_limit).and_yield
client.user('foo')
end
end
it 'does not crash (e.g. Settingslogic::MissingSetting) when verify_ssl config is not present' do
expect { client.api }.not_to raise_error
describe '#repository' do
it 'returns the details of a repository' do
client = described_class.new('foo')
expect(client.octokit).to receive(:repo).with('foo/bar')
expect(client).to receive(:with_rate_limit).and_yield
client.repository('foo/bar')
end
end
context 'when config is missing' do
before do
allow(Gitlab.config.omniauth).to receive(:providers).and_return([])
describe '#labels' do
it 'returns the labels' do
client = described_class.new('foo')
expect(client)
.to receive(:each_object)
.with(:labels, 'foo/bar')
client.labels('foo/bar')
end
end
describe '#milestones' do
it 'returns the milestones' do
client = described_class.new('foo')
expect(client)
.to receive(:each_object)
.with(:milestones, 'foo/bar')
it 'is still possible to get an Octokit client' do
expect { client.api }.not_to raise_error
client.milestones('foo/bar')
end
end
describe '#releases' do
it 'returns the releases' do
client = described_class.new('foo')
it 'is not be possible to get an OAuth2 client' do
expect { client.client }.to raise_error(Projects::ImportService::Error)
expect(client)
.to receive(:each_object)
.with(:releases, 'foo/bar')
client.releases('foo/bar')
end
end
context 'allow SSL verification to be configurable on API' do
describe '#each_page' do
let(:client) { described_class.new('foo') }
let(:object1) { double(:object1) }
let(:object2) { double(:object2) }
before do
github_provider['verify_ssl'] = false
end
allow(client)
.to receive(:with_rate_limit)
.and_yield
allow(client.octokit)
.to receive(:public_send)
.and_return([object1])
response = double(:response, data: [object2], rels: { next: nil })
next_page = double(:next_page, get: response)
it 'uses supplied value' do
expect(client.client.options[:connection_opts][:ssl]).to eq({ verify: false })
expect(client.api.connection_options[:ssl]).to eq({ verify: false })
allow(client.octokit)
.to receive(:last_response)
.and_return(double(:last_response, rels: { next: next_page }))
end
end
describe '#api_endpoint' do
context 'when provider does not specity an API endpoint' do
it 'uses GitHub root API endpoint' do
expect(client.api.api_endpoint).to eq 'https://api.github.com/'
context 'without a block' do
it 'returns an Enumerator' do
expect(client.each_page(:foo)).to be_an_instance_of(Enumerator)
end
it 'the returned Enumerator returns Page objects' do
enum = client.each_page(:foo)
page1 = enum.next
page2 = enum.next
expect(page1).to be_an_instance_of(described_class::Page)
expect(page2).to be_an_instance_of(described_class::Page)
expect(page1.objects).to eq([object1])
expect(page1.number).to eq(1)
expect(page2.objects).to eq([object2])
expect(page2.number).to eq(2)
end
end
context 'when provider specify a custom API endpoint' do
before do
github_provider['args']['client_options']['site'] = 'https://github.company.com/'
context 'with a block' do
it 'yields every retrieved page to the supplied block' do
pages = []
client.each_page(:foo) { |page| pages << page }
expect(pages[0]).to be_an_instance_of(described_class::Page)
expect(pages[1]).to be_an_instance_of(described_class::Page)
expect(pages[0].objects).to eq([object1])
expect(pages[0].number).to eq(1)
expect(pages[1].objects).to eq([object2])
expect(pages[1].number).to eq(2)
end
it 'uses the custom API endpoint' do
expect(OmniAuth::Strategies::GitHub).not_to receive(:default_options)
expect(client.api.api_endpoint).to eq 'https://github.company.com/'
it 'starts at the given page' do
pages = []
client.each_page(:foo, page: 2) { |page| pages << page }
expect(pages[0].number).to eq(2)
expect(pages[1].number).to eq(3)
end
end
end
context 'when given a host' do
subject(:client) { described_class.new(token, host: 'https://try.gitea.io/') }
describe '#with_rate_limit' do
let(:client) { described_class.new('foo') }
it 'builds a endpoint with the given host and the default API version' do
expect(client.api.api_endpoint).to eq 'https://try.gitea.io/api/v3/'
end
it 'yields the supplied block when enough requests remain' do
expect(client).to receive(:requests_remaining?).and_return(true)
expect { |b| client.with_rate_limit(&b) }.to yield_control
end
context 'when given an API version' do
subject(:client) { described_class.new(token, api_version: 'v3') }
it 'waits before yielding if not enough requests remain' do
expect(client).to receive(:requests_remaining?).and_return(false)
expect(client).to receive(:raise_or_wait_for_rate_limit)
it 'does not use the API version without a host' do
expect(client.api.api_endpoint).to eq 'https://api.github.com/'
end
expect { |b| client.with_rate_limit(&b) }.to yield_control
end
context 'when given a host and version' do
subject(:client) { described_class.new(token, host: 'https://try.gitea.io/', api_version: 'v3') }
it 'waits and retries the operation if all requests were consumed in the supplied block' do
retries = 0
expect(client).to receive(:requests_remaining?).and_return(true)
expect(client).to receive(:raise_or_wait_for_rate_limit)
it 'builds a endpoint with the given options' do
expect(client.api.api_endpoint).to eq 'https://try.gitea.io/api/v3/'
client.with_rate_limit do
if retries.zero?
retries += 1
raise(Octokit::TooManyRequests)
end
end
expect(retries).to eq(1)
end
it 'increments the request count counter' do
expect(client.request_count_counter)
.to receive(:increment)
.and_call_original
expect(client).to receive(:requests_remaining?).and_return(true)
client.with_rate_limit { }
end
end
describe '#requests_remaining?' do
let(:client) { described_class.new('foo') }
it 'returns true if enough requests remain' do
expect(client).to receive(:remaining_requests).and_return(9000)
expect(client.requests_remaining?).to eq(true)
end
it 'returns false if not enough requests remain' do
expect(client).to receive(:remaining_requests).and_return(1)
expect(client.requests_remaining?).to eq(false)
end
end
describe '#raise_or_wait_for_rate_limit' do
it 'raises RateLimitError when running in parallel mode' do
client = described_class.new('foo', parallel: true)
expect { client.raise_or_wait_for_rate_limit }
.to raise_error(Gitlab::GithubImport::RateLimitError)
end
it 'sleeps when running in sequential mode' do
client = described_class.new('foo', parallel: false)
expect(client).to receive(:rate_limit_resets_in).and_return(1)
expect(client).to receive(:sleep).with(1)
client.raise_or_wait_for_rate_limit
end
it 'increments the rate limit counter' do
client = described_class.new('foo', parallel: false)
expect(client)
.to receive(:rate_limit_resets_in)
.and_return(1)
expect(client)
.to receive(:sleep)
.with(1)
expect(client.rate_limit_counter)
.to receive(:increment)
.and_call_original
client.raise_or_wait_for_rate_limit
end
end
it 'does not raise error when rate limit is disabled' do
stub_request(:get, /api.github.com/)
allow(client.api).to receive(:rate_limit!).and_raise(Octokit::NotFound)
describe '#remaining_requests' do
it 'returns the number of remaining requests' do
client = described_class.new('foo')
rate_limit = double(remaining: 1)
expect(client.octokit).to receive(:rate_limit).and_return(rate_limit)
expect(client.remaining_requests).to eq(1)
end
end
describe '#rate_limit_resets_in' do
it 'returns the number of seconds after which the rate limit is reset' do
client = described_class.new('foo')
rate_limit = double(resets_in: 1)
expect(client.octokit).to receive(:rate_limit).and_return(rate_limit)
expect(client.rate_limit_resets_in).to eq(6)
end
end
expect { client.issues {} }.not_to raise_error
describe '#method_missing' do
it 'delegates missing methods to the request method' do
client = described_class.new('foo')
expect(client).to receive(:milestones).with(state: 'all')
client.milestones(state: 'all')
end
end
describe '#respond_to_missing?' do
it 'returns true for methods supported by Octokit' do
client = described_class.new('foo')
expect(client.respond_to?(:milestones)).to eq(true)
end
it 'returns false for methods not supported by Octokit' do
client = described_class.new('foo')
expect(client.respond_to?(:kittens)).to eq(false)
end
end
end
require 'spec_helper'
describe Gitlab::GithubImport::Importer::DiffNoteImporter do
let(:project) { create(:project) }
let(:client) { double(:client) }
let(:user) { create(:user) }
let(:created_at) { Time.new(2017, 1, 1, 12, 00) }
let(:updated_at) { Time.new(2017, 1, 1, 12, 15) }
let(:hunk) do
'@@ -1 +1 @@
-Hello
+Hello world'
end
let(:note) do
Gitlab::GithubImport::Representation::DiffNote.new(
noteable_type: 'MergeRequest',
noteable_id: 1,
commit_id: '123abc',
file_path: 'README.md',
diff_hunk: hunk,
author: Gitlab::GithubImport::Representation::User
.new(id: user.id, login: user.username),
note: 'Hello',
created_at: created_at,
updated_at: updated_at,
github_id: 1
)
end
let(:importer) { described_class.new(note, project, client) }
describe '#execute' do
context 'when the merge request no longer exists' do
it 'does not import anything' do
expect(Gitlab::Database).not_to receive(:bulk_insert)
importer.execute
end
end
context 'when the merge request exists' do
let!(:merge_request) do
create(:merge_request, source_project: project, target_project: project)
end
before do
allow(importer)
.to receive(:find_merge_request_id)
.and_return(merge_request.id)
end
it 'imports the note' do
allow(importer.user_finder)
.to receive(:author_id_for)
.and_return([user.id, true])
expect(Gitlab::Database)
.to receive(:bulk_insert)
.with(
LegacyDiffNote.table_name,
[
{
noteable_type: 'MergeRequest',
noteable_id: merge_request.id,
project_id: project.id,
author_id: user.id,
note: 'Hello',
system: false,
commit_id: '123abc',
line_code: note.line_code,
type: 'LegacyDiffNote',
created_at: created_at,
updated_at: updated_at,
st_diff: note.diff_hash.to_yaml
}
]
)
.and_call_original
importer.execute
end
it 'imports the note when the author could not be found' do
allow(importer.user_finder)
.to receive(:author_id_for)
.and_return([project.creator_id, false])
expect(Gitlab::Database)
.to receive(:bulk_insert)
.with(
LegacyDiffNote.table_name,
[
{
noteable_type: 'MergeRequest',
noteable_id: merge_request.id,
project_id: project.id,
author_id: project.creator_id,
note: "*Created by: #{user.username}*\n\nHello",
system: false,
commit_id: '123abc',
line_code: note.line_code,
type: 'LegacyDiffNote',
created_at: created_at,
updated_at: updated_at,
st_diff: note.diff_hash.to_yaml
}
]
)
.and_call_original
importer.execute
end
it 'produces a valid LegacyDiffNote' do
allow(importer.user_finder)
.to receive(:author_id_for)
.and_return([user.id, true])
importer.execute
note = project.notes.diff_notes.take
expect(note).to be_valid
expect(note.diff).to be_an_instance_of(Gitlab::Git::Diff)
end
it 'does not import the note when a foreign key error is raised' do
allow(importer.user_finder)
.to receive(:author_id_for)
.and_return([project.creator_id, false])
expect(Gitlab::Database)
.to receive(:bulk_insert)
.and_raise(ActiveRecord::InvalidForeignKey, 'invalid foreign key')
expect { importer.execute }.not_to raise_error
end
end
end
describe '#find_merge_request_id' do
it 'returns a merge request ID' do
expect_any_instance_of(Gitlab::GithubImport::IssuableFinder)
.to receive(:database_id)
.and_return(10)
expect(importer.find_merge_request_id).to eq(10)
end
end
end
require 'spec_helper'
describe Gitlab::GithubImport::Importer::DiffNotesImporter do
let(:project) { double(:project, id: 4, import_source: 'foo/bar') }
let(:client) { double(:client) }
let(:github_comment) do
double(
:response,
html_url: 'https://github.com/foo/bar/pull/42',
path: 'README.md',
commit_id: '123abc',
diff_hunk: "@@ -1 +1 @@\n-Hello\n+Hello world",
user: double(:user, id: 4, login: 'alice'),
body: 'Hello world',
created_at: Time.zone.now,
updated_at: Time.zone.now,
id: 1
)
end
describe '#parallel?' do
it 'returns true when running in parallel mode' do
importer = described_class.new(project, client)
expect(importer).to be_parallel
end
it 'returns false when running in sequential mode' do
importer = described_class.new(project, client, parallel: false)
expect(importer).not_to be_parallel
end
end
describe '#execute' do
context 'when running in parallel mode' do
it 'imports diff notes in parallel' do
importer = described_class.new(project, client)
expect(importer).to receive(:parallel_import)
importer.execute
end
end
context 'when running in sequential mode' do
it 'imports diff notes in sequence' do
importer = described_class.new(project, client, parallel: false)
expect(importer).to receive(:sequential_import)
importer.execute
end
end
end
describe '#sequential_import' do
it 'imports each diff note in sequence' do
importer = described_class.new(project, client, parallel: false)
diff_note_importer = double(:diff_note_importer)
allow(importer)
.to receive(:each_object_to_import)
.and_yield(github_comment)
expect(Gitlab::GithubImport::Importer::DiffNoteImporter)
.to receive(:new)
.with(
an_instance_of(Gitlab::GithubImport::Representation::DiffNote),
project,
client
)
.and_return(diff_note_importer)
expect(diff_note_importer).to receive(:execute)
importer.sequential_import
end
end
describe '#parallel_import' do
it 'imports each diff note in parallel' do
importer = described_class.new(project, client)
allow(importer)
.to receive(:each_object_to_import)
.and_yield(github_comment)
expect(Gitlab::GithubImport::ImportDiffNoteWorker)
.to receive(:perform_async)
.with(project.id, an_instance_of(Hash), an_instance_of(String))
waiter = importer.parallel_import
expect(waiter).to be_an_instance_of(Gitlab::JobWaiter)
expect(waiter.jobs_remaining).to eq(1)
end
end
describe '#id_for_already_imported_cache' do
it 'returns the ID of the given note' do
importer = described_class.new(project, client)
expect(importer.id_for_already_imported_cache(github_comment))
.to eq(1)
end
end
describe '#collection_options' do
it 'returns an empty Hash' do
# For large projects (e.g. kubernetes/kubernetes) GitHub's API may produce
# HTTP 500 errors when using explicit sorting options, regardless of what
# order you sort in. Not using any sorting options at all allows us to
# work around this.
importer = described_class.new(project, client)
expect(importer.collection_options).to eq({})
end
end
end
require 'spec_helper'
describe Gitlab::GithubImport::Importer::IssueAndLabelLinksImporter do
describe '#execute' do
it 'imports an issue and its labels' do
issue = double(:issue)
project = double(:project)
client = double(:client)
label_links_instance = double(:label_links_importer)
importer = described_class.new(issue, project, client)
expect(Gitlab::GithubImport::Importer::IssueImporter)
.to receive(:import_if_issue)
.with(issue, project, client)
expect(Gitlab::GithubImport::Importer::LabelLinksImporter)
.to receive(:new)
.with(issue, project, client)
.and_return(label_links_instance)
expect(label_links_instance)
.to receive(:execute)
importer.execute
end
end
end
require 'spec_helper'
describe Gitlab::GithubImport::Importer::IssueImporter, :clean_gitlab_redis_cache do
let(:project) { create(:project) }
let(:client) { double(:client) }
let(:user) { create(:user) }
let(:milestone) { create(:milestone, project: project) }
let(:created_at) { Time.new(2017, 1, 1, 12, 00) }
let(:updated_at) { Time.new(2017, 1, 1, 12, 15) }
let(:issue) do
Gitlab::GithubImport::Representation::Issue.new(
iid: 42,
title: 'My Issue',
description: 'This is my issue',
milestone_number: 1,
state: :opened,
assignees: [
Gitlab::GithubImport::Representation::User.new(id: 4, login: 'alice'),
Gitlab::GithubImport::Representation::User.new(id: 5, login: 'bob')
],
label_names: %w[bug],
author: Gitlab::GithubImport::Representation::User.new(id: 4, login: 'alice'),
created_at: created_at,
updated_at: updated_at,
pull_request: false
)
end
describe '.import_if_issue' do
it 'imports an issuable if it is a regular issue' do
importer = double(:importer)
expect(described_class)
.to receive(:new)
.with(issue, project, client)
.and_return(importer)
expect(importer).to receive(:execute)
described_class.import_if_issue(issue, project, client)
end
it 'does not import the issuable if it is a pull request' do
expect(issue).to receive(:pull_request?).and_return(true)
expect(described_class).not_to receive(:new)
described_class.import_if_issue(issue, project, client)
end
end
describe '#execute' do
let(:importer) { described_class.new(issue, project, client) }
it 'creates the issue and assignees' do
expect(importer)
.to receive(:create_issue)
.and_return(10)
expect(importer)
.to receive(:create_assignees)
.with(10)
expect(importer.issuable_finder)
.to receive(:cache_database_id)
.with(10)
importer.execute
end
end
describe '#create_issue' do
let(:importer) { described_class.new(issue, project, client) }
before do
allow(importer.milestone_finder)
.to receive(:id_for)
.with(issue)
.and_return(milestone.id)
end
context 'when the issue author could be found' do
it 'creates the issue with the found author as the issue author' do
allow(importer.user_finder)
.to receive(:author_id_for)
.with(issue)
.and_return([user.id, true])
expect(Gitlab::GithubImport)
.to receive(:insert_and_return_id)
.with(
{
iid: 42,
title: 'My Issue',
author_id: user.id,
project_id: project.id,
description: 'This is my issue',
milestone_id: milestone.id,
state: :opened,
created_at: created_at,
updated_at: updated_at
},
project.issues
)
.and_call_original
importer.create_issue
end
end
context 'when the issue author could not be found' do
it 'creates the issue with the project creator as the issue author' do
allow(importer.user_finder)
.to receive(:author_id_for)
.with(issue)
.and_return([project.creator_id, false])
expect(Gitlab::GithubImport)
.to receive(:insert_and_return_id)
.with(
{
iid: 42,
title: 'My Issue',
author_id: project.creator_id,
project_id: project.id,
description: "*Created by: alice*\n\nThis is my issue",
milestone_id: milestone.id,
state: :opened,
created_at: created_at,
updated_at: updated_at
},
project.issues
)
.and_call_original
importer.create_issue
end
end
context 'when the import fails due to a foreign key error' do
it 'does not raise any errors' do
allow(importer.user_finder)
.to receive(:author_id_for)
.with(issue)
.and_return([user.id, true])
expect(Gitlab::GithubImport)
.to receive(:insert_and_return_id)
.and_raise(ActiveRecord::InvalidForeignKey, 'invalid foreign key')
expect { importer.create_issue }.not_to raise_error
end
end
it 'produces a valid Issue' do
allow(importer.user_finder)
.to receive(:author_id_for)
.with(issue)
.and_return([user.id, true])
importer.create_issue
expect(project.issues.take).to be_valid
end
it 'returns the ID of the created issue' do
allow(importer.user_finder)
.to receive(:author_id_for)
.with(issue)
.and_return([user.id, true])
expect(importer.create_issue).to be_a_kind_of(Numeric)
end
end
describe '#create_assignees' do
it 'inserts the issue assignees in bulk' do
importer = described_class.new(issue, project, client)
allow(importer.user_finder)
.to receive(:user_id_for)
.ordered.with(issue.assignees[0])
.and_return(4)
allow(importer.user_finder)
.to receive(:user_id_for)
.ordered.with(issue.assignees[1])
.and_return(5)
expect(Gitlab::Database)
.to receive(:bulk_insert)
.with(
IssueAssignee.table_name,
[{ issue_id: 1, user_id: 4 }, { issue_id: 1, user_id: 5 }]
)
importer.create_assignees(1)
end
end
end
require 'spec_helper'
describe Gitlab::GithubImport::Importer::IssuesImporter do
let(:project) { double(:project, id: 4, import_source: 'foo/bar') }
let(:client) { double(:client) }
let(:created_at) { Time.new(2017, 1, 1, 12, 00) }
let(:updated_at) { Time.new(2017, 1, 1, 12, 15) }
let(:github_issue) do
double(
:response,
number: 42,
title: 'My Issue',
body: 'This is my issue',
milestone: double(:milestone, number: 4),
state: 'open',
assignees: [double(:user, id: 4, login: 'alice')],
labels: [double(:label, name: 'bug')],
user: double(:user, id: 4, login: 'alice'),
created_at: created_at,
updated_at: updated_at,
pull_request: false
)
end
describe '#parallel?' do
it 'returns true when running in parallel mode' do
importer = described_class.new(project, client)
expect(importer).to be_parallel
end
it 'returns false when running in sequential mode' do
importer = described_class.new(project, client, parallel: false)
expect(importer).not_to be_parallel
end
end
describe '#execute' do
context 'when running in parallel mode' do
it 'imports issues in parallel' do
importer = described_class.new(project, client)
expect(importer).to receive(:parallel_import)
importer.execute
end
end
context 'when running in sequential mode' do
it 'imports issues in sequence' do
importer = described_class.new(project, client, parallel: false)
expect(importer).to receive(:sequential_import)
importer.execute
end
end
end
describe '#sequential_import' do
it 'imports each issue in sequence' do
importer = described_class.new(project, client, parallel: false)
issue_importer = double(:importer)
allow(importer)
.to receive(:each_object_to_import)
.and_yield(github_issue)
expect(Gitlab::GithubImport::Importer::IssueAndLabelLinksImporter)
.to receive(:new)
.with(
an_instance_of(Gitlab::GithubImport::Representation::Issue),
project,
client
)
.and_return(issue_importer)
expect(issue_importer).to receive(:execute)
importer.sequential_import
end
end
describe '#parallel_import' do
it 'imports each issue in parallel' do
importer = described_class.new(project, client)
allow(importer)
.to receive(:each_object_to_import)
.and_yield(github_issue)
expect(Gitlab::GithubImport::ImportIssueWorker)
.to receive(:perform_async)
.with(project.id, an_instance_of(Hash), an_instance_of(String))
waiter = importer.parallel_import
expect(waiter).to be_an_instance_of(Gitlab::JobWaiter)
expect(waiter.jobs_remaining).to eq(1)
end
end
describe '#id_for_already_imported_cache' do
it 'returns the issue number of the given issue' do
importer = described_class.new(project, client)
expect(importer.id_for_already_imported_cache(github_issue))
.to eq(42)
end
end
end
require 'spec_helper'
describe Gitlab::GithubImport::Importer::LabelLinksImporter do
let(:project) { create(:project) }
let(:client) { double(:client) }
let(:issue) do
double(
:issue,
iid: 4,
label_names: %w[bug],
issuable_type: Issue,
pull_request?: false
)
end
let(:importer) { described_class.new(issue, project, client) }
describe '#execute' do
it 'creates the label links' do
importer = described_class.new(issue, project, client)
expect(importer).to receive(:create_labels)
importer.execute
end
end
describe '#create_labels' do
it 'inserts the label links in bulk' do
expect(importer.label_finder)
.to receive(:id_for)
.with('bug')
.and_return(2)
expect(importer)
.to receive(:find_target_id)
.and_return(1)
Timecop.freeze do
expect(Gitlab::Database)
.to receive(:bulk_insert)
.with(
LabelLink.table_name,
[
{
label_id: 2,
target_id: 1,
target_type: Issue,
created_at: Time.zone.now,
updated_at: Time.zone.now
}
]
)
importer.create_labels
end
end
it 'does not insert label links for non-existing labels' do
expect(importer.label_finder)
.to receive(:id_for)
.with('bug')
.and_return(nil)
expect(Gitlab::Database)
.to receive(:bulk_insert)
.with(LabelLink.table_name, [])
importer.create_labels
end
end
describe '#find_target_id' do
it 'returns the ID of the issuable to create the label link for' do
expect_any_instance_of(Gitlab::GithubImport::IssuableFinder)
.to receive(:database_id)
.and_return(10)
expect(importer.find_target_id).to eq(10)
end
end
end
require 'spec_helper'
describe Gitlab::GithubImport::Importer::LabelsImporter, :clean_gitlab_redis_cache do
let(:project) { create(:project, import_source: 'foo/bar') }
let(:client) { double(:client) }
let(:importer) { described_class.new(project, client) }
describe '#execute' do
it 'imports the labels in bulk' do
label_hash = { title: 'bug', color: '#fffaaa' }
expect(importer)
.to receive(:build_labels)
.and_return([label_hash])
expect(importer)
.to receive(:bulk_insert)
.with(Label, [label_hash])
expect(importer)
.to receive(:build_labels_cache)
importer.execute
end
end
describe '#build_labels' do
it 'returns an Array containnig label rows' do
label = double(:label, name: 'bug', color: 'ffffff')
expect(importer).to receive(:each_label).and_return([label])
rows = importer.build_labels
expect(rows.length).to eq(1)
expect(rows[0][:title]).to eq('bug')
end
it 'does not create labels that already exist' do
create(:label, project: project, title: 'bug')
label = double(:label, name: 'bug', color: 'ffffff')
expect(importer).to receive(:each_label).and_return([label])
expect(importer.build_labels).to be_empty
end
end
describe '#build_labels_cache' do
it 'builds the labels cache' do
expect_any_instance_of(Gitlab::GithubImport::LabelFinder)
.to receive(:build_cache)
importer.build_labels_cache
end
end
describe '#build' do
let(:label_hash) do
importer.build(double(:label, name: 'bug', color: 'ffffff'))
end
it 'returns the attributes of the label as a Hash' do
expect(label_hash).to be_an_instance_of(Hash)
end
context 'the returned Hash' do
it 'includes the label title' do
expect(label_hash[:title]).to eq('bug')
end
it 'includes the label color' do
expect(label_hash[:color]).to eq('#ffffff')
end
it 'includes the project ID' do
expect(label_hash[:project_id]).to eq(project.id)
end
it 'includes the label type' do
expect(label_hash[:type]).to eq('ProjectLabel')
end
it 'includes the created timestamp' do
Timecop.freeze do
expect(label_hash[:created_at]).to eq(Time.zone.now)
end
end
it 'includes the updated timestamp' do
Timecop.freeze do
expect(label_hash[:updated_at]).to eq(Time.zone.now)
end
end
end
end
describe '#each_label' do
it 'returns the labels' do
expect(client)
.to receive(:labels)
.with('foo/bar')
importer.each_label
end
end
end
require 'spec_helper'
describe Gitlab::GithubImport::Importer::MilestonesImporter, :clean_gitlab_redis_cache do
let(:project) { create(:project, import_source: 'foo/bar') }
let(:client) { double(:client) }
let(:importer) { described_class.new(project, client) }
let(:created_at) { Time.new(2017, 1, 1, 12, 00) }
let(:updated_at) { Time.new(2017, 1, 1, 12, 15) }
let(:milestone) do
double(
:milestone,
number: 1,
title: '1.0',
description: 'The first release',
state: 'open',
created_at: created_at,
updated_at: updated_at
)
end
describe '#execute' do
it 'imports the milestones in bulk' do
milestone_hash = { number: 1, title: '1.0' }
expect(importer)
.to receive(:build_milestones)
.and_return([milestone_hash])
expect(importer)
.to receive(:bulk_insert)
.with(Milestone, [milestone_hash])
expect(importer)
.to receive(:build_milestones_cache)
importer.execute
end
end
describe '#build_milestones' do
it 'returns an Array containnig milestone rows' do
expect(importer)
.to receive(:each_milestone)
.and_return([milestone])
rows = importer.build_milestones
expect(rows.length).to eq(1)
expect(rows[0][:title]).to eq('1.0')
end
it 'does not create milestones that already exist' do
create(:milestone, project: project, title: '1.0', iid: 1)
expect(importer)
.to receive(:each_milestone)
.and_return([milestone])
expect(importer.build_milestones).to be_empty
end
end
describe '#build_milestones_cache' do
it 'builds the milestones cache' do
expect_any_instance_of(Gitlab::GithubImport::MilestoneFinder)
.to receive(:build_cache)
importer.build_milestones_cache
end
end
describe '#build' do
let(:milestone_hash) { importer.build(milestone) }
it 'returns the attributes of the milestone as a Hash' do
expect(milestone_hash).to be_an_instance_of(Hash)
end
context 'the returned Hash' do
it 'includes the milestone number' do
expect(milestone_hash[:iid]).to eq(1)
end
it 'includes the milestone title' do
expect(milestone_hash[:title]).to eq('1.0')
end
it 'includes the milestone description' do
expect(milestone_hash[:description]).to eq('The first release')
end
it 'includes the project ID' do
expect(milestone_hash[:project_id]).to eq(project.id)
end
it 'includes the milestone state' do
expect(milestone_hash[:state]).to eq(:active)
end
it 'includes the created timestamp' do
expect(milestone_hash[:created_at]).to eq(created_at)
end
it 'includes the updated timestamp' do
expect(milestone_hash[:updated_at]).to eq(updated_at)
end
end
end
describe '#each_milestone' do
it 'returns the milestones' do
expect(client)
.to receive(:milestones)
.with('foo/bar', state: 'all')
importer.each_milestone
end
end
end
require 'spec_helper'
describe Gitlab::GithubImport::Importer::NoteImporter do
let(:client) { double(:client) }
let(:project) { create(:project) }
let(:user) { create(:user) }
let(:created_at) { Time.new(2017, 1, 1, 12, 00) }
let(:updated_at) { Time.new(2017, 1, 1, 12, 15) }
let(:github_note) do
Gitlab::GithubImport::Representation::Note.new(
noteable_id: 1,
noteable_type: 'Issue',
author: Gitlab::GithubImport::Representation::User.new(id: 4, login: 'alice'),
note: 'This is my note',
created_at: created_at,
updated_at: updated_at,
github_id: 1
)
end
let(:importer) { described_class.new(github_note, project, client) }
describe '#execute' do
context 'when the noteable exists' do
let!(:issue_row) { create(:issue, project: project, iid: 1) }
before do
allow(importer)
.to receive(:find_noteable_id)
.and_return(issue_row.id)
end
context 'when the author could be found' do
it 'imports the note with the found author as the note author' do
expect(importer.user_finder)
.to receive(:author_id_for)
.with(github_note)
.and_return([user.id, true])
expect(Gitlab::Database)
.to receive(:bulk_insert)
.with(
Note.table_name,
[
{
noteable_type: 'Issue',
noteable_id: issue_row.id,
project_id: project.id,
author_id: user.id,
note: 'This is my note',
system: false,
created_at: created_at,
updated_at: updated_at
}
]
)
.and_call_original
importer.execute
end
end
context 'when the note author could not be found' do
it 'imports the note with the project creator as the note author' do
expect(importer.user_finder)
.to receive(:author_id_for)
.with(github_note)
.and_return([project.creator_id, false])
expect(Gitlab::Database)
.to receive(:bulk_insert)
.with(
Note.table_name,
[
{
noteable_type: 'Issue',
noteable_id: issue_row.id,
project_id: project.id,
author_id: project.creator_id,
note: "*Created by: alice*\n\nThis is my note",
system: false,
created_at: created_at,
updated_at: updated_at
}
]
)
.and_call_original
importer.execute
end
end
end
context 'when the noteable does not exist' do
it 'does not import the note' do
expect(Gitlab::Database).not_to receive(:bulk_insert)
importer.execute
end
end
context 'when the import fails due to a foreign key error' do
it 'does not raise any errors' do
issue_row = create(:issue, project: project, iid: 1)
allow(importer)
.to receive(:find_noteable_id)
.and_return(issue_row.id)
allow(importer.user_finder)
.to receive(:author_id_for)
.with(github_note)
.and_return([user.id, true])
expect(Gitlab::Database)
.to receive(:bulk_insert)
.and_raise(ActiveRecord::InvalidForeignKey, 'invalid foreign key')
expect { importer.execute }.not_to raise_error
end
end
it 'produces a valid Note' do
issue_row = create(:issue, project: project, iid: 1)
allow(importer)
.to receive(:find_noteable_id)
.and_return(issue_row.id)
allow(importer.user_finder)
.to receive(:author_id_for)
.with(github_note)
.and_return([user.id, true])
importer.execute
expect(project.notes.take).to be_valid
end
end
describe '#find_noteable_id' do
it 'returns the ID of the noteable' do
expect_any_instance_of(Gitlab::GithubImport::IssuableFinder)
.to receive(:database_id)
.and_return(10)
expect(importer.find_noteable_id).to eq(10)
end
end
end
require 'spec_helper'
describe Gitlab::GithubImport::Importer::NotesImporter do
let(:project) { double(:project, id: 4, import_source: 'foo/bar') }
let(:client) { double(:client) }
let(:github_comment) do
double(
:response,
html_url: 'https://github.com/foo/bar/issues/42',
user: double(:user, id: 4, login: 'alice'),
body: 'Hello world',
created_at: Time.zone.now,
updated_at: Time.zone.now,
id: 1
)
end
describe '#parallel?' do
it 'returns true when running in parallel mode' do
importer = described_class.new(project, client)
expect(importer).to be_parallel
end
it 'returns false when running in sequential mode' do
importer = described_class.new(project, client, parallel: false)
expect(importer).not_to be_parallel
end
end
describe '#execute' do
context 'when running in parallel mode' do
it 'imports notes in parallel' do
importer = described_class.new(project, client)
expect(importer).to receive(:parallel_import)
importer.execute
end
end
context 'when running in sequential mode' do
it 'imports notes in sequence' do
importer = described_class.new(project, client, parallel: false)
expect(importer).to receive(:sequential_import)
importer.execute
end
end
end
describe '#sequential_import' do
it 'imports each note in sequence' do
importer = described_class.new(project, client, parallel: false)
note_importer = double(:note_importer)
allow(importer)
.to receive(:each_object_to_import)
.and_yield(github_comment)
expect(Gitlab::GithubImport::Importer::NoteImporter)
.to receive(:new)
.with(
an_instance_of(Gitlab::GithubImport::Representation::Note),
project,
client
)
.and_return(note_importer)
expect(note_importer).to receive(:execute)
importer.sequential_import
end
end
describe '#parallel_import' do
it 'imports each note in parallel' do
importer = described_class.new(project, client)
allow(importer)
.to receive(:each_object_to_import)
.and_yield(github_comment)
expect(Gitlab::GithubImport::ImportNoteWorker)
.to receive(:perform_async)
.with(project.id, an_instance_of(Hash), an_instance_of(String))
waiter = importer.parallel_import
expect(waiter).to be_an_instance_of(Gitlab::JobWaiter)
expect(waiter.jobs_remaining).to eq(1)
end
end
describe '#id_for_already_imported_cache' do
it 'returns the ID of the given note' do
importer = described_class.new(project, client)
expect(importer.id_for_already_imported_cache(github_comment))
.to eq(1)
end
end
describe '#collection_options' do
it 'returns an empty Hash' do
# For large projects (e.g. kubernetes/kubernetes) GitHub's API may produce
# HTTP 500 errors when using explicit sorting options, regardless of what
# order you sort in. Not using any sorting options at all allows us to
# work around this.
importer = described_class.new(project, client)
expect(importer.collection_options).to eq({})
end
end
end
require 'spec_helper'
describe Gitlab::GithubImport::Importer::PullRequestImporter, :clean_gitlab_redis_cache do
let(:project) { create(:project, :repository) }
let(:client) { double(:client) }
let(:user) { create(:user) }
let(:created_at) { Time.new(2017, 1, 1, 12, 00) }
let(:updated_at) { Time.new(2017, 1, 1, 12, 15) }
let(:merged_at) { Time.new(2017, 1, 1, 12, 17) }
let(:source_commit) { project.repository.commit('feature') }
let(:target_commit) { project.repository.commit('master') }
let(:milestone) { create(:milestone, project: project) }
let(:pull_request) do
alice = Gitlab::GithubImport::Representation::User.new(id: 4, login: 'alice')
Gitlab::GithubImport::Representation::PullRequest.new(
iid: 42,
title: 'My Pull Request',
description: 'This is my pull request',
source_branch: 'feature',
source_branch_sha: source_commit.id,
target_branch: 'master',
target_branch_sha: target_commit.id,
source_repository_id: 400,
target_repository_id: 200,
source_repository_owner: 'alice',
state: :closed,
milestone_number: milestone.iid,
author: alice,
assignee: alice,
created_at: created_at,
updated_at: updated_at,
merged_at: merged_at
)
end
let(:importer) { described_class.new(pull_request, project, client) }
describe '#execute' do
it 'imports the pull request' do
expect(importer)
.to receive(:create_merge_request)
.and_return(10)
expect_any_instance_of(Gitlab::GithubImport::IssuableFinder)
.to receive(:cache_database_id)
.with(10)
importer.execute
end
end
describe '#create_merge_request' do
before do
allow(importer.milestone_finder)
.to receive(:id_for)
.with(pull_request)
.and_return(milestone.id)
end
context 'when the author could be found' do
before do
allow(importer.user_finder)
.to receive(:author_id_for)
.with(pull_request)
.and_return([user.id, true])
allow(importer.user_finder)
.to receive(:assignee_id_for)
.with(pull_request)
.and_return(user.id)
end
it 'imports the pull request with the pull request author as the merge request author' do
expect(Gitlab::GithubImport)
.to receive(:insert_and_return_id)
.with(
{
iid: 42,
title: 'My Pull Request',
description: 'This is my pull request',
source_project_id: project.id,
target_project_id: project.id,
source_branch: 'alice:feature',
target_branch: 'master',
state: :merged,
milestone_id: milestone.id,
author_id: user.id,
assignee_id: user.id,
created_at: created_at,
updated_at: updated_at
},
project.merge_requests
)
.and_call_original
importer.create_merge_request
end
it 'returns the ID of the created merge request' do
id = importer.create_merge_request
expect(id).to be_a_kind_of(Numeric)
end
it 'creates the merge request diffs' do
importer.create_merge_request
mr = project.merge_requests.take
expect(mr.merge_request_diffs.exists?).to eq(true)
end
end
context 'when the author could not be found' do
it 'imports the pull request with the project creator as the merge request author' do
allow(importer.user_finder)
.to receive(:author_id_for)
.with(pull_request)
.and_return([project.creator_id, false])
allow(importer.user_finder)
.to receive(:assignee_id_for)
.with(pull_request)
.and_return(user.id)
expect(Gitlab::GithubImport)
.to receive(:insert_and_return_id)
.with(
{
iid: 42,
title: 'My Pull Request',
description: "*Created by: alice*\n\nThis is my pull request",
source_project_id: project.id,
target_project_id: project.id,
source_branch: 'alice:feature',
target_branch: 'master',
state: :merged,
milestone_id: milestone.id,
author_id: project.creator_id,
assignee_id: user.id,
created_at: created_at,
updated_at: updated_at
},
project.merge_requests
)
.and_call_original
importer.create_merge_request
end
end
context 'when the source and target branch are identical' do
it 'uses a generated source branch name for the merge request' do
allow(importer.user_finder)
.to receive(:author_id_for)
.with(pull_request)
.and_return([user.id, true])
allow(importer.user_finder)
.to receive(:assignee_id_for)
.with(pull_request)
.and_return(user.id)
allow(pull_request)
.to receive(:source_repository_id)
.and_return(pull_request.target_repository_id)
allow(pull_request)
.to receive(:source_branch)
.and_return('master')
expect(Gitlab::GithubImport)
.to receive(:insert_and_return_id)
.with(
{
iid: 42,
title: 'My Pull Request',
description: 'This is my pull request',
source_project_id: project.id,
target_project_id: project.id,
source_branch: 'master-42',
target_branch: 'master',
state: :merged,
milestone_id: milestone.id,
author_id: user.id,
assignee_id: user.id,
created_at: created_at,
updated_at: updated_at
},
project.merge_requests
)
.and_call_original
importer.create_merge_request
end
end
context 'when the import fails due to a foreign key error' do
it 'does not raise any errors' do
allow(importer.user_finder)
.to receive(:author_id_for)
.with(pull_request)
.and_return([user.id, true])
allow(importer.user_finder)
.to receive(:assignee_id_for)
.with(pull_request)
.and_return(user.id)
expect(Gitlab::GithubImport)
.to receive(:insert_and_return_id)
.and_raise(ActiveRecord::InvalidForeignKey, 'invalid foreign key')
expect { importer.create_merge_request }.not_to raise_error
end
end
end
end
require 'spec_helper'
describe Gitlab::GithubImport::Importer::PullRequestsImporter do
let(:project) { create(:project, import_source: 'foo/bar') }
let(:client) { double(:client) }
let(:pull_request) do
double(
:response,
number: 42,
title: 'My Pull Request',
body: 'This is my pull request',
state: 'closed',
head: double(
:head,
sha: '123abc',
ref: 'my-feature',
repo: double(:repo, id: 400),
user: double(:user, id: 4, login: 'alice')
),
base: double(
:base,
sha: '456def',
ref: 'master',
repo: double(:repo, id: 200)
),
milestone: double(:milestone, number: 4),
user: double(:user, id: 4, login: 'alice'),
assignee: double(:user, id: 4, login: 'alice'),
created_at: Time.zone.now,
updated_at: Time.zone.now,
merged_at: Time.zone.now
)
end
describe '#parallel?' do
it 'returns true when running in parallel mode' do
importer = described_class.new(project, client)
expect(importer).to be_parallel
end
it 'returns false when running in sequential mode' do
importer = described_class.new(project, client, parallel: false)
expect(importer).not_to be_parallel
end
end
describe '#execute' do
context 'when running in parallel mode' do
it 'imports pull requests in parallel' do
importer = described_class.new(project, client)
expect(importer).to receive(:parallel_import)
importer.execute
end
end
context 'when running in sequential mode' do
it 'imports pull requests in sequence' do
importer = described_class.new(project, client, parallel: false)
expect(importer).to receive(:sequential_import)
importer.execute
end
end
end
describe '#sequential_import' do
it 'imports each pull request in sequence' do
importer = described_class.new(project, client, parallel: false)
pull_request_importer = double(:pull_request_importer)
allow(importer)
.to receive(:each_object_to_import)
.and_yield(pull_request)
expect(Gitlab::GithubImport::Importer::PullRequestImporter)
.to receive(:new)
.with(
an_instance_of(Gitlab::GithubImport::Representation::PullRequest),
project,
client
)
.and_return(pull_request_importer)
expect(pull_request_importer).to receive(:execute)
importer.sequential_import
end
end
describe '#parallel_import' do
it 'imports each note in parallel' do
importer = described_class.new(project, client)
allow(importer)
.to receive(:each_object_to_import)
.and_yield(pull_request)
expect(Gitlab::GithubImport::ImportPullRequestWorker)
.to receive(:perform_async)
.with(project.id, an_instance_of(Hash), an_instance_of(String))
waiter = importer.parallel_import
expect(waiter).to be_an_instance_of(Gitlab::JobWaiter)
expect(waiter.jobs_remaining).to eq(1)
end
end
describe '#each_object_to_import', :clean_gitlab_redis_cache do
let(:importer) { described_class.new(project, client) }
before do
page = double(:page, objects: [pull_request], number: 1)
expect(client)
.to receive(:each_page)
.with(
:pull_requests,
'foo/bar',
{ state: 'all', sort: 'created', direction: 'asc', page: 1 }
)
.and_yield(page)
end
it 'yields every pull request to the supplied block' do
expect { |b| importer.each_object_to_import(&b) }
.to yield_with_args(pull_request)
end
it 'updates the repository if a pull request was updated after the last clone' do
expect(importer)
.to receive(:update_repository?)
.with(pull_request)
.and_return(true)
expect(importer)
.to receive(:update_repository)
importer.each_object_to_import { }
end
end
describe '#update_repository' do
it 'updates the repository' do
importer = described_class.new(project, client)
expect(project.repository)
.to receive(:fetch_remote)
.with('github', forced: false)
expect(Rails.logger)
.to receive(:info)
.with(an_instance_of(String))
expect(importer.repository_updates_counter)
.to receive(:increment)
.with(project: project.path_with_namespace)
.and_call_original
Timecop.freeze do
importer.update_repository
expect(project.last_repository_updated_at).to eq(Time.zone.now)
end
end
end
describe '#update_repository?' do
let(:importer) { described_class.new(project, client) }
context 'when the pull request was updated after the last update' do
let(:pr) do
double(
:pr,
updated_at: Time.zone.now,
head: double(:head, sha: '123'),
base: double(:base, sha: '456')
)
end
before do
allow(project)
.to receive(:last_repository_updated_at)
.and_return(1.year.ago)
end
it 'returns true when the head SHA is not present' do
expect(importer)
.to receive(:commit_exists?)
.with(pr.head.sha)
.and_return(false)
expect(importer.update_repository?(pr)).to eq(true)
end
it 'returns true when the base SHA is not present' do
expect(importer)
.to receive(:commit_exists?)
.with(pr.head.sha)
.and_return(true)
expect(importer)
.to receive(:commit_exists?)
.with(pr.base.sha)
.and_return(false)
expect(importer.update_repository?(pr)).to eq(true)
end
it 'returns false if both the head and base SHAs are present' do
expect(importer)
.to receive(:commit_exists?)
.with(pr.head.sha)
.and_return(true)
expect(importer)
.to receive(:commit_exists?)
.with(pr.base.sha)
.and_return(true)
expect(importer.update_repository?(pr)).to eq(false)
end
end
context 'when the pull request was updated before the last update' do
it 'returns false' do
pr = double(:pr, updated_at: 1.year.ago)
allow(project)
.to receive(:last_repository_updated_at)
.and_return(Time.zone.now)
expect(importer.update_repository?(pr)).to eq(false)
end
end
end
describe '#commit_exists?' do
let(:importer) { described_class.new(project, client) }
it 'returns true when a commit exists' do
expect(project.repository)
.to receive(:lookup)
.with('123')
.and_return(double(:commit))
expect(importer.commit_exists?('123')).to eq(true)
end
it 'returns false when a commit does not exist' do
expect(project.repository)
.to receive(:lookup)
.with('123')
.and_raise(Rugged::OdbError)
expect(importer.commit_exists?('123')).to eq(false)
end
end
describe '#id_for_already_imported_cache' do
it 'returns the PR number of the given PR' do
importer = described_class.new(project, client)
expect(importer.id_for_already_imported_cache(pull_request))
.to eq(42)
end
end
end
require 'spec_helper'
describe Gitlab::GithubImport::Importer::ReleasesImporter do
let(:project) { create(:project) }
let(:client) { double(:client) }
let(:importer) { described_class.new(project, client) }
let(:created_at) { Time.new(2017, 1, 1, 12, 00) }
let(:updated_at) { Time.new(2017, 1, 1, 12, 15) }
let(:release) do
double(
:release,
tag_name: '1.0',
body: 'This is my release',
created_at: created_at,
updated_at: updated_at
)
end
describe '#execute' do
it 'imports the releases in bulk' do
release_hash = {
tag_name: '1.0',
description: 'This is my release',
created_at: created_at,
updated_at: updated_at
}
expect(importer).to receive(:build_releases).and_return([release_hash])
expect(importer).to receive(:bulk_insert).with(Release, [release_hash])
importer.execute
end
end
describe '#build_releases' do
it 'returns an Array containnig release rows' do
expect(importer).to receive(:each_release).and_return([release])
rows = importer.build_releases
expect(rows.length).to eq(1)
expect(rows[0][:tag]).to eq('1.0')
end
it 'does not create releases that already exist' do
create(:release, project: project, tag: '1.0', description: '1.0')
expect(importer).to receive(:each_release).and_return([release])
expect(importer.build_releases).to be_empty
end
it 'uses a default release description if none is provided' do
expect(release).to receive(:body).and_return('')
expect(importer).to receive(:each_release).and_return([release])
release = importer.build_releases.first
expect(release[:description]).to eq('Release for tag 1.0')
end
end
describe '#build' do
let(:release_hash) { importer.build(release) }
it 'returns the attributes of the release as a Hash' do
expect(release_hash).to be_an_instance_of(Hash)
end
context 'the returned Hash' do
it 'includes the tag name' do
expect(release_hash[:tag]).to eq('1.0')
end
it 'includes the release description' do
expect(release_hash[:description]).to eq('This is my release')
end
it 'includes the project ID' do
expect(release_hash[:project_id]).to eq(project.id)
end
it 'includes the created timestamp' do
expect(release_hash[:created_at]).to eq(created_at)
end
it 'includes the updated timestamp' do
expect(release_hash[:updated_at]).to eq(updated_at)
end
end
end
describe '#each_release' do
let(:release) { double(:release) }
before do
allow(project).to receive(:import_source).and_return('foo/bar')
allow(client)
.to receive(:releases)
.with('foo/bar')
.and_return([release].to_enum)
end
it 'returns an Enumerator' do
expect(importer.each_release).to be_an_instance_of(Enumerator)
end
it 'yields every release to the Enumerator' do
expect(importer.each_release.next).to eq(release)
end
end
describe '#description_for' do
it 'returns the description when present' do
expect(importer.description_for(release)).to eq(release.body)
end
it 'returns a generated description when one is not present' do
allow(release).to receive(:body).and_return('')
expect(importer.description_for(release)).to eq('Release for tag 1.0')
end
end
end
require 'spec_helper'
describe Gitlab::GithubImport::Importer::RepositoryImporter do
let(:repository) { double(:repository) }
let(:client) { double(:client) }
let(:project) do
double(
:project,
import_url: 'foo.git',
import_source: 'foo/bar',
repository_storage_path: 'foo',
disk_path: 'foo',
repository: repository
)
end
let(:importer) { described_class.new(project, client) }
let(:shell_adapter) { Gitlab::Shell.new }
before do
# The method "gitlab_shell" returns a new instance every call, making
# it harder to set expectations. To work around this we'll stub the method
# and return the same instance on every call.
allow(importer).to receive(:gitlab_shell).and_return(shell_adapter)
end
describe '#import_wiki?' do
it 'returns true if the wiki should be imported' do
repo = double(:repo, has_wiki: true)
expect(client)
.to receive(:repository)
.with('foo/bar')
.and_return(repo)
expect(project)
.to receive(:wiki_repository_exists?)
.and_return(false)
expect(importer.import_wiki?).to eq(true)
end
it 'returns false if the GitHub wiki is disabled' do
repo = double(:repo, has_wiki: false)
expect(client)
.to receive(:repository)
.with('foo/bar')
.and_return(repo)
expect(importer.import_wiki?).to eq(false)
end
it 'returns false if the wiki has already been imported' do
repo = double(:repo, has_wiki: true)
expect(client)
.to receive(:repository)
.with('foo/bar')
.and_return(repo)
expect(project)
.to receive(:wiki_repository_exists?)
.and_return(true)
expect(importer.import_wiki?).to eq(false)
end
end
describe '#execute' do
it 'imports the repository and wiki' do
expect(repository)
.to receive(:empty_repo?)
.and_return(true)
expect(importer)
.to receive(:import_wiki?)
.and_return(true)
expect(importer)
.to receive(:import_repository)
.and_return(true)
expect(importer)
.to receive(:import_wiki_repository)
.and_return(true)
expect(importer)
.to receive(:update_clone_time)
expect(importer.execute).to eq(true)
end
it 'does not import the repository if it already exists' do
expect(repository)
.to receive(:empty_repo?)
.and_return(false)
expect(importer)
.to receive(:import_wiki?)
.and_return(true)
expect(importer)
.not_to receive(:import_repository)
expect(importer)
.to receive(:import_wiki_repository)
.and_return(true)
expect(importer)
.to receive(:update_clone_time)
expect(importer.execute).to eq(true)
end
it 'does not import the wiki if it is disabled' do
expect(repository)
.to receive(:empty_repo?)
.and_return(true)
expect(importer)
.to receive(:import_wiki?)
.and_return(false)
expect(importer)
.to receive(:import_repository)
.and_return(true)
expect(importer)
.to receive(:update_clone_time)
expect(importer)
.not_to receive(:import_wiki_repository)
expect(importer.execute).to eq(true)
end
it 'does not import the wiki if the repository could not be imported' do
expect(repository)
.to receive(:empty_repo?)
.and_return(true)
expect(importer)
.to receive(:import_wiki?)
.and_return(true)
expect(importer)
.to receive(:import_repository)
.and_return(false)
expect(importer)
.not_to receive(:update_clone_time)
expect(importer)
.not_to receive(:import_wiki_repository)
expect(importer.execute).to eq(false)
end
end
describe '#import_repository' do
it 'imports the repository' do
expect(project)
.to receive(:ensure_repository)
expect(importer)
.to receive(:configure_repository_remote)
expect(repository)
.to receive(:fetch_remote)
.with('github', forced: true)
expect(importer.import_repository).to eq(true)
end
it 'marks the import as failed when an error was raised' do
expect(project).to receive(:ensure_repository)
.and_raise(Gitlab::Git::Repository::NoRepository)
expect(importer)
.to receive(:fail_import)
.and_return(false)
expect(importer.import_repository).to eq(false)
end
end
describe '#configure_repository_remote' do
it 'configures the remote details' do
expect(repository)
.to receive(:remote_exists?)
.with('github')
.and_return(false)
expect(repository)
.to receive(:add_remote)
.with('github', 'foo.git')
expect(repository)
.to receive(:set_import_remote_as_mirror)
.with('github')
expect(repository)
.to receive(:add_remote_fetch_config)
importer.configure_repository_remote
end
it 'does not configure the remote if already configured' do
expect(repository)
.to receive(:remote_exists?)
.with('github')
.and_return(true)
expect(repository)
.not_to receive(:add_remote)
importer.configure_repository_remote
end
end
describe '#import_wiki_repository' do
it 'imports the wiki repository' do
expect(importer.gitlab_shell)
.to receive(:import_repository)
.with('foo', 'foo.wiki', 'foo.wiki.git')
expect(importer.import_wiki_repository).to eq(true)
end
it 'marks the import as failed if an error was raised' do
expect(importer.gitlab_shell)
.to receive(:import_repository)
.and_raise(Gitlab::Shell::Error)
expect(importer)
.to receive(:fail_import)
.and_return(false)
expect(importer.import_wiki_repository).to eq(false)
end
end
describe '#fail_import' do
it 'marks the import as failed' do
expect(project).to receive(:mark_import_as_failed).with('foo')
expect(importer.fail_import('foo')).to eq(false)
end
end
describe '#update_clone_time' do
it 'sets the timestamp for when the cloning process finished' do
Timecop.freeze do
expect(project)
.to receive(:update_column)
.with(:last_repository_updated_at, Time.zone.now)
importer.update_clone_time
end
end
end
end
require 'spec_helper'
describe Gitlab::GithubImport::IssuableFinder, :clean_gitlab_redis_cache do
let(:project) { double(:project, id: 4) }
let(:issue) do
double(:issue, issuable_type: MergeRequest, iid: 1)
end
let(:finder) { described_class.new(project, issue) }
describe '#database_id' do
it 'returns nil when no cache is in place' do
expect(finder.database_id).to be_nil
end
it 'returns the ID of an issuable when the cache is in place' do
finder.cache_database_id(10)
expect(finder.database_id).to eq(10)
end
it 'raises TypeError when the object is not supported' do
finder = described_class.new(project, double(:issue))
expect { finder.database_id }.to raise_error(TypeError)
end
end
describe '#cache_database_id' do
it 'caches the ID of a database row' do
expect(Gitlab::GithubImport::Caching)
.to receive(:write)
.with('github-import/issuable-finder/4/MergeRequest/1', 10)
finder.cache_database_id(10)
end
end
end
require 'spec_helper'
describe Gitlab::GithubImport::LabelFinder, :clean_gitlab_redis_cache do
let(:project) { create(:project) }
let(:finder) { described_class.new(project) }
let!(:bug) { create(:label, project: project, name: 'Bug') }
let!(:feature) { create(:label, project: project, name: 'Feature') }
describe '#id_for' do
context 'with a cache in place' do
before do
finder.build_cache
end
it 'returns the ID of the given label' do
expect(finder.id_for(feature.name)).to eq(feature.id)
end
it 'returns nil for an empty cache key' do
key = finder.cache_key_for(bug.name)
Gitlab::GithubImport::Caching.write(key, '')
expect(finder.id_for(bug.name)).to be_nil
end
it 'returns nil for a non existing label name' do
expect(finder.id_for('kittens')).to be_nil
end
end
context 'without a cache in place' do
it 'returns nil for a label' do
expect(finder.id_for(feature.name)).to be_nil
end
end
end
describe '#build_cache' do
it 'builds the cache of all project labels' do
expect(Gitlab::GithubImport::Caching)
.to receive(:write_multiple)
.with(
{
"github-import/label-finder/#{project.id}/Bug" => bug.id,
"github-import/label-finder/#{project.id}/Feature" => feature.id
}
)
.and_call_original
finder.build_cache
end
end
describe '#cache_key_for' do
it 'returns the cache key for a label name' do
expect(finder.cache_key_for('foo'))
.to eq("github-import/label-finder/#{project.id}/foo")
end
end
end
require 'spec_helper'
describe Gitlab::GithubImport::MarkdownText do
describe '.format' do
it 'formats the text' do
author = double(:author, login: 'Alice')
text = described_class.format('Hello', author)
expect(text).to eq("*Created by: Alice*\n\nHello")
end
end
describe '#to_s' do
it 'returns the text when the author was found' do
author = double(:author, login: 'Alice')
text = described_class.new('Hello', author, true)
expect(text.to_s).to eq('Hello')
end
it 'returns the text with an extra header when the author was not found' do
author = double(:author, login: 'Alice')
text = described_class.new('Hello', author)
expect(text.to_s).to eq("*Created by: Alice*\n\nHello")
end
end
end
require 'spec_helper'
describe Gitlab::GithubImport::MilestoneFinder, :clean_gitlab_redis_cache do
let!(:project) { create(:project) }
let!(:milestone) { create(:milestone, project: project) }
let(:finder) { described_class.new(project) }
describe '#id_for' do
let(:issuable) { double(:issuable, milestone_number: milestone.iid) }
context 'with a cache in place' do
before do
finder.build_cache
end
it 'returns the milestone ID of the given issuable' do
expect(finder.id_for(issuable)).to eq(milestone.id)
end
it 'returns nil for an empty cache key' do
key = finder.cache_key_for(milestone.iid)
Gitlab::GithubImport::Caching.write(key, '')
expect(finder.id_for(issuable)).to be_nil
end
it 'returns nil for an issuable with a non-existing milestone' do
expect(finder.id_for(double(:issuable, milestone_number: 5))).to be_nil
end
end
context 'without a cache in place' do
it 'returns nil' do
expect(finder.id_for(issuable)).to be_nil
end
end
end
describe '#build_cache' do
it 'builds the cache of all project milestones' do
expect(Gitlab::GithubImport::Caching)
.to receive(:write_multiple)
.with("github-import/milestone-finder/#{project.id}/1" => milestone.id)
.and_call_original
finder.build_cache
end
end
describe '#cache_key_for' do
it 'returns the cache key for an IID' do
expect(finder.cache_key_for(10))
.to eq("github-import/milestone-finder/#{project.id}/10")
end
end
end
require 'spec_helper'
describe Gitlab::GithubImport::PageCounter, :clean_gitlab_redis_cache do
let(:project) { double(:project, id: 1) }
let(:counter) { described_class.new(project, :issues) }
describe '#initialize' do
it 'sets the initial page number to 1 when no value is cached' do
expect(counter.current).to eq(1)
end
it 'sets the initial page number to the cached value when one is present' do
Gitlab::GithubImport::Caching.write(counter.cache_key, 2)
expect(described_class.new(project, :issues).current).to eq(2)
end
end
describe '#set' do
it 'overwrites the page number when the given number is greater than the current number' do
counter.set(4)
expect(counter.current).to eq(4)
end
it 'does not overwrite the page number when the given number is lower than the current number' do
counter.set(2)
counter.set(1)
expect(counter.current).to eq(2)
end
end
end
require 'spec_helper'
describe Gitlab::GithubImport::ParallelImporter do
describe '.async?' do
it 'returns true' do
expect(described_class).to be_async
end
end
describe '#execute', :clean_gitlab_redis_shared_state do
let(:project) { create(:project) }
let(:importer) { described_class.new(project) }
before do
expect(Gitlab::GithubImport::Stage::ImportRepositoryWorker)
.to receive(:perform_async)
.with(project.id)
.and_return('123')
end
it 'schedules the importing of the repository' do
expect(importer.execute).to eq(true)
end
it 'sets the JID in Redis' do
expect(Gitlab::SidekiqStatus)
.to receive(:set)
.with("github-importer/#{project.id}", StuckImportJobsWorker::IMPORT_JOBS_EXPIRATION)
.and_call_original
importer.execute
end
it 'updates the import JID of the project' do
importer.execute
expect(project.import_jid).to eq("github-importer/#{project.id}")
end
end
end
require 'spec_helper'
describe Gitlab::GithubImport::ParallelScheduling do
let(:importer_class) do
Class.new do
include(Gitlab::GithubImport::ParallelScheduling)
def collection_method
:issues
end
end
end
let(:project) { double(:project, id: 4, import_source: 'foo/bar') }
let(:client) { double(:client) }
describe '#parallel?' do
it 'returns true when running in parallel mode' do
expect(importer_class.new(project, client)).to be_parallel
end
it 'returns false when running in sequential mode' do
importer = importer_class.new(project, client, parallel: false)
expect(importer).not_to be_parallel
end
end
describe '#execute' do
it 'imports data in parallel when running in parallel mode' do
importer = importer_class.new(project, client)
waiter = double(:waiter)
expect(importer)
.to receive(:parallel_import)
.and_return(waiter)
expect(importer.execute)
.to eq(waiter)
end
it 'imports data in parallel when running in sequential mode' do
importer = importer_class.new(project, client, parallel: false)
expect(importer)
.to receive(:sequential_import)
.and_return([])
expect(importer.execute)
.to eq([])
end
it 'expires the cache used for tracking already imported objects' do
importer = importer_class.new(project, client)
expect(importer).to receive(:parallel_import)
expect(Gitlab::GithubImport::Caching)
.to receive(:expire)
.with(importer.already_imported_cache_key, a_kind_of(Numeric))
importer.execute
end
end
describe '#sequential_import' do
let(:importer) { importer_class.new(project, client, parallel: false) }
it 'imports data in sequence' do
repr_class = double(:representation_class)
repr_instance = double(:representation_instance)
gh_importer = double(:github_importer)
gh_importer_instance = double(:github_importer_instance)
object = double(:object)
expect(importer)
.to receive(:each_object_to_import)
.and_yield(object)
expect(importer)
.to receive(:representation_class)
.and_return(repr_class)
expect(repr_class)
.to receive(:from_api_response)
.with(object)
.and_return(repr_instance)
expect(importer)
.to receive(:importer_class)
.and_return(gh_importer)
expect(gh_importer)
.to receive(:new)
.with(repr_instance, project, client)
.and_return(gh_importer_instance)
expect(gh_importer_instance)
.to receive(:execute)
importer.sequential_import
end
end
describe '#parallel_import' do
let(:importer) { importer_class.new(project, client) }
it 'imports data in parallel' do
repr_class = double(:representation)
worker_class = double(:worker)
object = double(:object)
expect(importer)
.to receive(:each_object_to_import)
.and_yield(object)
expect(importer)
.to receive(:representation_class)
.and_return(repr_class)
expect(importer)
.to receive(:sidekiq_worker_class)
.and_return(worker_class)
expect(repr_class)
.to receive(:from_api_response)
.with(object)
.and_return({ title: 'Foo' })
expect(worker_class)
.to receive(:perform_async)
.with(project.id, { title: 'Foo' }, an_instance_of(String))
expect(importer.parallel_import)
.to be_an_instance_of(Gitlab::JobWaiter)
end
end
describe '#each_object_to_import' do
let(:importer) { importer_class.new(project, client) }
let(:object) { double(:object) }
before do
expect(importer)
.to receive(:collection_options)
.and_return({ state: 'all' })
end
it 'yields every object to import' do
page = double(:page, objects: [object], number: 1)
expect(client)
.to receive(:each_page)
.with(:issues, 'foo/bar', { state: 'all', page: 1 })
.and_yield(page)
expect(importer.page_counter)
.to receive(:set)
.with(1)
.and_return(true)
expect(importer)
.to receive(:already_imported?)
.with(object)
.and_return(false)
expect(importer)
.to receive(:mark_as_imported)
.with(object)
expect { |b| importer.each_object_to_import(&b) }
.to yield_with_args(object)
end
it 'resumes from the last page' do
page = double(:page, objects: [object], number: 2)
expect(importer.page_counter)
.to receive(:current)
.and_return(2)
expect(client)
.to receive(:each_page)
.with(:issues, 'foo/bar', { state: 'all', page: 2 })
.and_yield(page)
expect(importer.page_counter)
.to receive(:set)
.with(2)
.and_return(true)
expect(importer)
.to receive(:already_imported?)
.with(object)
.and_return(false)
expect(importer)
.to receive(:mark_as_imported)
.with(object)
expect { |b| importer.each_object_to_import(&b) }
.to yield_with_args(object)
end
it 'does not yield any objects if the page number was not set' do
page = double(:page, objects: [object], number: 1)
expect(client)
.to receive(:each_page)
.with(:issues, 'foo/bar', { state: 'all', page: 1 })
.and_yield(page)
expect(importer.page_counter)
.to receive(:set)
.with(1)
.and_return(false)
expect { |b| importer.each_object_to_import(&b) }
.not_to yield_control
end
it 'does not yield the object if it was already imported' do
page = double(:page, objects: [object], number: 1)
expect(client)
.to receive(:each_page)
.with(:issues, 'foo/bar', { state: 'all', page: 1 })
.and_yield(page)
expect(importer.page_counter)
.to receive(:set)
.with(1)
.and_return(true)
expect(importer)
.to receive(:already_imported?)
.with(object)
.and_return(true)
expect(importer)
.not_to receive(:mark_as_imported)
expect { |b| importer.each_object_to_import(&b) }
.not_to yield_control
end
end
describe '#already_imported?', :clean_gitlab_redis_cache do
let(:importer) { importer_class.new(project, client) }
it 'returns false when an object has not yet been imported' do
object = double(:object, id: 10)
expect(importer)
.to receive(:id_for_already_imported_cache)
.with(object)
.and_return(object.id)
expect(importer.already_imported?(object))
.to eq(false)
end
it 'returns true when an object has already been imported' do
object = double(:object, id: 10)
allow(importer)
.to receive(:id_for_already_imported_cache)
.with(object)
.and_return(object.id)
importer.mark_as_imported(object)
expect(importer.already_imported?(object))
.to eq(true)
end
end
describe '#mark_as_imported', :clean_gitlab_redis_cache do
it 'marks an object as already imported' do
object = double(:object, id: 10)
importer = importer_class.new(project, client)
expect(importer)
.to receive(:id_for_already_imported_cache)
.with(object)
.and_return(object.id)
expect(Gitlab::GithubImport::Caching)
.to receive(:set_add)
.with(importer.already_imported_cache_key, object.id)
.and_call_original
importer.mark_as_imported(object)
end
end
end
require 'spec_helper'
describe Gitlab::GithubImport::Representation::DiffNote do
let(:hunk) do
'@@ -1 +1 @@
-Hello
+Hello world'
end
let(:created_at) { Time.new(2017, 1, 1, 12, 00) }
let(:updated_at) { Time.new(2017, 1, 1, 12, 15) }
shared_examples 'a DiffNote' do
it 'returns an instance of DiffNote' do
expect(note).to be_an_instance_of(described_class)
end
context 'the returned DiffNote' do
it 'includes the number of the note' do
expect(note.noteable_id).to eq(42)
end
it 'includes the file path of the diff' do
expect(note.file_path).to eq('README.md')
end
it 'includes the commit ID' do
expect(note.commit_id).to eq('123abc')
end
it 'includes the user details' do
expect(note.author)
.to be_an_instance_of(Gitlab::GithubImport::Representation::User)
expect(note.author.id).to eq(4)
expect(note.author.login).to eq('alice')
end
it 'includes the note body' do
expect(note.note).to eq('Hello world')
end
it 'includes the created timestamp' do
expect(note.created_at).to eq(created_at)
end
it 'includes the updated timestamp' do
expect(note.updated_at).to eq(updated_at)
end
it 'includes the GitHub ID' do
expect(note.github_id).to eq(1)
end
it 'returns the noteable type' do
expect(note.noteable_type).to eq('MergeRequest')
end
end
end
describe '.from_api_response' do
let(:response) do
double(
:response,
html_url: 'https://github.com/foo/bar/pull/42',
path: 'README.md',
commit_id: '123abc',
diff_hunk: hunk,
user: double(:user, id: 4, login: 'alice'),
body: 'Hello world',
created_at: created_at,
updated_at: updated_at,
id: 1
)
end
it_behaves_like 'a DiffNote' do
let(:note) { described_class.from_api_response(response) }
end
it 'does not set the user if the response did not include a user' do
allow(response)
.to receive(:user)
.and_return(nil)
note = described_class.from_api_response(response)
expect(note.author).to be_nil
end
end
describe '.from_json_hash' do
it_behaves_like 'a DiffNote' do
let(:hash) do
{
'noteable_type' => 'MergeRequest',
'noteable_id' => 42,
'file_path' => 'README.md',
'commit_id' => '123abc',
'diff_hunk' => hunk,
'author' => { 'id' => 4, 'login' => 'alice' },
'note' => 'Hello world',
'created_at' => created_at.to_s,
'updated_at' => updated_at.to_s,
'github_id' => 1
}
end
let(:note) { described_class.from_json_hash(hash) }
end
it 'does not convert the author if it was not specified' do
hash = {
'noteable_type' => 'MergeRequest',
'noteable_id' => 42,
'file_path' => 'README.md',
'commit_id' => '123abc',
'diff_hunk' => hunk,
'note' => 'Hello world',
'created_at' => created_at.to_s,
'updated_at' => updated_at.to_s,
'github_id' => 1
}
note = described_class.from_json_hash(hash)
expect(note.author).to be_nil
end
end
describe '#line_code' do
it 'returns a String' do
note = described_class.new(diff_hunk: hunk, file_path: 'README.md')
expect(note.line_code).to be_an_instance_of(String)
end
end
describe '#diff_hash' do
it 'returns a Hash containing the diff details' do
note = described_class.from_json_hash(
'noteable_type' => 'MergeRequest',
'noteable_id' => 42,
'file_path' => 'README.md',
'commit_id' => '123abc',
'diff_hunk' => hunk,
'author' => { 'id' => 4, 'login' => 'alice' },
'note' => 'Hello world',
'created_at' => created_at.to_s,
'updated_at' => updated_at.to_s,
'github_id' => 1
)
expect(note.diff_hash).to eq(
diff: hunk,
new_path: 'README.md',
old_path: 'README.md',
a_mode: '100644',
b_mode: '100644',
new_file: false
)
end
end
end
require 'spec_helper'
describe Gitlab::GithubImport::Representation::ExposeAttribute do
it 'defines a getter method that returns an attribute value' do
klass = Class.new do
include Gitlab::GithubImport::Representation::ExposeAttribute
expose_attribute :number
attr_reader :attributes
def initialize
@attributes = { number: 42 }
end
end
expect(klass.new.number).to eq(42)
end
end
require 'spec_helper'
describe Gitlab::GithubImport::Representation::Issue do
let(:created_at) { Time.new(2017, 1, 1, 12, 00) }
let(:updated_at) { Time.new(2017, 1, 1, 12, 15) }
shared_examples 'an Issue' do
it 'returns an instance of Issue' do
expect(issue).to be_an_instance_of(described_class)
end
context 'the returned Issue' do
it 'includes the issue number' do
expect(issue.iid).to eq(42)
end
it 'includes the issue title' do
expect(issue.title).to eq('My Issue')
end
it 'includes the issue description' do
expect(issue.description).to eq('This is my issue')
end
it 'includes the milestone number' do
expect(issue.milestone_number).to eq(4)
end
it 'includes the issue state' do
expect(issue.state).to eq(:opened)
end
it 'includes the issue assignees' do
expect(issue.assignees[0])
.to be_an_instance_of(Gitlab::GithubImport::Representation::User)
expect(issue.assignees[0].id).to eq(4)
expect(issue.assignees[0].login).to eq('alice')
end
it 'includes the label names' do
expect(issue.label_names).to eq(%w[bug])
end
it 'includes the author details' do
expect(issue.author)
.to be_an_instance_of(Gitlab::GithubImport::Representation::User)
expect(issue.author.id).to eq(4)
expect(issue.author.login).to eq('alice')
end
it 'includes the created timestamp' do
expect(issue.created_at).to eq(created_at)
end
it 'includes the updated timestamp' do
expect(issue.updated_at).to eq(updated_at)
end
it 'is not a pull request' do
expect(issue.pull_request?).to eq(false)
end
end
end
describe '.from_api_response' do
let(:response) do
double(
:response,
number: 42,
title: 'My Issue',
body: 'This is my issue',
milestone: double(:milestone, number: 4),
state: 'open',
assignees: [double(:user, id: 4, login: 'alice')],
labels: [double(:label, name: 'bug')],
user: double(:user, id: 4, login: 'alice'),
created_at: created_at,
updated_at: updated_at,
pull_request: false
)
end
it_behaves_like 'an Issue' do
let(:issue) { described_class.from_api_response(response) }
end
it 'does not set the user if the response did not include a user' do
allow(response)
.to receive(:user)
.and_return(nil)
issue = described_class.from_api_response(response)
expect(issue.author).to be_nil
end
end
describe '.from_json_hash' do
it_behaves_like 'an Issue' do
let(:hash) do
{
'iid' => 42,
'title' => 'My Issue',
'description' => 'This is my issue',
'milestone_number' => 4,
'state' => 'opened',
'assignees' => [{ 'id' => 4, 'login' => 'alice' }],
'label_names' => %w[bug],
'author' => { 'id' => 4, 'login' => 'alice' },
'created_at' => created_at.to_s,
'updated_at' => updated_at.to_s,
'pull_request' => false
}
end
let(:issue) { described_class.from_json_hash(hash) }
end
it 'does not convert the author if it was not specified' do
hash = {
'iid' => 42,
'title' => 'My Issue',
'description' => 'This is my issue',
'milestone_number' => 4,
'state' => 'opened',
'assignees' => [{ 'id' => 4, 'login' => 'alice' }],
'label_names' => %w[bug],
'created_at' => created_at.to_s,
'updated_at' => updated_at.to_s,
'pull_request' => false
}
issue = described_class.from_json_hash(hash)
expect(issue.author).to be_nil
end
end
describe '#labels?' do
it 'returns true when the issue has labels assigned' do
issue = described_class.new(label_names: %w[bug])
expect(issue.labels?).to eq(true)
end
it 'returns false when the issue has no labels assigned' do
issue = described_class.new(label_names: [])
expect(issue.labels?).to eq(false)
end
end
describe '#pull_request?' do
it 'returns false for an issue' do
issue = described_class.new(pull_request: false)
expect(issue.pull_request?).to eq(false)
end
it 'returns true for a pull request' do
issue = described_class.new(pull_request: true)
expect(issue.pull_request?).to eq(true)
end
end
describe '#truncated_title' do
it 'truncates the title to 255 characters' do
object = described_class.new(title: 'm' * 300)
expect(object.truncated_title.length).to eq(255)
end
it 'does not truncate the title if it is shorter than 255 characters' do
object = described_class.new(title: 'foo')
expect(object.truncated_title).to eq('foo')
end
end
end
require 'spec_helper'
describe Gitlab::GithubImport::Representation::Note do
let(:created_at) { Time.new(2017, 1, 1, 12, 00) }
let(:updated_at) { Time.new(2017, 1, 1, 12, 15) }
shared_examples 'a Note' do
it 'returns an instance of Note' do
expect(note).to be_an_instance_of(described_class)
end
context 'the returned Note' do
it 'includes the noteable ID' do
expect(note.noteable_id).to eq(42)
end
it 'includes the noteable type' do
expect(note.noteable_type).to eq('Issue')
end
it 'includes the author details' do
expect(note.author)
.to be_an_instance_of(Gitlab::GithubImport::Representation::User)
expect(note.author.id).to eq(4)
expect(note.author.login).to eq('alice')
end
it 'includes the note body' do
expect(note.note).to eq('Hello world')
end
it 'includes the created timestamp' do
expect(note.created_at).to eq(created_at)
end
it 'includes the updated timestamp' do
expect(note.updated_at).to eq(updated_at)
end
it 'includes the GitHub ID' do
expect(note.github_id).to eq(1)
end
end
end
describe '.from_api_response' do
let(:response) do
double(
:response,
html_url: 'https://github.com/foo/bar/issues/42',
user: double(:user, id: 4, login: 'alice'),
body: 'Hello world',
created_at: created_at,
updated_at: updated_at,
id: 1
)
end
it_behaves_like 'a Note' do
let(:note) { described_class.from_api_response(response) }
end
it 'does not set the user if the response did not include a user' do
allow(response)
.to receive(:user)
.and_return(nil)
note = described_class.from_api_response(response)
expect(note.author).to be_nil
end
end
describe '.from_json_hash' do
it_behaves_like 'a Note' do
let(:hash) do
{
'noteable_id' => 42,
'noteable_type' => 'Issue',
'author' => { 'id' => 4, 'login' => 'alice' },
'note' => 'Hello world',
'created_at' => created_at.to_s,
'updated_at' => updated_at.to_s,
'github_id' => 1
}
end
let(:note) { described_class.from_json_hash(hash) }
end
it 'does not convert the author if it was not specified' do
hash = {
'noteable_id' => 42,
'noteable_type' => 'Issue',
'note' => 'Hello world',
'created_at' => created_at.to_s,
'updated_at' => updated_at.to_s,
'github_id' => 1
}
note = described_class.from_json_hash(hash)
expect(note.author).to be_nil
end
end
end
require 'spec_helper'
describe Gitlab::GithubImport::Representation::PullRequest do
let(:created_at) { Time.new(2017, 1, 1, 12, 00) }
let(:updated_at) { Time.new(2017, 1, 1, 12, 15) }
let(:merged_at) { Time.new(2017, 1, 1, 12, 17) }
shared_examples 'a PullRequest' do
it 'returns an instance of PullRequest' do
expect(pr).to be_an_instance_of(described_class)
end
context 'the returned PullRequest' do
it 'includes the pull request number' do
expect(pr.iid).to eq(42)
end
it 'includes the pull request title' do
expect(pr.title).to eq('My Pull Request')
end
it 'includes the pull request description' do
expect(pr.description).to eq('This is my pull request')
end
it 'includes the source branch name' do
expect(pr.source_branch).to eq('my-feature')
end
it 'includes the source branch SHA' do
expect(pr.source_branch_sha).to eq('123abc')
end
it 'includes the target branch name' do
expect(pr.target_branch).to eq('master')
end
it 'includes the target branch SHA' do
expect(pr.target_branch_sha).to eq('456def')
end
it 'includes the milestone number' do
expect(pr.milestone_number).to eq(4)
end
it 'includes the user details' do
expect(pr.author)
.to be_an_instance_of(Gitlab::GithubImport::Representation::User)
expect(pr.author.id).to eq(4)
expect(pr.author.login).to eq('alice')
end
it 'includes the assignee details' do
expect(pr.assignee)
.to be_an_instance_of(Gitlab::GithubImport::Representation::User)
expect(pr.assignee.id).to eq(4)
expect(pr.assignee.login).to eq('alice')
end
it 'includes the created timestamp' do
expect(pr.created_at).to eq(created_at)
end
it 'includes the updated timestamp' do
expect(pr.updated_at).to eq(updated_at)
end
it 'includes the merged timestamp' do
expect(pr.merged_at).to eq(merged_at)
end
it 'includes the source repository ID' do
expect(pr.source_repository_id).to eq(400)
end
it 'includes the target repository ID' do
expect(pr.target_repository_id).to eq(200)
end
it 'includes the source repository owner name' do
expect(pr.source_repository_owner).to eq('alice')
end
it 'includes the pull request state' do
expect(pr.state).to eq(:merged)
end
end
end
describe '.from_api_response' do
let(:response) do
double(
:response,
number: 42,
title: 'My Pull Request',
body: 'This is my pull request',
state: 'closed',
head: double(
:head,
sha: '123abc',
ref: 'my-feature',
repo: double(:repo, id: 400),
user: double(:user, id: 4, login: 'alice')
),
base: double(
:base,
sha: '456def',
ref: 'master',
repo: double(:repo, id: 200)
),
milestone: double(:milestone, number: 4),
user: double(:user, id: 4, login: 'alice'),
assignee: double(:user, id: 4, login: 'alice'),
created_at: created_at,
updated_at: updated_at,
merged_at: merged_at
)
end
it_behaves_like 'a PullRequest' do
let(:pr) { described_class.from_api_response(response) }
end
it 'does not set the user if the response did not include a user' do
allow(response)
.to receive(:user)
.and_return(nil)
pr = described_class.from_api_response(response)
expect(pr.author).to be_nil
end
end
describe '.from_json_hash' do
it_behaves_like 'a PullRequest' do
let(:hash) do
{
'iid' => 42,
'title' => 'My Pull Request',
'description' => 'This is my pull request',
'source_branch' => 'my-feature',
'source_branch_sha' => '123abc',
'target_branch' => 'master',
'target_branch_sha' => '456def',
'source_repository_id' => 400,
'target_repository_id' => 200,
'source_repository_owner' => 'alice',
'state' => 'closed',
'milestone_number' => 4,
'author' => { 'id' => 4, 'login' => 'alice' },
'assignee' => { 'id' => 4, 'login' => 'alice' },
'created_at' => created_at.to_s,
'updated_at' => updated_at.to_s,
'merged_at' => merged_at.to_s
}
end
let(:pr) { described_class.from_json_hash(hash) }
end
it 'does not convert the author if it was not specified' do
hash = {
'iid' => 42,
'title' => 'My Pull Request',
'description' => 'This is my pull request',
'source_branch' => 'my-feature',
'source_branch_sha' => '123abc',
'target_branch' => 'master',
'target_branch_sha' => '456def',
'source_repository_id' => 400,
'target_repository_id' => 200,
'source_repository_owner' => 'alice',
'state' => 'closed',
'milestone_number' => 4,
'assignee' => { 'id' => 4, 'login' => 'alice' },
'created_at' => created_at.to_s,
'updated_at' => updated_at.to_s,
'merged_at' => merged_at.to_s
}
pr = described_class.from_json_hash(hash)
expect(pr.author).to be_nil
end
end
describe '#state' do
it 'returns :opened for an open pull request' do
pr = described_class.new(state: :opened)
expect(pr.state).to eq(:opened)
end
it 'returns :closed for a closed pull request' do
pr = described_class.new(state: :closed)
expect(pr.state).to eq(:closed)
end
it 'returns :merged for a merged pull request' do
pr = described_class.new(state: :closed, merged_at: merged_at)
expect(pr.state).to eq(:merged)
end
end
describe '#cross_project?' do
it 'returns false for a pull request submitted from the target project' do
pr = described_class.new(source_repository_id: 1, target_repository_id: 1)
expect(pr).not_to be_cross_project
end
it 'returns true for a pull request submitted from a different project' do
pr = described_class.new(source_repository_id: 1, target_repository_id: 2)
expect(pr).to be_cross_project
end
it 'returns true if no source repository is present' do
pr = described_class.new(target_repository_id: 2)
expect(pr).to be_cross_project
end
end
describe '#formatted_source_branch' do
context 'for a cross-project pull request' do
it 'includes the owner name in the branch name' do
pr = described_class.new(
source_repository_owner: 'foo',
source_branch: 'branch',
target_branch: 'master',
source_repository_id: 1,
target_repository_id: 2
)
expect(pr.formatted_source_branch).to eq('foo:branch')
end
end
context 'for a regular pull request' do
it 'returns the source branch name' do
pr = described_class.new(
source_repository_owner: 'foo',
source_branch: 'branch',
target_branch: 'master',
source_repository_id: 1,
target_repository_id: 1
)
expect(pr.formatted_source_branch).to eq('branch')
end
end
context 'for a pull request with the same source and target branches' do
it 'returns a generated source branch name' do
pr = described_class.new(
iid: 1,
source_repository_owner: 'foo',
source_branch: 'branch',
target_branch: 'branch',
source_repository_id: 1,
target_repository_id: 1
)
expect(pr.formatted_source_branch).to eq('branch-1')
end
end
end
describe '#truncated_title' do
it 'truncates the title to 255 characters' do
object = described_class.new(title: 'm' * 300)
expect(object.truncated_title.length).to eq(255)
end
it 'does not truncate the title if it is shorter than 255 characters' do
object = described_class.new(title: 'foo')
expect(object.truncated_title).to eq('foo')
end
end
end
require 'spec_helper'
describe Gitlab::GithubImport::Representation::ToHash do
describe '#to_hash' do
let(:user) { double(:user, attributes: { login: 'alice' }) }
let(:issue) do
double(
:issue,
attributes: { user: user, assignees: [user], number: 42 }
)
end
let(:issue_hash) { issue.to_hash }
before do
user.extend(described_class)
issue.extend(described_class)
end
it 'converts an object to a Hash' do
expect(issue_hash).to be_an_instance_of(Hash)
end
it 'converts nested objects to Hashes' do
expect(issue_hash[:user]).to eq({ login: 'alice' })
end
it 'converts Array values to Hashes' do
expect(issue_hash[:assignees]).to eq([{ login: 'alice' }])
end
it 'keeps values as-is if they do not respond to #to_hash' do
expect(issue_hash[:number]).to eq(42)
end
end
end
require 'spec_helper'
describe Gitlab::GithubImport::Representation::User do
shared_examples 'a User' do
it 'returns an instance of User' do
expect(user).to be_an_instance_of(described_class)
end
context 'the returned User' do
it 'includes the user ID' do
expect(user.id).to eq(42)
end
it 'includes the username' do
expect(user.login).to eq('alice')
end
end
end
describe '.from_api_response' do
it_behaves_like 'a User' do
let(:response) { double(:response, id: 42, login: 'alice') }
let(:user) { described_class.from_api_response(response) }
end
end
describe '.from_json_hash' do
it_behaves_like 'a User' do
let(:hash) { { 'id' => 42, 'login' => 'alice' } }
let(:user) { described_class.from_json_hash(hash) }
end
end
end
require 'spec_helper'
describe Gitlab::GithubImport::Representation do
describe '.symbolize_hash' do
it 'returns a Hash with the keys as Symbols' do
hash = described_class.symbolize_hash('number' => 10)
expect(hash).to eq({ number: 10 })
end
it 'parses timestamp fields into Time instances' do
hash = described_class.symbolize_hash('created_at' => '2017-01-01 12:00')
expect(hash[:created_at]).to be_an_instance_of(Time)
end
end
end
require 'spec_helper'
describe Gitlab::GithubImport::SequentialImporter do
describe '#execute' do
it 'imports a project in sequence' do
repository = double(:repository)
project = double(:project, id: 1, repository: repository)
importer = described_class.new(project, token: 'foo')
expect_any_instance_of(Gitlab::GithubImport::Importer::RepositoryImporter)
.to receive(:execute)
described_class::SEQUENTIAL_IMPORTERS.each do |klass|
instance = double(:instance)
expect(klass).to receive(:new)
.with(project, importer.client)
.and_return(instance)
expect(instance).to receive(:execute)
end
described_class::PARALLEL_IMPORTERS.each do |klass|
instance = double(:instance)
expect(klass).to receive(:new)
.with(project, importer.client, parallel: false)
.and_return(instance)
expect(instance).to receive(:execute)
end
expect(repository).to receive(:after_import)
expect(importer.execute).to eq(true)
end
end
end
require 'spec_helper'
describe Gitlab::GithubImport::UserFinder, :clean_gitlab_redis_cache do
let(:project) { create(:project) }
let(:client) { double(:client) }
let(:finder) { described_class.new(project, client) }
describe '#author_id_for' do
it 'returns the user ID for the author of an object' do
user = double(:user, id: 4, login: 'kittens')
note = double(:note, author: user)
expect(finder).to receive(:user_id_for).with(user).and_return(42)
expect(finder.author_id_for(note)).to eq([42, true])
end
it 'returns the ID of the project creator if no user ID could be found' do
user = double(:user, id: 4, login: 'kittens')
note = double(:note, author: user)
expect(finder).to receive(:user_id_for).with(user).and_return(nil)
expect(finder.author_id_for(note)).to eq([project.creator_id, false])
end
it 'returns the ID of the ghost user when the object has no user' do
note = double(:note, author: nil)
expect(finder.author_id_for(note)).to eq([User.ghost.id, true])
end
it 'returns the ID of the ghost user when the given object is nil' do
expect(finder.author_id_for(nil)).to eq([User.ghost.id, true])
end
end
describe '#assignee_id_for' do
it 'returns the user ID for the assignee of an issuable' do
user = double(:user, id: 4, login: 'kittens')
issue = double(:issue, assignee: user)
expect(finder).to receive(:user_id_for).with(user).and_return(42)
expect(finder.assignee_id_for(issue)).to eq(42)
end
it 'returns nil if the issuable does not have an assignee' do
issue = double(:issue, assignee: nil)
expect(finder).not_to receive(:user_id_for)
expect(finder.assignee_id_for(issue)).to be_nil
end
end
describe '#user_id_for' do
it 'returns the user ID for the given user' do
user = double(:user, id: 4, login: 'kittens')
expect(finder).to receive(:find).with(user.id, user.login).and_return(42)
expect(finder.user_id_for(user)).to eq(42)
end
end
describe '#find' do
let(:user) { create(:user) }
before do
allow(finder).to receive(:email_for_github_username)
.and_return(user.email)
end
context 'without a cache' do
before do
allow(finder).to receive(:find_from_cache).and_return([false, nil])
expect(finder).to receive(:find_id_from_database).and_call_original
end
it 'finds a GitLab user for a GitHub user ID' do
user.identities.create!(provider: :github, extern_uid: 42)
expect(finder.find(42, user.username)).to eq(user.id)
end
it 'finds a GitLab user for a GitHub Email address' do
expect(finder.find(42, user.username)).to eq(user.id)
end
end
context 'with a cache' do
it 'returns the cached user ID' do
expect(finder).to receive(:find_from_cache).and_return([true, user.id])
expect(finder).not_to receive(:find_id_from_database)
expect(finder.find(42, user.username)).to eq(user.id)
end
it 'does not query the database if the cache key exists but is empty' do
expect(finder).to receive(:find_from_cache).and_return([true, nil])
expect(finder).not_to receive(:find_id_from_database)
expect(finder.find(42, user.username)).to be_nil
end
end
end
describe '#find_from_cache' do
it 'retrieves a GitLab user ID for a GitHub user ID' do
expect(finder)
.to receive(:cached_id_for_github_id)
.with(42)
.and_return([true, 4])
expect(finder.find_from_cache(42)).to eq([true, 4])
end
it 'retrieves a GitLab user ID for a GitHub Email address' do
email = 'kittens@example.com'
expect(finder)
.to receive(:cached_id_for_github_id)
.with(42)
.and_return([false, nil])
expect(finder)
.to receive(:cached_id_for_github_email)
.with(email)
.and_return([true, 4])
expect(finder.find_from_cache(42, email)).to eq([true, 4])
end
it 'does not query the cache for an Email address when none is given' do
expect(finder)
.to receive(:cached_id_for_github_id)
.with(42)
.and_return([false, nil])
expect(finder).not_to receive(:cached_id_for_github_id)
expect(finder.find_from_cache(42)).to eq([false])
end
end
describe '#find_id_from_database' do
let(:user) { create(:user) }
it 'returns the GitLab user ID for a GitHub user ID' do
user.identities.create!(provider: :github, extern_uid: 42)
expect(finder.find_id_from_database(42, user.email)).to eq(user.id)
end
it 'returns the GitLab user ID for a GitHub Email address' do
expect(finder.find_id_from_database(42, user.email)).to eq(user.id)
end
end
describe '#email_for_github_username' do
let(:email) { 'kittens@example.com' }
context 'when an Email address is cached' do
it 'reads the Email address from the cache' do
expect(Gitlab::GithubImport::Caching)
.to receive(:read)
.and_return(email)
expect(client).not_to receive(:user)
expect(finder.email_for_github_username('kittens')).to eq(email)
end
end
context 'when an Email address is not cached' do
let(:user) { double(:user, email: email) }
it 'retrieves the Email address from the GitHub API' do
expect(client).to receive(:user).with('kittens').and_return(user)
expect(finder.email_for_github_username('kittens')).to eq(email)
end
it 'caches the Email address when an Email address is available' do
expect(client).to receive(:user).with('kittens').and_return(user)
expect(Gitlab::GithubImport::Caching)
.to receive(:write)
.with(an_instance_of(String), email)
finder.email_for_github_username('kittens')
end
it 'returns nil if the user does not exist' do
expect(client)
.to receive(:user)
.with('kittens')
.and_return(nil)
expect(Gitlab::GithubImport::Caching)
.not_to receive(:write)
expect(finder.email_for_github_username('kittens')).to be_nil
end
end
end
describe '#cached_id_for_github_id' do
let(:id) { 4 }
it 'reads a user ID from the cache' do
Gitlab::GithubImport::Caching
.write(described_class::ID_CACHE_KEY % id, 4)
expect(finder.cached_id_for_github_id(id)).to eq([true, 4])
end
it 'reads a non existing cache key' do
expect(finder.cached_id_for_github_id(id)).to eq([false, nil])
end
end
describe '#cached_id_for_github_email' do
let(:email) { 'kittens@example.com' }
it 'reads a user ID from the cache' do
Gitlab::GithubImport::Caching
.write(described_class::ID_FOR_EMAIL_CACHE_KEY % email, 4)
expect(finder.cached_id_for_github_email(email)).to eq([true, 4])
end
it 'reads a non existing cache key' do
expect(finder.cached_id_for_github_email(email)).to eq([false, nil])
end
end
describe '#id_for_github_id' do
let(:id) { 4 }
it 'queries and caches the user ID for a given GitHub ID' do
expect(finder).to receive(:query_id_for_github_id)
.with(id)
.and_return(42)
expect(Gitlab::GithubImport::Caching)
.to receive(:write)
.with(described_class::ID_CACHE_KEY % id, 42)
finder.id_for_github_id(id)
end
it 'caches a nil value if no ID could be found' do
expect(finder).to receive(:query_id_for_github_id)
.with(id)
.and_return(nil)
expect(Gitlab::GithubImport::Caching)
.to receive(:write)
.with(described_class::ID_CACHE_KEY % id, nil)
finder.id_for_github_id(id)
end
end
describe '#id_for_github_email' do
let(:email) { 'kittens@example.com' }
it 'queries and caches the user ID for a given Email address' do
expect(finder).to receive(:query_id_for_github_email)
.with(email)
.and_return(42)
expect(Gitlab::GithubImport::Caching)
.to receive(:write)
.with(described_class::ID_FOR_EMAIL_CACHE_KEY % email, 42)
finder.id_for_github_email(email)
end
it 'caches a nil value if no ID could be found' do
expect(finder).to receive(:query_id_for_github_email)
.with(email)
.and_return(nil)
expect(Gitlab::GithubImport::Caching)
.to receive(:write)
.with(described_class::ID_FOR_EMAIL_CACHE_KEY % email, nil)
finder.id_for_github_email(email)
end
end
describe '#query_id_for_github_id' do
it 'returns the ID of the user for the given GitHub user ID' do
user = create(:user)
user.identities.create!(provider: :github, extern_uid: '42')
expect(finder.query_id_for_github_id(42)).to eq(user.id)
end
it 'returns nil when no user ID could be found' do
expect(finder.query_id_for_github_id(42)).to be_nil
end
end
describe '#query_id_for_github_email' do
it 'returns the ID of the user for the given Email address' do
user = create(:user, email: 'kittens@example.com')
expect(finder.query_id_for_github_email(user.email)).to eq(user.id)
end
it 'returns nil if no user ID could be found' do
expect(finder.query_id_for_github_email('kittens@example.com')).to be_nil
end
end
describe '#read_id_from_cache' do
it 'reads an ID from the cache' do
Gitlab::GithubImport::Caching.write('foo', 10)
expect(finder.read_id_from_cache('foo')).to eq([true, 10])
end
it 'reads a cache key with an empty value' do
Gitlab::GithubImport::Caching.write('foo', nil)
expect(finder.read_id_from_cache('foo')).to eq([true, nil])
end
it 'reads a cache key that does not exist' do
expect(finder.read_id_from_cache('foo')).to eq([false, nil])
end
end
end
require 'spec_helper'
describe Gitlab::GithubImport do
let(:project) { double(:project) }
describe '.new_client_for' do
it 'returns a new Client with a custom token' do
expect(described_class::Client)
.to receive(:new)
.with('123', parallel: true)
described_class.new_client_for(project, token: '123')
end
it 'returns a new Client with a token stored in the import data' do
import_data = double(:import_data, credentials: { user: '123' })
expect(project)
.to receive(:import_data)
.and_return(import_data)
expect(described_class::Client)
.to receive(:new)
.with('123', parallel: true)
described_class.new_client_for(project)
end
end
describe '.insert_and_return_id' do
let(:attributes) { { iid: 1, title: 'foo' } }
let(:project) { create(:project) }
context 'on PostgreSQL' do
it 'returns the ID returned by the query' do
expect(Gitlab::Database)
.to receive(:bulk_insert)
.with(Issue.table_name, [attributes], return_ids: true)
.and_return([10])
id = described_class.insert_and_return_id(attributes, project.issues)
expect(id).to eq(10)
end
end
context 'on MySQL' do
it 'uses a separate query to retrieve the ID' do
issue = create(:issue, project: project, iid: attributes[:iid])
expect(Gitlab::Database)
.to receive(:bulk_insert)
.with(Issue.table_name, [attributes], return_ids: true)
.and_return([])
id = described_class.insert_and_return_id(attributes, project.issues)
expect(id).to eq(issue.id)
end
end
end
describe '.ghost_user_id', :clean_gitlab_redis_cache do
it 'returns the ID of the ghost user' do
expect(described_class.ghost_user_id).to eq(User.ghost.id)
end
it 'caches the ghost user ID' do
expect(Gitlab::GithubImport::Caching)
.to receive(:write)
.once
.and_call_original
2.times do
described_class.ghost_user_id
end
end
end
end
......@@ -56,14 +56,14 @@ describe Gitlab::ImportSources do
describe '.importer' do
import_sources = {
'github' => Github::Import,
'github' => Gitlab::GithubImport::ParallelImporter,
'bitbucket' => Gitlab::BitbucketImport::Importer,
'gitlab' => Gitlab::GitlabImport::Importer,
'google_code' => Gitlab::GoogleCodeImport::Importer,
'fogbugz' => Gitlab::FogbugzImport::Importer,
'git' => nil,
'gitlab_project' => Gitlab::ImportExport::Importer,
'gitea' => Gitlab::GithubImport::Importer
'gitea' => Gitlab::LegacyGithubImport::Importer
}
import_sources.each do |name, klass|
......
require 'spec_helper'
describe Gitlab::GithubImport::BranchFormatter do
describe Gitlab::LegacyGithubImport::BranchFormatter do
let(:project) { create(:project, :repository) }
let(:commit) { create(:commit, project: project) }
let(:repo) { double }
......
require 'spec_helper'
describe Gitlab::LegacyGithubImport::Client do
let(:token) { '123456' }
let(:github_provider) { Settingslogic.new('app_id' => 'asd123', 'app_secret' => 'asd123', 'name' => 'github', 'args' => { 'client_options' => {} }) }
subject(:client) { described_class.new(token) }
before do
allow(Gitlab.config.omniauth).to receive(:providers).and_return([github_provider])
end
it 'convert OAuth2 client options to symbols' do
client.client.options.keys.each do |key|
expect(key).to be_kind_of(Symbol)
end
end
it 'does not crash (e.g. Settingslogic::MissingSetting) when verify_ssl config is not present' do
expect { client.api }.not_to raise_error
end
context 'when config is missing' do
before do
allow(Gitlab.config.omniauth).to receive(:providers).and_return([])
end
it 'is still possible to get an Octokit client' do
expect { client.api }.not_to raise_error
end
it 'is not be possible to get an OAuth2 client' do
expect { client.client }.to raise_error(Projects::ImportService::Error)
end
end
context 'allow SSL verification to be configurable on API' do
before do
github_provider['verify_ssl'] = false
end
it 'uses supplied value' do
expect(client.client.options[:connection_opts][:ssl]).to eq({ verify: false })
expect(client.api.connection_options[:ssl]).to eq({ verify: false })
end
end
describe '#api_endpoint' do
context 'when provider does not specity an API endpoint' do
it 'uses GitHub root API endpoint' do
expect(client.api.api_endpoint).to eq 'https://api.github.com/'
end
end
context 'when provider specify a custom API endpoint' do
before do
github_provider['args']['client_options']['site'] = 'https://github.company.com/'
end
it 'uses the custom API endpoint' do
expect(OmniAuth::Strategies::GitHub).not_to receive(:default_options)
expect(client.api.api_endpoint).to eq 'https://github.company.com/'
end
end
context 'when given a host' do
subject(:client) { described_class.new(token, host: 'https://try.gitea.io/') }
it 'builds a endpoint with the given host and the default API version' do
expect(client.api.api_endpoint).to eq 'https://try.gitea.io/api/v3/'
end
end
context 'when given an API version' do
subject(:client) { described_class.new(token, api_version: 'v3') }
it 'does not use the API version without a host' do
expect(client.api.api_endpoint).to eq 'https://api.github.com/'
end
end
context 'when given a host and version' do
subject(:client) { described_class.new(token, host: 'https://try.gitea.io/', api_version: 'v3') }
it 'builds a endpoint with the given options' do
expect(client.api.api_endpoint).to eq 'https://try.gitea.io/api/v3/'
end
end
end
it 'does not raise error when rate limit is disabled' do
stub_request(:get, /api.github.com/)
allow(client.api).to receive(:rate_limit!).and_raise(Octokit::NotFound)
expect { client.issues {} }.not_to raise_error
end
end
require 'spec_helper'
describe Gitlab::GithubImport::CommentFormatter do
describe Gitlab::LegacyGithubImport::CommentFormatter do
let(:client) { double }
let(:project) { create(:project) }
let(:octocat) { double(id: 123456, login: 'octocat', email: 'octocat@example.com') }
......
require 'spec_helper'
describe Gitlab::GithubImport::Importer do
shared_examples 'Gitlab::GithubImport::Importer#execute' do
describe Gitlab::LegacyGithubImport::Importer do
shared_examples 'Gitlab::LegacyGithubImport::Importer#execute' do
let(:expected_not_called) { [] }
before do
......@@ -35,7 +35,7 @@ describe Gitlab::GithubImport::Importer do
end
end
shared_examples 'Gitlab::GithubImport::Importer#execute an error occurs' do
shared_examples 'Gitlab::LegacyGithubImport::Importer#execute an error occurs' do
before do
allow(project).to receive(:import_data).and_return(double.as_null_object)
......@@ -178,7 +178,7 @@ describe Gitlab::GithubImport::Importer do
end
end
shared_examples 'Gitlab::GithubImport unit-testing' do
shared_examples 'Gitlab::LegacyGithubImport unit-testing' do
describe '#clean_up_restored_branches' do
subject { described_class.new(project) }
......@@ -188,7 +188,7 @@ describe Gitlab::GithubImport::Importer do
end
context 'when pull request stills open' do
let(:gh_pull_request) { Gitlab::GithubImport::PullRequestFormatter.new(project, pull_request) }
let(:gh_pull_request) { Gitlab::LegacyGithubImport::PullRequestFormatter.new(project, pull_request) }
it 'does not remove branches' do
expect(subject).not_to receive(:remove_branch)
......@@ -197,7 +197,7 @@ describe Gitlab::GithubImport::Importer do
end
context 'when pull request is closed' do
let(:gh_pull_request) { Gitlab::GithubImport::PullRequestFormatter.new(project, closed_pull_request) }
let(:gh_pull_request) { Gitlab::LegacyGithubImport::PullRequestFormatter.new(project, closed_pull_request) }
it 'does remove branches' do
expect(subject).to receive(:remove_branch).at_least(2).times
......@@ -262,14 +262,14 @@ describe Gitlab::GithubImport::Importer do
let(:repo_root) { 'https://github.com' }
subject { described_class.new(project) }
it_behaves_like 'Gitlab::GithubImport::Importer#execute'
it_behaves_like 'Gitlab::GithubImport::Importer#execute an error occurs'
it_behaves_like 'Gitlab::GithubImport unit-testing'
it_behaves_like 'Gitlab::LegacyGithubImport::Importer#execute'
it_behaves_like 'Gitlab::LegacyGithubImport::Importer#execute an error occurs'
it_behaves_like 'Gitlab::LegacyGithubImport unit-testing'
describe '#client' do
it 'instantiates a Client' do
allow(project).to receive(:import_data).and_return(double(credentials: credentials))
expect(Gitlab::GithubImport::Client).to receive(:new).with(
expect(Gitlab::LegacyGithubImport::Client).to receive(:new).with(
credentials[:user],
{}
)
......@@ -288,16 +288,16 @@ describe Gitlab::GithubImport::Importer do
project.update(import_type: 'gitea', import_url: "#{repo_root}/foo/group/project.git")
end
it_behaves_like 'Gitlab::GithubImport::Importer#execute' do
it_behaves_like 'Gitlab::LegacyGithubImport::Importer#execute' do
let(:expected_not_called) { [:import_releases] }
end
it_behaves_like 'Gitlab::GithubImport::Importer#execute an error occurs'
it_behaves_like 'Gitlab::GithubImport unit-testing'
it_behaves_like 'Gitlab::LegacyGithubImport::Importer#execute an error occurs'
it_behaves_like 'Gitlab::LegacyGithubImport unit-testing'
describe '#client' do
it 'instantiates a Client' do
allow(project).to receive(:import_data).and_return(double(credentials: credentials))
expect(Gitlab::GithubImport::Client).to receive(:new).with(
expect(Gitlab::LegacyGithubImport::Client).to receive(:new).with(
credentials[:user],
{ host: "#{repo_root}:443/foo", api_version: 'v1' }
)
......
require 'spec_helper'
describe Gitlab::GithubImport::IssuableFormatter do
describe Gitlab::LegacyGithubImport::IssuableFormatter do
let(:raw_data) do
double(number: 42)
end
......
require 'spec_helper'
describe Gitlab::GithubImport::IssueFormatter do
describe Gitlab::LegacyGithubImport::IssueFormatter do
let(:client) { double }
let!(:project) { create(:project, namespace: create(:namespace, path: 'octocat')) }
let(:octocat) { double(id: 123456, login: 'octocat', email: 'octocat@example.com') }
......@@ -30,7 +30,7 @@ describe Gitlab::GithubImport::IssueFormatter do
allow(client).to receive(:user).and_return(octocat)
end
shared_examples 'Gitlab::GithubImport::IssueFormatter#attributes' do
shared_examples 'Gitlab::LegacyGithubImport::IssueFormatter#attributes' do
context 'when issue is open' do
let(:raw_data) { double(base_data.merge(state: 'open')) }
......@@ -135,7 +135,7 @@ describe Gitlab::GithubImport::IssueFormatter do
end
end
shared_examples 'Gitlab::GithubImport::IssueFormatter#number' do
shared_examples 'Gitlab::LegacyGithubImport::IssueFormatter#number' do
let(:raw_data) { double(base_data.merge(number: 1347)) }
it 'returns issue number' do
......@@ -144,8 +144,8 @@ describe Gitlab::GithubImport::IssueFormatter do
end
context 'when importing a GitHub project' do
it_behaves_like 'Gitlab::GithubImport::IssueFormatter#attributes'
it_behaves_like 'Gitlab::GithubImport::IssueFormatter#number'
it_behaves_like 'Gitlab::LegacyGithubImport::IssueFormatter#attributes'
it_behaves_like 'Gitlab::LegacyGithubImport::IssueFormatter#number'
end
context 'when importing a Gitea project' do
......@@ -153,8 +153,8 @@ describe Gitlab::GithubImport::IssueFormatter do
project.update(import_type: 'gitea')
end
it_behaves_like 'Gitlab::GithubImport::IssueFormatter#attributes'
it_behaves_like 'Gitlab::GithubImport::IssueFormatter#number'
it_behaves_like 'Gitlab::LegacyGithubImport::IssueFormatter#attributes'
it_behaves_like 'Gitlab::LegacyGithubImport::IssueFormatter#number'
end
describe '#has_comments?' do
......
require 'spec_helper'
describe Gitlab::GithubImport::LabelFormatter do
describe Gitlab::LegacyGithubImport::LabelFormatter do
let(:project) { create(:project) }
let(:raw) { double(name: 'improvements', color: 'e6e6e6') }
......
require 'spec_helper'
describe Gitlab::GithubImport::MilestoneFormatter do
describe Gitlab::LegacyGithubImport::MilestoneFormatter do
let(:project) { create(:project) }
let(:created_at) { DateTime.strptime('2011-01-26T19:01:12Z') }
let(:updated_at) { DateTime.strptime('2011-01-27T19:01:12Z') }
......@@ -19,7 +19,7 @@ describe Gitlab::GithubImport::MilestoneFormatter do
subject(:formatter) { described_class.new(project, raw_data) }
shared_examples 'Gitlab::GithubImport::MilestoneFormatter#attributes' do
shared_examples 'Gitlab::LegacyGithubImport::MilestoneFormatter#attributes' do
let(:data) { base_data.merge(iid_attr => 1347) }
context 'when milestone is open' do
......@@ -82,7 +82,7 @@ describe Gitlab::GithubImport::MilestoneFormatter do
end
context 'when importing a GitHub project' do
it_behaves_like 'Gitlab::GithubImport::MilestoneFormatter#attributes'
it_behaves_like 'Gitlab::LegacyGithubImport::MilestoneFormatter#attributes'
end
context 'when importing a Gitea project' do
......@@ -91,6 +91,6 @@ describe Gitlab::GithubImport::MilestoneFormatter do
project.update(import_type: 'gitea')
end
it_behaves_like 'Gitlab::GithubImport::MilestoneFormatter#attributes'
it_behaves_like 'Gitlab::LegacyGithubImport::MilestoneFormatter#attributes'
end
end
require 'spec_helper'
describe Gitlab::GithubImport::ProjectCreator do
describe Gitlab::LegacyGithubImport::ProjectCreator do
let(:user) { create(:user) }
let(:namespace) { create(:group, owner: user) }
......
require 'spec_helper'
describe Gitlab::GithubImport::PullRequestFormatter do
describe Gitlab::LegacyGithubImport::PullRequestFormatter do
let(:client) { double }
let(:project) { create(:project, :repository) }
let(:source_sha) { create(:commit, project: project).id }
......@@ -44,7 +44,7 @@ describe Gitlab::GithubImport::PullRequestFormatter do
allow(client).to receive(:user).and_return(octocat)
end
shared_examples 'Gitlab::GithubImport::PullRequestFormatter#attributes' do
shared_examples 'Gitlab::LegacyGithubImport::PullRequestFormatter#attributes' do
context 'when pull request is open' do
let(:raw_data) { double(base_data.merge(state: 'open')) }
......@@ -189,7 +189,7 @@ describe Gitlab::GithubImport::PullRequestFormatter do
end
end
shared_examples 'Gitlab::GithubImport::PullRequestFormatter#number' do
shared_examples 'Gitlab::LegacyGithubImport::PullRequestFormatter#number' do
let(:raw_data) { double(base_data) }
it 'returns pull request number' do
......@@ -197,7 +197,7 @@ describe Gitlab::GithubImport::PullRequestFormatter do
end
end
shared_examples 'Gitlab::GithubImport::PullRequestFormatter#source_branch_name' do
shared_examples 'Gitlab::LegacyGithubImport::PullRequestFormatter#source_branch_name' do
context 'when source branch exists' do
let(:raw_data) { double(base_data) }
......@@ -231,7 +231,7 @@ describe Gitlab::GithubImport::PullRequestFormatter do
end
end
shared_examples 'Gitlab::GithubImport::PullRequestFormatter#target_branch_name' do
shared_examples 'Gitlab::LegacyGithubImport::PullRequestFormatter#target_branch_name' do
context 'when target branch exists' do
let(:raw_data) { double(base_data) }
......@@ -250,10 +250,10 @@ describe Gitlab::GithubImport::PullRequestFormatter do
end
context 'when importing a GitHub project' do
it_behaves_like 'Gitlab::GithubImport::PullRequestFormatter#attributes'
it_behaves_like 'Gitlab::GithubImport::PullRequestFormatter#number'
it_behaves_like 'Gitlab::GithubImport::PullRequestFormatter#source_branch_name'
it_behaves_like 'Gitlab::GithubImport::PullRequestFormatter#target_branch_name'
it_behaves_like 'Gitlab::LegacyGithubImport::PullRequestFormatter#attributes'
it_behaves_like 'Gitlab::LegacyGithubImport::PullRequestFormatter#number'
it_behaves_like 'Gitlab::LegacyGithubImport::PullRequestFormatter#source_branch_name'
it_behaves_like 'Gitlab::LegacyGithubImport::PullRequestFormatter#target_branch_name'
end
context 'when importing a Gitea project' do
......@@ -261,10 +261,10 @@ describe Gitlab::GithubImport::PullRequestFormatter do
project.update(import_type: 'gitea')
end
it_behaves_like 'Gitlab::GithubImport::PullRequestFormatter#attributes'
it_behaves_like 'Gitlab::GithubImport::PullRequestFormatter#number'
it_behaves_like 'Gitlab::GithubImport::PullRequestFormatter#source_branch_name'
it_behaves_like 'Gitlab::GithubImport::PullRequestFormatter#target_branch_name'
it_behaves_like 'Gitlab::LegacyGithubImport::PullRequestFormatter#attributes'
it_behaves_like 'Gitlab::LegacyGithubImport::PullRequestFormatter#number'
it_behaves_like 'Gitlab::LegacyGithubImport::PullRequestFormatter#source_branch_name'
it_behaves_like 'Gitlab::LegacyGithubImport::PullRequestFormatter#target_branch_name'
end
describe '#valid?' do
......
require 'spec_helper'
describe Gitlab::GithubImport::ReleaseFormatter do
describe Gitlab::LegacyGithubImport::ReleaseFormatter do
let!(:project) { create(:project, namespace: create(:namespace, path: 'octocat')) }
let(:octocat) { double(id: 123456, login: 'octocat') }
let(:created_at) { DateTime.strptime('2011-01-26T19:01:12Z') }
......
require 'spec_helper'
describe Gitlab::GithubImport::UserFormatter do
describe Gitlab::LegacyGithubImport::UserFormatter do
let(:client) { double }
let(:octocat) { double(id: 123456, login: 'octocat', email: 'octocat@example.com') }
......
require 'spec_helper'
describe Gitlab::GithubImport::WikiFormatter do
describe Gitlab::LegacyGithubImport::WikiFormatter do
let(:project) do
create(:project,
namespace: create(:namespace, path: 'gitlabhq'),
......
......@@ -3016,4 +3016,77 @@ describe Project do
end
end
end
describe '#after_import' do
let(:project) { build(:project) }
it 'runs the correct hooks' do
expect(project.repository).to receive(:after_import)
expect(project).to receive(:import_finish)
expect(project).to receive(:update_project_counter_caches)
expect(project).to receive(:remove_import_jid)
project.after_import
end
end
describe '#update_project_counter_caches' do
let(:project) { create(:project) }
it 'updates all project counter caches' do
expect_any_instance_of(Projects::OpenIssuesCountService)
.to receive(:refresh_cache)
.and_call_original
expect_any_instance_of(Projects::OpenMergeRequestsCountService)
.to receive(:refresh_cache)
.and_call_original
project.update_project_counter_caches
end
end
describe '#remove_import_jid', :clean_gitlab_redis_cache do
let(:project) { }
context 'without an import JID' do
it 'does nothing' do
project = create(:project)
expect(Gitlab::SidekiqStatus)
.not_to receive(:unset)
project.remove_import_jid
end
end
context 'with an import JID' do
it 'unsets the import JID' do
project = create(:project, import_jid: '123')
expect(Gitlab::SidekiqStatus)
.to receive(:unset)
.with('123')
.and_call_original
project.remove_import_jid
expect(project.import_jid).to be_nil
end
end
end
describe '#wiki_repository_exists?' do
it 'returns true when the wiki repository exists' do
project = create(:project, :wiki_repo)
expect(project.wiki_repository_exists?).to eq(true)
end
it 'returns false when the wiki repository does not exist' do
project = create(:project)
expect(project.wiki_repository_exists?).to eq(false)
end
end
end
......@@ -841,6 +841,19 @@ describe User do
end
end
describe '.by_any_email' do
it 'returns an ActiveRecord::Relation' do
expect(described_class.by_any_email('foo@example.com'))
.to be_a_kind_of(ActiveRecord::Relation)
end
it 'returns a relation of users' do
user = create(:user)
expect(described_class.by_any_email(user.email)).to eq([user])
end
end
describe '.search' do
let!(:user) { create(:user, name: 'user', username: 'usern', email: 'email@gmail.com') }
let!(:user2) { create(:user, name: 'user name', username: 'username', email: 'someemail@gmail.com') }
......
......@@ -6,6 +6,41 @@ describe Projects::ImportService do
subject { described_class.new(project, user) }
describe '#async?' do
it 'returns true for an asynchronous importer' do
importer_class = double(:importer, async?: true)
allow(subject).to receive(:has_importer?).and_return(true)
allow(subject).to receive(:importer_class).and_return(importer_class)
expect(subject).to be_async
end
it 'returns false for a regular importer' do
importer_class = double(:importer, async?: false)
allow(subject).to receive(:has_importer?).and_return(true)
allow(subject).to receive(:importer_class).and_return(importer_class)
expect(subject).not_to be_async
end
it 'returns false when the importer does not define #async?' do
importer_class = double(:importer)
allow(subject).to receive(:has_importer?).and_return(true)
allow(subject).to receive(:importer_class).and_return(importer_class)
expect(subject).not_to be_async
end
it 'returns false when the importer does not exist' do
allow(subject).to receive(:has_importer?).and_return(false)
expect(subject).not_to be_async
end
end
describe '#execute' do
context 'with unknown url' do
before do
......@@ -37,21 +72,24 @@ describe Projects::ImportService do
end
context 'with a Github repository' do
it 'succeeds if repository import is successfully' do
expect_any_instance_of(Github::Import).to receive(:execute).and_return(true)
it 'succeeds if repository import was scheduled' do
expect_any_instance_of(Gitlab::GithubImport::ParallelImporter)
.to receive(:execute)
.and_return(true)
result = subject.execute
expect(result[:status]).to eq :success
end
it 'fails if repository import fails' do
expect_any_instance_of(Repository).to receive(:fetch_remote).and_raise(Gitlab::Shell::Error.new('Failed to import the repository'))
it 'fails if repository import was not scheduled' do
expect_any_instance_of(Gitlab::GithubImport::ParallelImporter)
.to receive(:execute)
.and_return(false)
result = subject.execute
expect(result[:status]).to eq :error
expect(result[:message]).to eq "Error importing repository #{project.import_url} into #{project.path_with_namespace} - The remote data could not be imported."
end
end
......@@ -92,47 +130,22 @@ describe Projects::ImportService do
end
it 'succeeds if importer succeeds' do
allow_any_instance_of(Github::Import).to receive(:execute).and_return(true)
allow_any_instance_of(Gitlab::GithubImport::ParallelImporter)
.to receive(:execute).and_return(true)
result = subject.execute
expect(result[:status]).to eq :success
end
it 'flushes various caches' do
allow_any_instance_of(Github::Import).to receive(:execute)
.and_return(true)
expect_any_instance_of(Repository).to receive(:expire_content_cache)
subject.execute
end
it 'fails if importer fails' do
allow_any_instance_of(Github::Import).to receive(:execute).and_return(false)
result = subject.execute
expect(result[:status]).to eq :error
expect(result[:message]).to eq "Error importing repository #{project.import_url} into #{project.full_path} - The remote data could not be imported."
end
it 'fails if importer raise an error' do
allow_any_instance_of(Github::Import).to receive(:execute).and_raise(Projects::ImportService::Error.new('Github: failed to connect API'))
allow_any_instance_of(Gitlab::GithubImport::ParallelImporter)
.to receive(:execute)
.and_return(false)
result = subject.execute
expect(result[:status]).to eq :error
expect(result[:message]).to eq "Error importing repository #{project.import_url} into #{project.full_path} - Github: failed to connect API"
end
it 'expires content cache after error' do
allow_any_instance_of(Project).to receive(:repository_exists?).and_return(false)
expect_any_instance_of(Repository).to receive(:fetch_remote).and_raise(Gitlab::Shell::Error.new)
expect_any_instance_of(Repository).to receive(:expire_content_cache)
subject.execute
end
end
......
......@@ -14,7 +14,7 @@ shared_examples 'a GitHub-ish import controller: POST personal_access_token' do
it "updates access token" do
token = 'asdfasdf9876'
allow_any_instance_of(Gitlab::GithubImport::Client)
allow_any_instance_of(Gitlab::LegacyGithubImport::Client)
.to receive(:user).and_return(true)
post :personal_access_token, personal_access_token: token
......@@ -79,7 +79,7 @@ shared_examples 'a GitHub-ish import controller: GET status' do
end
it "handles an invalid access token" do
allow_any_instance_of(Gitlab::GithubImport::Client)
allow_any_instance_of(Gitlab::LegacyGithubImport::Client)
.to receive(:repos).and_raise(Octokit::Unauthorized)
get :status
......@@ -110,7 +110,7 @@ shared_examples 'a GitHub-ish import controller: POST create' do
context "when the repository owner is the provider user" do
context "when the provider user and GitLab user's usernames match" do
it "takes the current user's namespace" do
expect(Gitlab::GithubImport::ProjectCreator)
expect(Gitlab::LegacyGithubImport::ProjectCreator)
.to receive(:new).with(provider_repo, provider_repo.name, user.namespace, user, access_params, type: provider)
.and_return(double(execute: true))
......@@ -122,7 +122,7 @@ shared_examples 'a GitHub-ish import controller: POST create' do
let(:provider_username) { "someone_else" }
it "takes the current user's namespace" do
expect(Gitlab::GithubImport::ProjectCreator)
expect(Gitlab::LegacyGithubImport::ProjectCreator)
.to receive(:new).with(provider_repo, provider_repo.name, user.namespace, user, access_params, type: provider)
.and_return(double(execute: true))
......@@ -149,7 +149,7 @@ shared_examples 'a GitHub-ish import controller: POST create' do
end
it "takes the existing namespace" do
expect(Gitlab::GithubImport::ProjectCreator)
expect(Gitlab::LegacyGithubImport::ProjectCreator)
.to receive(:new).with(provider_repo, provider_repo.name, existing_namespace, user, access_params, type: provider)
.and_return(double(execute: true))
......@@ -161,7 +161,7 @@ shared_examples 'a GitHub-ish import controller: POST create' do
it "creates a project using user's namespace" do
create(:user, username: other_username)
expect(Gitlab::GithubImport::ProjectCreator)
expect(Gitlab::LegacyGithubImport::ProjectCreator)
.to receive(:new).with(provider_repo, provider_repo.name, user.namespace, user, access_params, type: provider)
.and_return(double(execute: true))
......@@ -173,14 +173,14 @@ shared_examples 'a GitHub-ish import controller: POST create' do
context "when a namespace with the provider user's username doesn't exist" do
context "when current user can create namespaces" do
it "creates the namespace" do
expect(Gitlab::GithubImport::ProjectCreator)
expect(Gitlab::LegacyGithubImport::ProjectCreator)
.to receive(:new).and_return(double(execute: true))
expect { post :create, target_namespace: provider_repo.name, format: :js }.to change(Namespace, :count).by(1)
end
it "takes the new namespace" do
expect(Gitlab::GithubImport::ProjectCreator)
expect(Gitlab::LegacyGithubImport::ProjectCreator)
.to receive(:new).with(provider_repo, provider_repo.name, an_instance_of(Group), user, access_params, type: provider)
.and_return(double(execute: true))
......@@ -194,14 +194,14 @@ shared_examples 'a GitHub-ish import controller: POST create' do
end
it "doesn't create the namespace" do
expect(Gitlab::GithubImport::ProjectCreator)
expect(Gitlab::LegacyGithubImport::ProjectCreator)
.to receive(:new).and_return(double(execute: true))
expect { post :create, format: :js }.not_to change(Namespace, :count)
end
it "takes the current user's namespace" do
expect(Gitlab::GithubImport::ProjectCreator)
expect(Gitlab::LegacyGithubImport::ProjectCreator)
.to receive(:new).with(provider_repo, provider_repo.name, user.namespace, user, access_params, type: provider)
.and_return(double(execute: true))
......@@ -219,7 +219,7 @@ shared_examples 'a GitHub-ish import controller: POST create' do
end
it 'takes the selected namespace and name' do
expect(Gitlab::GithubImport::ProjectCreator)
expect(Gitlab::LegacyGithubImport::ProjectCreator)
.to receive(:new).with(provider_repo, test_name, test_namespace, user, access_params, type: provider)
.and_return(double(execute: true))
......@@ -227,7 +227,7 @@ shared_examples 'a GitHub-ish import controller: POST create' do
end
it 'takes the selected name and default namespace' do
expect(Gitlab::GithubImport::ProjectCreator)
expect(Gitlab::LegacyGithubImport::ProjectCreator)
.to receive(:new).with(provider_repo, test_name, user.namespace, user, access_params, type: provider)
.and_return(double(execute: true))
......@@ -245,7 +245,7 @@ shared_examples 'a GitHub-ish import controller: POST create' do
end
it 'takes the selected namespace and name' do
expect(Gitlab::GithubImport::ProjectCreator)
expect(Gitlab::LegacyGithubImport::ProjectCreator)
.to receive(:new).with(provider_repo, test_name, nested_namespace, user, access_params, type: provider)
.and_return(double(execute: true))
......@@ -257,7 +257,7 @@ shared_examples 'a GitHub-ish import controller: POST create' do
let(:test_name) { 'test_name' }
it 'takes the selected namespace and name' do
expect(Gitlab::GithubImport::ProjectCreator)
expect(Gitlab::LegacyGithubImport::ProjectCreator)
.to receive(:new).with(provider_repo, test_name, kind_of(Namespace), user, access_params, type: provider)
.and_return(double(execute: true))
......@@ -265,7 +265,7 @@ shared_examples 'a GitHub-ish import controller: POST create' do
end
it 'creates the namespaces' do
allow(Gitlab::GithubImport::ProjectCreator)
allow(Gitlab::LegacyGithubImport::ProjectCreator)
.to receive(:new).with(provider_repo, test_name, kind_of(Namespace), user, access_params, type: provider)
.and_return(double(execute: true))
......@@ -274,7 +274,7 @@ shared_examples 'a GitHub-ish import controller: POST create' do
end
it 'new namespace has the right parent' do
allow(Gitlab::GithubImport::ProjectCreator)
allow(Gitlab::LegacyGithubImport::ProjectCreator)
.to receive(:new).with(provider_repo, test_name, kind_of(Namespace), user, access_params, type: provider)
.and_return(double(execute: true))
......@@ -289,7 +289,7 @@ shared_examples 'a GitHub-ish import controller: POST create' do
let!(:parent_namespace) { create(:group, name: 'foo', owner: user) }
it 'takes the selected namespace and name' do
expect(Gitlab::GithubImport::ProjectCreator)
expect(Gitlab::LegacyGithubImport::ProjectCreator)
.to receive(:new).with(provider_repo, test_name, kind_of(Namespace), user, access_params, type: provider)
.and_return(double(execute: true))
......@@ -297,7 +297,7 @@ shared_examples 'a GitHub-ish import controller: POST create' do
end
it 'creates the namespaces' do
allow(Gitlab::GithubImport::ProjectCreator)
allow(Gitlab::LegacyGithubImport::ProjectCreator)
.to receive(:new).with(provider_repo, test_name, kind_of(Namespace), user, access_params, type: provider)
.and_return(double(execute: true))
......
require 'spec_helper'
describe Gitlab::GithubImport::NotifyUponDeath do
let(:worker_class) do
Class.new do
include Sidekiq::Worker
include Gitlab::GithubImport::NotifyUponDeath
end
end
describe '.sidekiq_retries_exhausted' do
it 'notifies the JobWaiter when 3 arguments are given and the last is a String' do
job = { 'args' => [12, {}, '123abc'], 'jid' => '123' }
expect(Gitlab::JobWaiter)
.to receive(:notify)
.with('123abc', '123')
worker_class.sidekiq_retries_exhausted_block.call(job)
end
it 'does not notify the JobWaiter when only 2 arguments are given' do
job = { 'args' => [12, {}], 'jid' => '123' }
expect(Gitlab::JobWaiter)
.not_to receive(:notify)
worker_class.sidekiq_retries_exhausted_block.call(job)
end
it 'does not notify the JobWaiter when only 1 argument is given' do
job = { 'args' => [12], 'jid' => '123' }
expect(Gitlab::JobWaiter)
.not_to receive(:notify)
worker_class.sidekiq_retries_exhausted_block.call(job)
end
it 'does not notify the JobWaiter when the last argument is not a String' do
job = { 'args' => [12, {}, 40], 'jid' => '123' }
expect(Gitlab::JobWaiter)
.not_to receive(:notify)
worker_class.sidekiq_retries_exhausted_block.call(job)
end
end
end
require 'spec_helper'
describe Gitlab::GithubImport::ObjectImporter do
let(:worker) do
Class.new do
include(Gitlab::GithubImport::ObjectImporter)
def counter_name
:dummy_counter
end
def counter_description
'This is a counter'
end
end.new
end
describe '#import' do
it 'imports the object' do
representation_class = double(:representation_class)
importer_class = double(:importer_class)
importer_instance = double(:importer_instance)
representation = double(:representation)
project = double(:project, path_with_namespace: 'foo/bar')
client = double(:client)
expect(worker)
.to receive(:representation_class)
.and_return(representation_class)
expect(worker)
.to receive(:importer_class)
.and_return(importer_class)
expect(representation_class)
.to receive(:from_json_hash)
.with(an_instance_of(Hash))
.and_return(representation)
expect(importer_class)
.to receive(:new)
.with(representation, project, client)
.and_return(importer_instance)
expect(importer_instance)
.to receive(:execute)
expect(worker.counter)
.to receive(:increment)
.with(project: 'foo/bar')
.and_call_original
worker.import(project, client, { 'number' => 10 })
end
end
describe '#counter' do
it 'returns a Prometheus counter' do
expect(worker)
.to receive(:counter_name)
.and_call_original
expect(worker)
.to receive(:counter_description)
.and_call_original
worker.counter
end
end
end
require 'spec_helper'
describe Gitlab::GithubImport::Queue do
it 'sets the Sidekiq options for the worker' do
worker = Class.new do
include Sidekiq::Worker
include Gitlab::GithubImport::Queue
end
expect(worker.sidekiq_options['queue']).to eq('github_importer')
end
end
require 'spec_helper'
describe Gitlab::GithubImport::ReschedulingMethods do
let(:worker) do
Class.new { include(Gitlab::GithubImport::ReschedulingMethods) }.new
end
describe '#perform' do
context 'with a non-existing project' do
it 'does not perform any work' do
expect(worker)
.not_to receive(:try_import)
worker.perform(-1, {})
end
it 'notifies any waiters so they do not wait forever' do
expect(worker)
.to receive(:notify_waiter)
.with('123')
worker.perform(-1, {}, '123')
end
end
context 'with an existing project' do
let(:project) { create(:project) }
it 'notifies any waiters upon successfully importing the data' do
expect(worker)
.to receive(:try_import)
.with(
an_instance_of(Project),
an_instance_of(Gitlab::GithubImport::Client),
{ 'number' => 2 }
)
.and_return(true)
expect(worker)
.to receive(:notify_waiter).with('123')
worker.perform(project.id, { 'number' => 2 }, '123')
end
it 'reschedules itself if the data could not be imported' do
expect(worker)
.to receive(:try_import)
.with(
an_instance_of(Project),
an_instance_of(Gitlab::GithubImport::Client),
{ 'number' => 2 }
)
.and_return(false)
expect(worker)
.not_to receive(:notify_waiter)
expect_any_instance_of(Gitlab::GithubImport::Client)
.to receive(:rate_limit_resets_in)
.and_return(14)
expect(worker.class)
.to receive(:perform_in)
.with(14, project.id, { 'number' => 2 }, '123')
worker.perform(project.id, { 'number' => 2 }, '123')
end
end
end
describe '#try_import' do
it 'returns true when the import succeeds' do
expect(worker)
.to receive(:import)
.with(10, 20)
expect(worker.try_import(10, 20)).to eq(true)
end
it 'returns false when the import fails due to hitting the GitHub API rate limit' do
expect(worker)
.to receive(:import)
.with(10, 20)
.and_raise(Gitlab::GithubImport::RateLimitError)
expect(worker.try_import(10, 20)).to eq(false)
end
end
describe '#notify_waiter' do
it 'notifies the waiter if a waiter key is specified' do
expect(worker)
.to receive(:jid)
.and_return('abc123')
expect(Gitlab::JobWaiter)
.to receive(:notify)
.with('123', 'abc123')
worker.notify_waiter('123')
end
it 'does not notify any waiters if no waiter key is specified' do
expect(Gitlab::JobWaiter)
.not_to receive(:notify)
worker.notify_waiter(nil)
end
end
end
require 'spec_helper'
describe Gitlab::GithubImport::StageMethods do
let(:project) { create(:project) }
let(:worker) do
Class.new { include(Gitlab::GithubImport::StageMethods) }.new
end
describe '#perform' do
it 'returns if no project could be found' do
expect(worker).not_to receive(:try_import)
worker.perform(-1)
end
it 'imports the data when the project exists' do
allow(worker)
.to receive(:find_project)
.with(project.id)
.and_return(project)
expect(worker)
.to receive(:try_import)
.with(
an_instance_of(Gitlab::GithubImport::Client),
an_instance_of(Project)
)
worker.perform(project.id)
end
end
describe '#try_import' do
it 'imports the project' do
client = double(:client)
expect(worker)
.to receive(:import)
.with(client, project)
worker.try_import(client, project)
end
it 'reschedules the worker if RateLimitError was raised' do
client = double(:client, rate_limit_resets_in: 10)
expect(worker)
.to receive(:import)
.with(client, project)
.and_raise(Gitlab::GithubImport::RateLimitError)
expect(worker.class)
.to receive(:perform_in)
.with(10, project.id)
worker.try_import(client, project)
end
end
describe '#find_project' do
it 'returns a Project for an existing ID' do
project.update_column(:import_status, 'started')
expect(worker.find_project(project.id)).to eq(project)
end
it 'returns nil for a project that failed importing' do
project.update_column(:import_status, 'failed')
expect(worker.find_project(project.id)).to be_nil
end
it 'returns nil for a non-existing project ID' do
expect(worker.find_project(-1)).to be_nil
end
end
end
require 'spec_helper'
describe Gitlab::GithubImport::AdvanceStageWorker, :clean_gitlab_redis_shared_state do
let(:project) { create(:project, import_jid: '123') }
let(:worker) { described_class.new }
describe '#perform' do
context 'when the project no longer exists' do
it 'does not perform any work' do
expect(worker).not_to receive(:wait_for_jobs)
worker.perform(-1, { '123' => 2 }, :finish)
end
end
context 'when there are remaining jobs' do
before do
allow(worker)
.to receive(:find_project)
.and_return(project)
end
it 'reschedules itself' do
expect(worker)
.to receive(:wait_for_jobs)
.with({ '123' => 2 })
.and_return({ '123' => 1 })
expect(described_class)
.to receive(:perform_in)
.with(described_class::INTERVAL, project.id, { '123' => 1 }, :finish)
worker.perform(project.id, { '123' => 2 }, :finish)
end
end
context 'when there are no remaining jobs' do
before do
allow(worker)
.to receive(:find_project)
.and_return(project)
allow(worker)
.to receive(:wait_for_jobs)
.with({ '123' => 2 })
.and_return({})
end
it 'schedules the next stage' do
expect(project)
.to receive(:refresh_import_jid_expiration)
expect(Gitlab::GithubImport::Stage::FinishImportWorker)
.to receive(:perform_async)
.with(project.id)
worker.perform(project.id, { '123' => 2 }, :finish)
end
it 'raises KeyError when the stage name is invalid' do
expect { worker.perform(project.id, { '123' => 2 }, :kittens) }
.to raise_error(KeyError)
end
end
end
describe '#wait_for_jobs' do
it 'waits for jobs to complete and returns a new pair of keys to wait for' do
waiter1 = double(:waiter1, jobs_remaining: 1, key: '123')
waiter2 = double(:waiter2, jobs_remaining: 0, key: '456')
expect(Gitlab::JobWaiter)
.to receive(:new)
.ordered
.with(2, '123')
.and_return(waiter1)
expect(Gitlab::JobWaiter)
.to receive(:new)
.ordered
.with(1, '456')
.and_return(waiter2)
expect(waiter1)
.to receive(:wait)
.with(described_class::BLOCKING_WAIT_TIME)
expect(waiter2)
.to receive(:wait)
.with(described_class::BLOCKING_WAIT_TIME)
new_waiters = worker.wait_for_jobs({ '123' => 2, '456' => 1 })
expect(new_waiters).to eq({ '123' => 1 })
end
end
describe '#find_project' do
it 'returns a Project' do
project.update_column(:import_status, 'started')
found = worker.find_project(project.id)
expect(found).to be_an_instance_of(Project)
# This test is there to make sure we only select the columns we care
# about.
expect(found.attributes).to eq({ 'id' => nil, 'import_jid' => '123' })
end
it 'returns nil if the project import is not running' do
expect(worker.find_project(project.id)).to be_nil
end
end
end
require 'spec_helper'
describe Gitlab::GithubImport::ImportDiffNoteWorker do
let(:worker) { described_class.new }
describe '#import' do
it 'imports a diff note' do
project = double(:project, path_with_namespace: 'foo/bar')
client = double(:client)
importer = double(:importer)
hash = {
'noteable_id' => 42,
'path' => 'README.md',
'commit_id' => '123abc',
'diff_hunk' => "@@ -1 +1 @@\n-Hello\n+Hello world",
'user' => { 'id' => 4, 'login' => 'alice' },
'note' => 'Hello world',
'created_at' => Time.zone.now.to_s,
'updated_at' => Time.zone.now.to_s
}
expect(Gitlab::GithubImport::Importer::DiffNoteImporter)
.to receive(:new)
.with(
an_instance_of(Gitlab::GithubImport::Representation::DiffNote),
project,
client
)
.and_return(importer)
expect(importer)
.to receive(:execute)
expect(worker.counter)
.to receive(:increment)
.with(project: 'foo/bar')
.and_call_original
worker.import(project, client, hash)
end
end
end
require 'spec_helper'
describe Gitlab::GithubImport::ImportIssueWorker do
let(:worker) { described_class.new }
describe '#import' do
it 'imports an issue' do
project = double(:project, path_with_namespace: 'foo/bar')
client = double(:client)
importer = double(:importer)
hash = {
'iid' => 42,
'title' => 'My Issue',
'description' => 'This is my issue',
'milestone_number' => 4,
'state' => 'opened',
'assignees' => [{ 'id' => 4, 'login' => 'alice' }],
'label_names' => %w[bug],
'user' => { 'id' => 4, 'login' => 'alice' },
'created_at' => Time.zone.now.to_s,
'updated_at' => Time.zone.now.to_s,
'pull_request' => false
}
expect(Gitlab::GithubImport::Importer::IssueAndLabelLinksImporter)
.to receive(:new)
.with(
an_instance_of(Gitlab::GithubImport::Representation::Issue),
project,
client
)
.and_return(importer)
expect(importer)
.to receive(:execute)
expect(worker.counter)
.to receive(:increment)
.with(project: 'foo/bar')
.and_call_original
worker.import(project, client, hash)
end
end
end
require 'spec_helper'
describe Gitlab::GithubImport::ImportNoteWorker do
let(:worker) { described_class.new }
describe '#import' do
it 'imports a note' do
project = double(:project, path_with_namespace: 'foo/bar')
client = double(:client)
importer = double(:importer)
hash = {
'noteable_id' => 42,
'noteable_type' => 'issues',
'user' => { 'id' => 4, 'login' => 'alice' },
'note' => 'Hello world',
'created_at' => Time.zone.now.to_s,
'updated_at' => Time.zone.now.to_s
}
expect(Gitlab::GithubImport::Importer::NoteImporter)
.to receive(:new)
.with(
an_instance_of(Gitlab::GithubImport::Representation::Note),
project,
client
)
.and_return(importer)
expect(importer)
.to receive(:execute)
expect(worker.counter)
.to receive(:increment)
.with(project: 'foo/bar')
.and_call_original
worker.import(project, client, hash)
end
end
end
require 'spec_helper'
describe Gitlab::GithubImport::ImportPullRequestWorker do
let(:worker) { described_class.new }
describe '#import' do
it 'imports a pull request' do
project = double(:project, path_with_namespace: 'foo/bar')
client = double(:client)
importer = double(:importer)
hash = {
'iid' => 42,
'title' => 'My Pull Request',
'description' => 'This is my pull request',
'source_branch' => 'my-feature',
'source_branch_sha' => '123abc',
'target_branch' => 'master',
'target_branch_sha' => '456def',
'source_repository_id' => 400,
'target_repository_id' => 200,
'source_repository_owner' => 'alice',
'state' => 'closed',
'milestone_number' => 4,
'user' => { 'id' => 4, 'login' => 'alice' },
'assignee' => { 'id' => 4, 'login' => 'alice' },
'created_at' => Time.zone.now.to_s,
'updated_at' => Time.zone.now.to_s,
'merged_at' => Time.zone.now.to_s
}
expect(Gitlab::GithubImport::Importer::PullRequestImporter)
.to receive(:new)
.with(
an_instance_of(Gitlab::GithubImport::Representation::PullRequest),
project,
client
)
.and_return(importer)
expect(importer)
.to receive(:execute)
expect(worker.counter)
.to receive(:increment)
.with(project: 'foo/bar')
.and_call_original
worker.import(project, client, hash)
end
end
end
require 'spec_helper'
describe Gitlab::GithubImport::RefreshImportJidWorker do
let(:worker) { described_class.new }
describe '.perform_in_the_future' do
it 'schedules a job in the future' do
expect(described_class)
.to receive(:perform_in)
.with(1.minute.to_i, 10, '123')
described_class.perform_in_the_future(10, '123')
end
end
describe '#perform' do
let(:project) { create(:project, import_jid: '123abc') }
context 'when the project does not exist' do
it 'does nothing' do
expect(Gitlab::SidekiqStatus)
.not_to receive(:running?)
worker.perform(-1, '123')
end
end
context 'when the job is running' do
it 'refreshes the import JID and reschedules itself' do
allow(worker)
.to receive(:find_project)
.with(project.id)
.and_return(project)
expect(Gitlab::SidekiqStatus)
.to receive(:running?)
.with('123')
.and_return(true)
expect(project)
.to receive(:refresh_import_jid_expiration)
expect(worker.class)
.to receive(:perform_in_the_future)
.with(project.id, '123')
worker.perform(project.id, '123')
end
end
context 'when the job is no longer running' do
it 'returns' do
allow(worker)
.to receive(:find_project)
.with(project.id)
.and_return(project)
expect(Gitlab::SidekiqStatus)
.to receive(:running?)
.with('123')
.and_return(false)
expect(project)
.not_to receive(:refresh_import_jid_expiration)
worker.perform(project.id, '123')
end
end
end
describe '#find_project' do
it 'returns a Project' do
project = create(:project, import_status: 'started')
expect(worker.find_project(project.id)).to be_an_instance_of(Project)
end
it 'only selects the import JID field' do
project = create(:project, import_status: 'started', import_jid: '123abc')
expect(worker.find_project(project.id).attributes)
.to eq({ 'id' => nil, 'import_jid' => '123abc' })
end
it 'returns nil for a project for which the import process failed' do
project = create(:project, import_status: 'failed')
expect(worker.find_project(project.id)).to be_nil
end
it 'returns nil for a non-existing project' do
expect(worker.find_project(-1)).to be_nil
end
end
end
require 'spec_helper'
describe Gitlab::GithubImport::Stage::FinishImportWorker do
let(:project) { create(:project) }
let(:worker) { described_class.new }
describe '#perform' do
it 'marks the import as finished' do
expect(project).to receive(:after_import)
expect(worker).to receive(:report_import_time).with(project)
worker.import(double(:client), project)
end
end
describe '#report_import_time' do
it 'reports the total import time' do
expect(worker.histogram)
.to receive(:observe)
.with({ project: project.path_with_namespace }, a_kind_of(Numeric))
.and_call_original
expect(worker.counter)
.to receive(:increment)
.and_call_original
expect(worker.logger).to receive(:info).with(an_instance_of(String))
worker.report_import_time(project)
end
end
end
require 'spec_helper'
describe Gitlab::GithubImport::Stage::ImportBaseDataWorker do
let(:project) { create(:project) }
let(:worker) { described_class.new }
describe '#import' do
it 'imports the base data of a project' do
importer = double(:importer)
client = double(:client)
described_class::IMPORTERS.each do |klass|
expect(klass)
.to receive(:new)
.with(project, client)
.and_return(importer)
expect(importer).to receive(:execute)
end
expect(project).to receive(:refresh_import_jid_expiration)
expect(Gitlab::GithubImport::Stage::ImportPullRequestsWorker)
.to receive(:perform_async)
.with(project.id)
worker.import(client, project)
end
end
end
require 'spec_helper'
describe Gitlab::GithubImport::Stage::ImportIssuesAndDiffNotesWorker do
let(:project) { create(:project) }
let(:worker) { described_class.new }
describe '#import' do
it 'imports the issues and diff notes' do
client = double(:client)
described_class::IMPORTERS.each do |klass|
importer = double(:importer)
waiter = Gitlab::JobWaiter.new(2, '123')
expect(klass)
.to receive(:new)
.with(project, client)
.and_return(importer)
expect(importer)
.to receive(:execute)
.and_return(waiter)
end
expect(Gitlab::GithubImport::AdvanceStageWorker)
.to receive(:perform_async)
.with(project.id, { '123' => 2 }, :notes)
worker.import(client, project)
end
end
end
require 'spec_helper'
describe Gitlab::GithubImport::Stage::ImportNotesWorker do
let(:project) { create(:project) }
let(:worker) { described_class.new }
describe '#import' do
it 'imports all the notes' do
importer = double(:importer)
client = double(:client)
waiter = Gitlab::JobWaiter.new(2, '123')
expect(Gitlab::GithubImport::Importer::NotesImporter)
.to receive(:new)
.with(project, client)
.and_return(importer)
expect(importer)
.to receive(:execute)
.and_return(waiter)
expect(Gitlab::GithubImport::AdvanceStageWorker)
.to receive(:perform_async)
.with(project.id, { '123' => 2 }, :finish)
worker.import(client, project)
end
end
end
require 'spec_helper'
describe Gitlab::GithubImport::Stage::ImportPullRequestsWorker do
let(:project) { create(:project) }
let(:worker) { described_class.new }
describe '#import' do
it 'imports all the pull requests' do
importer = double(:importer)
client = double(:client)
waiter = Gitlab::JobWaiter.new(2, '123')
expect(Gitlab::GithubImport::Importer::PullRequestsImporter)
.to receive(:new)
.with(project, client)
.and_return(importer)
expect(importer)
.to receive(:execute)
.and_return(waiter)
expect(project)
.to receive(:refresh_import_jid_expiration)
expect(Gitlab::GithubImport::AdvanceStageWorker)
.to receive(:perform_async)
.with(project.id, { '123' => 2 }, :issues_and_diff_notes)
worker.import(client, project)
end
end
end
require 'spec_helper'
describe Gitlab::GithubImport::Stage::ImportRepositoryWorker do
let(:project) { double(:project, id: 4) }
let(:worker) { described_class.new }
describe '#import' do
before do
expect(Gitlab::GithubImport::RefreshImportJidWorker)
.to receive(:perform_in_the_future)
.with(project.id, '123')
expect(worker)
.to receive(:jid)
.and_return('123')
end
context 'when the import succeeds' do
it 'schedules the importing of the base data' do
client = double(:client)
expect_any_instance_of(Gitlab::GithubImport::Importer::RepositoryImporter)
.to receive(:execute)
.and_return(true)
expect(Gitlab::GithubImport::Stage::ImportBaseDataWorker)
.to receive(:perform_async)
.with(project.id)
worker.import(client, project)
end
end
context 'when the import fails' do
it 'does not schedule the importing of the base data' do
client = double(:client)
expect_any_instance_of(Gitlab::GithubImport::Importer::RepositoryImporter)
.to receive(:execute)
.and_return(false)
expect(Gitlab::GithubImport::Stage::ImportBaseDataWorker)
.not_to receive(:perform_async)
worker.import(client, project)
end
end
end
end
......@@ -59,5 +59,28 @@ describe RepositoryImportWorker do
expect(project.reload.import_status).to eq('failed')
end
end
context 'when using an asynchronous importer' do
it 'does not mark the import process as finished' do
service = double(:service)
allow(Projects::ImportService)
.to receive(:new)
.and_return(service)
allow(service)
.to receive(:execute)
.and_return(true)
allow(service)
.to receive(:async?)
.and_return(true)
expect_any_instance_of(Project)
.not_to receive(:import_finish)
subject.perform(project.id)
end
end
end
end
Markdown is supported
0%
or
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment