gitlab: handle protected branches

Depends-On: https://review.opendev.org/#/c/756725/
Change-Id: Ib31ee55b28145aa9b22f11e976d76bf3ad3df04a
This commit is contained in:
Pierre-Louis Bonicoli 2020-10-08 16:50:51 +02:00
parent c94622ec45
commit 8be15d9aad
No known key found for this signature in database
GPG Key ID: 06914C4A5EDAA6DD
14 changed files with 519 additions and 65 deletions

View File

@ -199,9 +199,10 @@ configuration. Some examples of tenant definitions are:
.. attr:: exclude-unprotected-branches
Define if unprotected github branches should be
processed. Defaults to the tenant wide setting of
exclude-unprotected-branches.
Define if unprotected branches should be processed.
Defaults to the tenant wide setting of
exclude-unprotected-branches. This currently only affects
GitHub and GitLab projects.
.. attr:: extra-config-paths
@ -264,7 +265,7 @@ configuration. Some examples of tenant definitions are:
.. attr:: exclude-unprotected-branches
:default: false
When using a branch and pull model on a shared GitHub repository
When using a branch and pull model on a shared repository
there are usually one or more protected branches which are gated
and a dynamic number of personal/feature branches which are the
source for the pull requests. These branches can potentially
@ -272,7 +273,7 @@ configuration. Some examples of tenant definitions are:
wide configuration. In order to deal with this Zuul's operations
can be limited to the protected branches which are gated. This
is a tenant wide setting and can be overridden per project.
This currently only affects GitHub projects.
This currently only affects GitHub and GitLab projects.
.. attr:: default-parent
:default: base

View File

@ -31,6 +31,7 @@ import os
import queue
import random
import re
from collections import defaultdict, namedtuple
from logging import Logger
from queue import Queue
from typing import Callable, Optional, Any, Iterable, Generator, List, Dict
@ -63,6 +64,7 @@ import testtools
import testtools.content
import testtools.content_type
from git.exc import NoSuchPathError
from git.util import IterableList
import yaml
import paramiko
@ -184,6 +186,28 @@ def never_capture():
return decorator
def registerProjects(source_name, client, config):
path = config.get('scheduler', 'tenant_config')
with open(os.path.join(FIXTURE_DIR, path)) as f:
tenant_config = yaml.safe_load(f.read())
for tenant in tenant_config:
sources = tenant['tenant']['source']
conf = sources.get(source_name)
if not conf:
return
projects = conf.get('config-projects', [])
projects.extend(conf.get('untrusted-projects', []))
for project in projects:
if isinstance(project, dict):
# This can be a dict with the project as the only key
client.addProjectByName(
list(project.keys())[0])
else:
client.addProjectByName(project)
class GerritDriverMock(GerritDriver):
def __init__(self, registry, changes: Dict[str, Dict[str, Change]],
upstream_root: str, additional_event_queues, poller_events,
@ -229,28 +253,6 @@ class GithubDriverMock(GithubDriver):
self.rpcclient = rpcclient
self.git_url_with_auth = git_url_with_auth
def registerGithubProjects(self, connection):
path = self.config.get('scheduler', 'tenant_config')
with open(os.path.join(FIXTURE_DIR, path)) as f:
tenant_config = yaml.safe_load(f.read())
for tenant in tenant_config:
sources = tenant['tenant']['source']
conf = sources.get(connection.source.name)
if not conf:
return
projects = conf.get('config-projects', [])
projects.extend(conf.get('untrusted-projects', []))
client = connection.getGithubClient(None)
for project in projects:
if isinstance(project, dict):
# This can be a dict with the project as the only key
client.addProjectByName(
list(project.keys())[0])
else:
client.addProjectByName(project)
def getConnection(self, name, config):
server = config.get('server', 'github.com')
db = self.changes.setdefault(server, {})
@ -261,7 +263,8 @@ class GithubDriverMock(GithubDriver):
git_url_with_auth=self.git_url_with_auth)
self.additional_event_queues.append(connection.event_queue)
setattr(self.registry, 'fake_' + name, connection)
self.registerGithubProjects(connection)
client = connection.getGithubClient(None)
registerProjects(connection.source.name, client, self.config)
return connection
@ -290,11 +293,12 @@ class PagureDriverMock(PagureDriver):
class GitlabDriverMock(GitlabDriver):
def __init__(self, registry, changes: Dict[str, Dict[str, Change]],
upstream_root: str, additional_event_queues,
rpcclient: RPCClient):
config: ConfigParser, upstream_root: str,
additional_event_queues, rpcclient: RPCClient):
super(GitlabDriverMock, self).__init__()
self.registry = registry
self.changes = changes
self.config = config
self.upstream_root = upstream_root
self.additional_event_queues = additional_event_queues
self.rpcclient = rpcclient
@ -308,6 +312,8 @@ class GitlabDriverMock(GitlabDriver):
upstream_root=self.upstream_root)
self.additional_event_queues.append(connection.event_queue)
setattr(self.registry, 'fake_' + name, connection)
registerProjects(connection.source.name, connection.gl_client,
self.config)
return connection
@ -337,7 +343,8 @@ class TestConnectionRegistry(ConnectionRegistry):
self.registerDriver(PagureDriverMock(
self, changes, upstream_root, additional_event_queues, rpcclient))
self.registerDriver(GitlabDriverMock(
self, changes, upstream_root, additional_event_queues, rpcclient))
self, changes, config, upstream_root, additional_event_queues,
rpcclient))
self.registerDriver(ElasticsearchDriver())
@ -1782,6 +1789,20 @@ class FakeGitlabConnection(gitlabconnection.GitlabConnection):
self.upstream_root = upstream_root
self.mr_number = 0
def addProject(self, project):
super(FakeGitlabConnection, self).addProject(project)
self.gl_client.addProject(project)
def protectBranch(self, owner, project, branch, protected=True):
if branch in self.gl_client.fake_repos[(owner, project)]:
del self.gl_client.fake_repos[(owner, project)][branch]
fake_branch = FakeBranch(branch, protected=protected)
self.gl_client.fake_repos[(owner, project)].append(fake_branch)
def deleteBranch(self, owner, project, branch):
if branch in self.gl_client.fake_repos[(owner, project)]:
del self.gl_client.fake_repos[(owner, project)][branch]
def getGitUrl(self, project):
return 'file://' + os.path.join(self.upstream_root, project.name)
@ -1852,12 +1873,16 @@ class FakeGitlabConnection(gitlabconnection.GitlabConnection):
self.gl_client.community_edition = False
FakeBranch = namedtuple('Branch', ('name', 'protected'))
class FakeGitlabAPIClient(gitlabconnection.GitlabAPIClient):
log = logging.getLogger("zuul.test.FakeGitlabAPIClient")
def __init__(self, baseurl, api_token, merge_requests_db={}):
super(FakeGitlabAPIClient, self).__init__(baseurl, api_token)
self.merge_requests = merge_requests_db
self.fake_repos = defaultdict(lambda: IterableList('name'))
self.community_edition = False
def gen_error(self, verb):
@ -1901,9 +1926,15 @@ class FakeGitlabAPIClient(gitlabconnection.GitlabAPIClient):
'merge_status': mr.merge_status,
}, 200, "", "GET"
match = re.match('.+/projects/(.+)/repository/branches$', url)
match = re.match('.+/projects/(.+)/'
'(repository/branches|protected_branches)$', url)
if match:
return [{'name': 'master'}], 200, "", "GET"
protected = url.endswith('protected_branches')
project = urllib.parse.unquote(match.group(1)).split('/')
branches = [{'name': branch.name, 'protected': branch.protected}
for branch in self.fake_repos[tuple(project)]
if ((not protected) or branch.protected)]
return branches, 200, "", "GET"
match = re.match(
r'.+/projects/(.+)/merge_requests/(\d+)/approvals$', url)
@ -1918,6 +1949,18 @@ class FakeGitlabAPIClient(gitlabconnection.GitlabAPIClient):
'approved': mr.approved,
}, 200, "", "GET"
match = re.match(r'.+/projects/(.+)/repository/branches/(.+)$', url)
if match:
project, branch = match.groups()
project = urllib.parse.unquote(project)
branch = urllib.parse.unquote(branch)
owner, name = project.split('/')
if branch in self.fake_repos[(owner, name)]:
protected = self.fake_repos[(owner, name)][branch].protected
return {'protected': protected}, 200, "", "GET"
else:
return {}, 404, "", "GET"
def post(self, url, params=None, zuul_event_id=None):
self.log.info(
@ -1958,6 +2001,16 @@ class FakeGitlabAPIClient(gitlabconnection.GitlabAPIClient):
return {}, 200, "", "PUT"
def addProject(self, project):
self.addProjectByName(project.name)
def addProjectByName(self, project_name):
owner, proj = project_name.split('/')
repo = self.fake_repos[(owner, proj)]
branch = FakeBranch('master', False)
if 'master' not in repo:
repo.append(branch)
class GitlabChangeReference(git.Reference):
_common_path_default = "refs/merge-requests"

View File

@ -0,0 +1,22 @@
- pipeline:
name: check
manager: independent
trigger:
gitlab:
- event: gl_merge_request
action:
- opened
- changed
success:
gitlab:
comment: true
failure:
gitlab:
comment: true
start:
gitlab:
comment: true
- job:
name: base
parent: null

View File

@ -0,0 +1 @@
test

View File

@ -0,0 +1,2 @@
- hosts: all
tasks: []

View File

@ -0,0 +1,9 @@
- job:
name: project-test
run: playbooks/project-test.yaml
- project:
name: org/project1
check:
jobs:
- project-test

View File

@ -0,0 +1 @@
test

View File

@ -0,0 +1,2 @@
- hosts: all
tasks: []

View File

@ -0,0 +1,8 @@
- job:
name: used-job
run: playbooks/used-job.yaml
- project:
check:
jobs:
- used-job

View File

@ -0,0 +1 @@
test

View File

@ -0,0 +1,12 @@
- tenant:
name: tenant-one
source:
gitlab:
config-projects:
- org/common-config
untrusted-projects:
- org/project1
- org/project2:
exclude-unprotected-branches: true
- org/project3:
exclude-unprotected-branches: true

View File

@ -1341,7 +1341,7 @@ class TestGithubUnprotectedBranches(ZuulTestCase):
# Enable branch protection on org/project3@stable. We'll use a PR on
# this branch as a depends-on to validate that the stable branch
# which is not protected in org/project3 is not filtered out.
# which is not protected in org/project2 is not filtered out.
repo = github.repo_from_project('org/project3')
self.create_branch('org/project3', 'stable')
repo._set_branch_protection('stable', True)

View File

@ -20,8 +20,8 @@ import socket
import zuul.rpcclient
from tests.base import ZuulTestCase, simple_layout
from tests.base import ZuulWebFixture
from tests.base import random_sha1, simple_layout
from tests.base import ZuulTestCase, ZuulWebFixture
from testtools.matchers import MatchesRegex
@ -511,7 +511,7 @@ class TestGitlabDriver(ZuulTestCase):
self.waitUntilSettled()
# The changes for the job from project2 should include the project1
# PR content
# MR content
changes = self.getJobFromHistory(
'project2-test', 'org/project2').changes
@ -687,7 +687,7 @@ class TestGitlabDriver(ZuulTestCase):
self.waitUntilSettled()
# The changes for the job from project4 should include the project3
# PR content
# MR content
changes = self.getJobFromHistory(
'project4-test', 'org/project4').changes
@ -698,3 +698,325 @@ class TestGitlabDriver(ZuulTestCase):
self.assertTrue(A.is_merged)
self.assertTrue(B.is_merged)
class TestGitlabUnprotectedBranches(ZuulTestCase):
config_file = 'zuul-gitlab-driver.conf'
tenant_config_file = 'config/unprotected-branches-gitlab/main.yaml'
def test_unprotected_branches(self):
tenant = self.scheds.first.sched.abide.tenants\
.get('tenant-one')
project1 = tenant.untrusted_projects[0]
project2 = tenant.untrusted_projects[1]
tpc1 = tenant.project_configs[project1.canonical_name]
tpc2 = tenant.project_configs[project2.canonical_name]
# project1 should have parsed master
self.assertIn('master', tpc1.parsed_branch_config.keys())
# project2 should have no parsed branch
self.assertEqual(0, len(tpc2.parsed_branch_config.keys()))
# now enable branch protection and trigger reload
self.fake_gitlab.protectBranch('org', 'project2', 'master')
self.scheds.execute(lambda app: app.sched.reconfigure(app.config))
self.waitUntilSettled()
tenant = self.scheds.first.sched.abide.tenants.get('tenant-one')
tpc1 = tenant.project_configs[project1.canonical_name]
tpc2 = tenant.project_configs[project2.canonical_name]
# project1 and project2 should have parsed master now
self.assertIn('master', tpc1.parsed_branch_config.keys())
self.assertIn('master', tpc2.parsed_branch_config.keys())
def test_filtered_branches_in_build(self):
"""
Tests unprotected branches are filtered in builds if excluded
"""
self.executor_server.keep_jobdir = True
# Enable branch protection on org/project2@master
self.create_branch('org/project2', 'feat-x')
self.fake_gitlab.protectBranch('org', 'project2', 'master',
protected=True)
# Enable branch protection on org/project3@stable. We'll use a MR on
# this branch as a depends-on to validate that the stable branch
# which is not protected in org/project2 is not filtered out.
self.create_branch('org/project3', 'stable')
self.fake_gitlab.protectBranch('org', 'project3', 'stable',
protected=True)
self.scheds.execute(lambda app: app.sched.reconfigure(app.config))
self.waitUntilSettled()
A = self.fake_gitlab.openFakeMergeRequest('org/project3', 'stable',
'A')
msg = "Depends-On: %s" % A.url
B = self.fake_gitlab.openFakeMergeRequest('org/project2', 'master',
'B', description=msg)
self.fake_gitlab.emitEvent(B.getMergeRequestOpenedEvent())
self.waitUntilSettled()
build = self.history[0]
path = os.path.join(
build.jobdir.src_root, 'gitlab', 'org/project2')
build_repo = git.Repo(path)
branches = [x.name for x in build_repo.branches]
self.assertNotIn('feat-x', branches)
self.assertHistory([
dict(name='used-job', result='SUCCESS',
changes="%s,%s %s,%s" % (A.number, A.sha,
B.number, B.sha)),
])
def test_unfiltered_branches_in_build(self):
"""
Tests unprotected branches are not filtered in builds if not excluded
"""
self.executor_server.keep_jobdir = True
# Enable branch protection on org/project1@master
self.create_branch('org/project1', 'feat-x')
self.fake_gitlab.protectBranch('org', 'project1', 'master',
protected=True)
self.scheds.execute(lambda app: app.sched.reconfigure(app.config))
self.waitUntilSettled()
A = self.fake_gitlab.openFakeMergeRequest('org/project1', 'master',
'A')
self.fake_gitlab.emitEvent(A.getMergeRequestOpenedEvent())
self.waitUntilSettled()
build = self.history[0]
path = os.path.join(
build.jobdir.src_root, 'gitlab', 'org/project1')
build_repo = git.Repo(path)
branches = [x.name for x in build_repo.branches]
self.assertIn('feat-x', branches)
self.assertHistory([
dict(name='project-test', result='SUCCESS',
changes="%s,%s" % (A.number, A.sha)),
])
def test_unprotected_push(self):
"""Test that unprotected pushes don't cause tenant reconfigurations"""
# Prepare repo with an initial commit
A = self.fake_gitlab.openFakeMergeRequest('org/project2', 'master',
'A')
zuul_yaml = [
{'job': {
'name': 'used-job2',
'run': 'playbooks/used-job.yaml'
}},
{'project': {
'check': {
'jobs': [
'used-job2'
]
}
}}
]
A.addCommit({'zuul.yaml': yaml.dump(zuul_yaml)})
A.mergeMergeRequest()
# Do a push on top of A
pevent = self.fake_gitlab.getPushEvent(project='org/project2',
before=A.sha,
branch='refs/heads/master')
# record previous tenant reconfiguration time, which may not be set
old = self.scheds.first.sched.tenant_last_reconfigured\
.get('tenant-one', 0)
self.waitUntilSettled()
self.fake_gitlab.emitEvent(pevent)
self.waitUntilSettled()
new = self.scheds.first.sched.tenant_last_reconfigured\
.get('tenant-one', 0)
# We don't expect a reconfiguration because the push was to an
# unprotected branch
self.assertEqual(old, new)
# now enable branch protection and trigger the push event again
self.fake_gitlab.protectBranch('org', 'project2', 'master')
self.fake_gitlab.emitEvent(pevent)
self.waitUntilSettled()
new = self.scheds.first.sched.tenant_last_reconfigured\
.get('tenant-one', 0)
# We now expect that zuul reconfigured itself
self.assertLess(old, new)
def test_protected_branch_delete(self):
"""Test that protected branch deletes trigger a tenant reconfig"""
# Prepare repo with an initial commit and enable branch protection
self.fake_gitlab.protectBranch('org', 'project2', 'master')
A = self.fake_gitlab.openFakeMergeRequest('org/project2', 'master',
'A')
A.mergeMergeRequest()
# add a spare branch so that the project is not empty after master gets
# deleted.
self.create_branch('org/project2', 'feat-x')
self.fake_gitlab.protectBranch('org', 'project2', 'feat-x',
protected=False)
self.scheds.execute(lambda app: app.sched.reconfigure(app.config))
self.waitUntilSettled()
# record previous tenant reconfiguration time, which may not be set
old = self.scheds.first.sched.tenant_last_reconfigured\
.get('tenant-one', 0)
self.waitUntilSettled()
# Delete the branch
self.fake_gitlab.deleteBranch('org', 'project2', 'master')
pevent = self.fake_gitlab.getPushEvent(project='org/project2',
before=A.sha,
after='0' * 40,
branch='refs/heads/master')
self.fake_gitlab.emitEvent(pevent)
self.waitUntilSettled()
new = self.scheds.first.sched.tenant_last_reconfigured\
.get('tenant-one', 0)
# We now expect that zuul reconfigured itself as we deleted a protected
# branch
self.assertLess(old, new)
# This test verifies that a PR is considered in case it was created for
# a branch just has been set to protected before a tenant reconfiguration
# took place.
def test_reconfigure_on_pr_to_new_protected_branch(self):
self.create_branch('org/project2', 'release')
self.create_branch('org/project2', 'feature')
self.fake_gitlab.protectBranch('org', 'project2', 'master')
self.fake_gitlab.protectBranch('org', 'project2', 'release',
protected=False)
self.fake_gitlab.protectBranch('org', 'project2', 'feature',
protected=False)
self.scheds.execute(lambda app: app.sched.reconfigure(app.config))
self.waitUntilSettled()
self.fake_gitlab.protectBranch('org', 'project2', 'release')
self.executor_server.hold_jobs_in_build = True
A = self.fake_gitlab.openFakeMergeRequest(
'org/project2', 'release', 'A')
self.fake_gitlab.emitEvent(A.getMergeRequestOpenedEvent())
self.waitUntilSettled()
self.executor_server.hold_jobs_in_build = False
self.executor_server.release()
self.waitUntilSettled()
self.assertEqual('SUCCESS',
self.getJobFromHistory('used-job').result)
job = self.getJobFromHistory('used-job')
zuulvars = job.parameters['zuul']
self.assertEqual(str(A.number), zuulvars['change'])
self.assertEqual(str(A.sha), zuulvars['patchset'])
self.assertEqual('release', zuulvars['branch'])
self.assertEqual(1, len(self.history))
def _test_push_event_reconfigure(self, project, branch,
expect_reconfigure=False,
old_sha=None, new_sha=None,
modified_files=None,
removed_files=None,
expected_cat_jobs=None):
pevent = self.fake_gitlab.getPushEvent(
project=project,
branch='refs/heads/%s' % branch,
before=old_sha,
after=new_sha)
# record previous tenant reconfiguration time, which may not be set
old = self.scheds.first.sched.tenant_last_reconfigured\
.get('tenant-one', 0)
self.waitUntilSettled()
if expected_cat_jobs is not None:
# clear the gearman jobs history so we can count the cat jobs
# issued during reconfiguration
self.gearman_server.jobs_history.clear()
self.fake_gitlab.emitEvent(pevent)
self.waitUntilSettled()
new = self.scheds.first.sched.tenant_last_reconfigured\
.get('tenant-one', 0)
if expect_reconfigure:
# New timestamp should be greater than the old timestamp
self.assertLess(old, new)
else:
# Timestamps should be equal as no reconfiguration shall happen
self.assertEqual(old, new)
if expected_cat_jobs is not None:
# Check the expected number of cat jobs here as the (empty) config
# of org/project should be cached.
cat_jobs = set([job for job in self.gearman_server.jobs_history
if job.name == b'merger:cat'])
self.assertEqual(expected_cat_jobs, len(cat_jobs), cat_jobs)
def test_push_event_reconfigure_complex_branch(self):
branch = 'feature/somefeature'
project = 'org/project2'
# prepare an existing branch
self.create_branch(project, branch)
self.fake_gitlab.protectBranch(*project.split('/'), branch,
protected=False)
self.fake_gitlab.emitEvent(
self.fake_gitlab.getPushEvent(
project,
branch='refs/heads/%s' % branch))
self.waitUntilSettled()
A = self.fake_gitlab.openFakeMergeRequest(project, branch, 'A')
old_sha = A.sha
A.mergeMergeRequest()
new_sha = random_sha1()
# branch is not protected, no reconfiguration even if config file
self._test_push_event_reconfigure(project, branch,
expect_reconfigure=False,
old_sha=old_sha,
new_sha=new_sha,
modified_files=['zuul.yaml'],
expected_cat_jobs=0)
# branch is not protected: no reconfiguration
self.fake_gitlab.deleteBranch(*project.split('/'), branch)
self._test_push_event_reconfigure(project, branch,
expect_reconfigure=False,
old_sha=new_sha,
new_sha='0' * 40,
removed_files=['zuul.yaml'])

View File

@ -25,12 +25,13 @@ import requests
import dateutil.parser
from urllib.parse import quote_plus
from typing import List, Optional
from zuul.connection import BaseConnection
from zuul.connection import CachedBranchConnection
from zuul.web.handler import BaseWebController
from zuul.lib.gearworker import ZuulGearWorker
from zuul.lib.logutil import get_annotated_logger
from zuul.model import Ref, Branch, Tag
from zuul.model import Branch, Project, Ref, Tag
from zuul.driver.gitlab.gitlabmodel import GitlabTriggerEvent, MergeRequest
@ -178,13 +179,10 @@ class GitlabEventConnector(threading.Thread):
event.ref = body['ref']
event.newrev = body['after']
event.oldrev = body['before']
if event.newrev == '0' * 40:
event.branch_deleted = True
elif event.oldrev == '0' * 40:
event.branch_created = True
else:
event.branch_updated = True
event.type = 'gl_push'
self.connection.clearConnectionCacheOnBranchEvent(event)
return event
# https://gitlab.com/help/user/project/integrations/webhooks#tag-events
@ -222,6 +220,7 @@ class GitlabEventConnector(threading.Thread):
if event:
event.zuul_event_id = str(uuid.uuid4())
event.timestamp = ts
event.project_hostname = self.connection.canonical_hostname
if event.change_number:
project = self.connection.source.getProject(event.project_name)
self.connection._getChange(project,
@ -230,7 +229,13 @@ class GitlabEventConnector(threading.Thread):
refresh=True,
url=event.change_url,
event=event)
event.project_hostname = self.connection.canonical_hostname
# If this event references a branch and we're excluding
# unprotected branches, we might need to check whether the
# branch is now protected.
if hasattr(event, "branch") and event.branch:
self.connection.checkBranchCache(event.project_name, event)
self.connection.logEvent(event)
self.connection.sched.addEvent(event)
@ -304,13 +309,32 @@ class GitlabAPIClient():
return resp[0]
# https://docs.gitlab.com/ee/api/branches.html#list-repository-branches
def get_project_branches(self, project_name, zuul_event_id=None):
path = "/projects/%s/repository/branches" % (
quote_plus(project_name))
resp = self.get(self.baseurl + path, zuul_event_id=zuul_event_id)
def get_project_branches(self, project_name, exclude_unprotected,
zuul_event_id=None):
if exclude_unprotected:
path = "/projects/{}/protected_branches"
else:
path = "/projects/{}/repository/branches"
url = self.baseurl + path.format(quote_plus(project_name))
resp = self.get(url, zuul_event_id=zuul_event_id)
self._manage_error(*resp, zuul_event_id=zuul_event_id)
return [branch['name'] for branch in resp[0]]
# https://docs.gitlab.com/ee/api/branches.html#get-single-repository-branch
def get_project_branch(self, project_name, branch_name,
zuul_event_id=None):
path = "/projects/{}/repository/branches/{}"
path = path.format(quote_plus(project_name), quote_plus(branch_name))
url = self.baseurl + path
resp = self.get(url, zuul_event_id=zuul_event_id)
try:
self._manage_error(*resp, zuul_event_id=zuul_event_id)
except GitlabAPIClientException:
if resp[1] != 404:
raise
return {}
return resp[0]
# https://docs.gitlab.com/ee/api/notes.html#create-new-merge-request-note
def comment_mr(self, project_name, number, msg, zuul_event_id=None):
path = "/projects/%s/merge_requests/%s/notes" % (
@ -367,7 +391,7 @@ class GitlabAPIClient():
return resp[0]
class GitlabConnection(BaseConnection):
class GitlabConnection(CachedBranchConnection):
driver_name = 'gitlab'
log = logging.getLogger("zuul.GitlabConnection")
payload_path = 'payload'
@ -376,7 +400,6 @@ class GitlabConnection(BaseConnection):
super(GitlabConnection, self).__init__(
driver, connection_name, connection_config)
self.projects = {}
self.project_branch_cache = {}
self._change_cache = {}
self.server = self.connection_config.get('server', 'gitlab.com')
self.baseurl = self.connection_config.get(
@ -434,20 +457,17 @@ class GitlabConnection(BaseConnection):
def addProject(self, project):
self.projects[project.name] = project
def getProjectBranches(self, project, tenant):
branches = self.project_branch_cache.get(project.name)
if branches is not None:
return branches
branches = self.gl_client.get_project_branches(project.name)
self.project_branch_cache[project.name] = branches
self.log.info("Got branches for %s" % project.name)
def _fetchProjectBranches(self, project: Project,
exclude_unprotected: bool) -> List[str]:
branches = self.gl_client.get_project_branches(project.name,
exclude_unprotected)
return branches
def clearCache(self):
self.project_branch_cache = {}
def isBranchProtected(self, project_name: str, branch_name: str,
zuul_event_id=None) -> Optional[bool]:
branch = self.gl_client.get_project_branch(project_name, branch_name,
zuul_event_id)
return branch.get('protected')
def getGitwebUrl(self, project, sha=None):
url = '%s/%s' % (self.baseurl, project)