Gitlab - bootstrap the driver structure + Webhook support

This patch bring the base structure for a Gitlab driver.

Change-Id: If8844f84c5a04f2b60bd8bebeb9d9d5a20af687e
This commit is contained in:
Fabien Boucher 2019-09-30 13:02:58 +02:00 committed by Tristan Cacqueray
parent b5f294b225
commit 6821db44dd
12 changed files with 600 additions and 1 deletions

View File

@ -64,6 +64,7 @@ import zuul.driver.gerrit.gerritsource as gerritsource
import zuul.driver.gerrit.gerritconnection as gerritconnection
import zuul.driver.github.githubconnection as githubconnection
import zuul.driver.pagure.pagureconnection as pagureconnection
import zuul.driver.gitlab.gitlabconnection as gitlabconnection
import zuul.driver.github
import zuul.driver.sql
import zuul.scheduler
@ -1494,6 +1495,18 @@ class FakePagureConnection(pagureconnection.PagureConnection):
self.zuul_web_port = port
class FakeGitlabConnection(gitlabconnection.GitlabConnection):
log = logging.getLogger("zuul.test.FakeGitlabConnection")
def __init__(self, driver, connection_name, connection_config, rpcclient,
changes_db=None, upstream_root=None):
super(FakeGitlabConnection, self).__init__(driver, connection_name,
connection_config)
def setZuulWebPort(self, port):
self.zuul_web_port = port
class GithubChangeReference(git.Reference):
_common_path_default = "refs/pull"
_points_to_commits_only = True
@ -3353,6 +3366,7 @@ class ZuulTestCase(BaseTestCase):
self.gerrit_changes_dbs = {}
self.github_changes_dbs = {}
self.pagure_changes_dbs = {}
self.gitlab_changes_dbs = {}
def getGerritConnection(driver, name, config):
db = self.gerrit_changes_dbs.setdefault(config['server'], {})
@ -3428,6 +3442,22 @@ class ZuulTestCase(BaseTestCase):
'zuul.driver.pagure.PagureDriver.getConnection',
getPagureConnection))
def getGitlabConnection(driver, name, config):
server = config.get('server', 'gitlab.com')
db = self.gitlab_changes_dbs.setdefault(server, {})
con = FakeGitlabConnection(
driver, name, config,
self.rpcclient,
changes_db=db,
upstream_root=self.upstream_root)
self.event_queues.append(con.event_queue)
setattr(self, 'fake_' + name, con)
return con
self.useFixture(fixtures.MonkeyPatch(
'zuul.driver.gitlab.GitlabDriver.getConnection',
getGitlabConnection))
# Set up smtp related fakes
# TODO(jhesketh): This should come from lib.connections for better
# coverage

View File

@ -0,0 +1,4 @@
- pipeline:
name: check
manager: independent
trigger: {}

18
tests/fixtures/zuul-gitlab-driver.conf vendored Normal file
View File

@ -0,0 +1,18 @@
[gearman]
server=127.0.0.1
[web]
status_url=http://zuul.example.com/status/#{change.number},{change.patchset}
[merger]
git_dir=/tmp/zuul-test/git
git_user_email=zuul@example.com
git_user_name=zuul
[executor]
git_dir=/tmp/zuul-test/executor-git
[connection gitlab]
driver=gitlab
server=gitlab
api_token=0000000000000000000000000000000000000000

View File

@ -0,0 +1,49 @@
# Copyright 2019 Red Hat
#
# Licensed under the Apache License, Version 2.0 (the "License"); you may
# not use this file except in compliance with the License. You may obtain
# a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
# WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
# License for the specific language governing permissions and limitations
# under the License.
import socket
from tests.base import ZuulTestCase, simple_layout
from tests.base import ZuulWebFixture
class TestGitlabWebhook(ZuulTestCase):
config_file = 'zuul-gitlab-driver.conf'
def setUp(self):
super().setUp()
# Start the web server
self.web = self.useFixture(
ZuulWebFixture(self.gearman_server.port,
self.config, self.test_root))
host = '127.0.0.1'
# Wait until web server is started
while True:
port = self.web.port
try:
with socket.create_connection((host, port)):
break
except ConnectionRefusedError:
pass
self.fake_gitlab.setZuulWebPort(port)
def tearDown(self):
super(TestGitlabWebhook, self).tearDown()
@simple_layout('layouts/basic-gitlab.yaml', driver='gitlab')
def test_webhook(self):
pass

View File

@ -124,7 +124,8 @@ class WebServer(zuul.cmd.ZuulDaemonApp):
self.configure_connections(
include_drivers=[zuul.driver.sql.SQLDriver,
zuul.driver.github.GithubDriver,
zuul.driver.pagure.PagureDriver])
zuul.driver.pagure.PagureDriver,
zuul.driver.gitlab.GitlabDriver])
self.configure_authenticators()
self._run()
except Exception:

View File

@ -0,0 +1,50 @@
# Copyright 2018 Red Hat, Inc.
#
# Licensed under the Apache License, Version 2.0 (the "License"); you may
# not use this file except in compliance with the License. You may obtain
# a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
# WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
# License for the specific language governing permissions and limitations
# under the License.
from zuul.driver import Driver, ConnectionInterface, TriggerInterface
from zuul.driver import SourceInterface, ReporterInterface
from zuul.driver.gitlab import gitlabconnection
from zuul.driver.gitlab import gitlabsource
from zuul.driver.gitlab import gitlabreporter
from zuul.driver.gitlab import gitlabtrigger
class GitlabDriver(Driver, ConnectionInterface, TriggerInterface,
SourceInterface, ReporterInterface):
name = 'gitlab'
def getConnection(self, name, config):
return gitlabconnection.GitlabConnection(self, name, config)
def getTrigger(self, connection, config=None):
return gitlabtrigger.GitlabTrigger(self, connection, config)
def getSource(self, connection):
return gitlabsource.GitlabSource(self, connection)
def getReporter(self, connection, pipeline, config=None):
return gitlabreporter.GitlabReporter(
self, connection, pipeline, config)
def getTriggerSchema(self):
return gitlabtrigger.getSchema()
def getReporterSchema(self):
return gitlabreporter.getSchema()
def getRequireSchema(self):
return gitlabsource.getRequireSchema()
def getRejectSchema(self):
return gitlabsource.getRejectSchema()

View File

@ -0,0 +1,224 @@
# Copyright 2019 Red Hat, Inc.
#
# Licensed under the Apache License, Version 2.0 (the "License"); you may
# not use this file except in compliance with the License. You may obtain
# a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
# WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
# License for the specific language governing permissions and limitations
# under the License.
import logging
import threading
import json
import queue
import cherrypy
import voluptuous as v
import time
from zuul.connection import BaseConnection
from zuul.web.handler import BaseWebController
from zuul.lib.gearworker import ZuulGearWorker
class GitlabGearmanWorker(object):
"""A thread that answers gearman requests"""
log = logging.getLogger("zuul.GitlabGearmanWorker")
def __init__(self, connection):
self.config = connection.sched.config
self.connection = connection
handler = "gitlab:%s:payload" % self.connection.connection_name
self.jobs = {
handler: self.handle_payload,
}
self.gearworker = ZuulGearWorker(
'Zuul Gitlab Worker',
'zuul.GitlabGearmanWorker',
'gitlab',
self.config,
self.jobs)
def handle_payload(self, job):
args = json.loads(job.arguments)
payload = args["payload"]
self.log.info(
"Gitlab Webhook Received (event kind: %(object_kind)s ",
"event name: %(event_name)s)" % payload)
try:
self.__dispatch_event(payload)
output = {'return_code': 200}
except Exception:
output = {'return_code': 503}
self.log.exception("Exception handling Gitlab event:")
job.sendWorkComplete(json.dumps(output))
def __dispatch_event(self, payload):
event = payload['event_name']
try:
self.log.info("Dispatching event %s" % event)
self.connection.addEvent(payload, event)
except Exception as err:
message = 'Exception dispatching event: %s' % str(err)
self.log.exception(message)
raise Exception(message)
def start(self):
self.gearworker.start()
def stop(self):
self.gearworker.stop()
class GitlabEventConnector(threading.Thread):
"""Move events from Gitlab into the scheduler"""
log = logging.getLogger("zuul.GitlabEventConnector")
def __init__(self, connection):
super(GitlabEventConnector, self).__init__()
self.daemon = True
self.connection = connection
self._stopped = False
self.event_handler_mapping = {}
def stop(self):
self._stopped = True
self.connection.addEvent(None)
def _handleEvent(self):
ts, json_body, event_type = self.connection.getEvent()
if self._stopped:
return
self.log.info("Received event: %s" % str(event_type))
if event_type not in self.event_handler_mapping:
message = "Unhandled Gitlab event: %s" % event_type
self.log.info(message)
return
def run(self):
while True:
if self._stopped:
return
try:
self._handleEvent()
except Exception:
self.log.exception("Exception moving Gitlab event:")
finally:
self.connection.eventDone()
class GitlabConnection(BaseConnection):
driver_name = 'gitlab'
log = logging.getLogger("zuul.GitlabConnection")
payload_path = 'payload'
def __init__(self, driver, connection_name, connection_config):
super(GitlabConnection, self).__init__(
driver, connection_name, connection_config)
self.projects = {}
self.server = self.connection_config.get('server', 'gitlab.com')
self.canonical_hostname = self.connection_config.get(
'canonical_hostname', self.server)
self.webhook_token = self.connection_config.get(
'webhook_token', '')
self.sched = None
self.event_queue = queue.Queue()
def _start_event_connector(self):
self.gitlab_event_connector = GitlabEventConnector(self)
self.gitlab_event_connector.start()
def _stop_event_connector(self):
if self.gitlab_event_connector:
self.gitlab_event_connector.stop()
self.gitlab_event_connector.join()
def onLoad(self):
self.log.info('Starting Gitlab connection: %s' % self.connection_name)
self.gearman_worker = GitlabGearmanWorker(self)
self.log.info('Starting event connector')
self._start_event_connector()
self.log.info('Starting GearmanWorker')
self.gearman_worker.start()
def onStop(self):
if hasattr(self, 'gearman_worker'):
self.gearman_worker.stop()
self._stop_event_connector()
def addEvent(self, data, event=None):
return self.event_queue.put((time.time(), data, event))
def getEvent(self):
return self.event_queue.get()
def eventDone(self):
self.event_queue.task_done()
def getWebController(self, zuul_web):
return GitlabWebController(zuul_web, self)
def getChange(self, event):
return None
def getProject(self, name):
return self.projects.get(name)
def addProject(self, project):
self.projects[project.name] = project
class GitlabWebController(BaseWebController):
log = logging.getLogger("zuul.GitlabWebController")
def __init__(self, zuul_web, connection):
self.connection = connection
self.zuul_web = zuul_web
def _validate_token(self, headers):
try:
event_token = headers['x-gitlab-token']
except KeyError:
raise cherrypy.HTTPError(401, 'x-gitlab-token header missing.')
configured_token = self.connection.webhook_token
if not configured_token == event_token:
self.log.debug(
"Missmatch (Incoming token: %s, Configured token: %s)" % (
event_token, configured_token))
raise cherrypy.HTTPError(
401,
'Token does not match the server side configured token')
@cherrypy.expose
@cherrypy.tools.json_out(content_type='application/json; charset=utf-8')
def payload(self):
headers = dict()
for key, value in cherrypy.request.headers.items():
headers[key.lower()] = value
body = cherrypy.request.body.read()
self.log.info("Event header: %s" % headers)
self.log.info("Event body: %s" % body)
self._validate_token(headers)
json_payload = json.loads(body.decode('utf-8'))
job = self.zuul_web.rpc.submitJob(
'gitlab:%s:payload' % self.connection.connection_name,
{'payload': json_payload})
return json.loads(job.data[0])
def getSchema():
return v.Any(str, v.Schema(dict))

View File

@ -0,0 +1,37 @@
# Copyright 2019 Red Hat, Inc.
#
# Licensed under the Apache License, Version 2.0 (the "License"); you may
# not use this file except in compliance with the License. You may obtain
# a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
# WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
# License for the specific language governing permissions and limitations
# under the License.
from zuul.model import Change, TriggerEvent, EventFilter, RefFilter
class PullRequest(Change):
def __init__(self, project):
super(PullRequest, self).__init__(project)
class GitlabTriggerEvent(TriggerEvent):
def __init__(self):
super(GitlabTriggerEvent, self).__init__()
class GitlabEventFilter(EventFilter):
def __init__(self, trigger):
super(GitlabEventFilter, self).__init__()
# The RefFilter should be understood as RequireFilter (it maps to
# pipeline requires definition)
class GitlabRefFilter(RefFilter):
def __init__(self, connection_name):
RefFilter.__init__(self, connection_name)

View File

@ -0,0 +1,42 @@
# Copyright 2019 Red Hat, Inc.
#
# Licensed under the Apache License, Version 2.0 (the "License"); you may
# not use this file except in compliance with the License. You may obtain
# a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
# WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
# License for the specific language governing permissions and limitations
# under the License.
import logging
import voluptuous as v
from zuul.reporter import BaseReporter
class GitlabReporter(BaseReporter):
"""Sends off reports to Gitlab."""
name = 'gitlab'
log = logging.getLogger("zuul.GitlabReporter")
def __init__(self, driver, connection, pipeline, config=None):
super(GitlabReporter, self).__init__(driver, connection, config)
def report(self, item):
"""Report on an event."""
raise NotImplementedError()
def mergePull(self, item):
raise NotImplementedError()
def getSubmitAllowNeeds(self):
return []
def getSchema():
return v.Schema({})

View File

@ -0,0 +1,109 @@
# Copyright 2019 Red Hat, Inc.
#
# Licensed under the Apache License, Version 2.0 (the "License"); you may
# not use this file except in compliance with the License. You may obtain
# a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
# WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
# License for the specific language governing permissions and limitations
# under the License.
import logging
from zuul.model import Project
from zuul.source import BaseSource
from zuul.driver.gitlab.gitlabmodel import GitlabRefFilter
class GitlabSource(BaseSource):
name = 'gitlab'
log = logging.getLogger("zuul.source.GitlabSource")
def __init__(self, driver, connection, config=None):
hostname = connection.canonical_hostname
super(GitlabSource, self).__init__(driver, connection,
hostname, config)
def getRefSha(self, project, ref):
"""Return a sha for a given project ref."""
raise NotImplementedError()
def waitForRefSha(self, project, ref, old_sha=''):
"""Block until a ref shows up in a given project."""
raise NotImplementedError()
def isMerged(self, change, head=None):
"""Determine if change is merged."""
raise NotImplementedError()
def canMerge(self, change, allow_needs, event=None):
"""Determine if change can merge."""
raise NotImplementedError()
def postConfig(self):
"""Called after configuration has been processed."""
raise NotImplementedError()
def getChange(self, event, refresh=False):
raise NotImplementedError()
def getChangeByURL(self, url):
raise NotImplementedError()
def getChangesDependingOn(self, change, projects, tenant):
raise NotImplementedError()
def getCachedChanges(self):
raise NotImplementedError()
def getProject(self, name):
p = self.connection.getProject(name)
if not p:
p = Project(name, self)
self.connection.addProject(p)
return p
def getProjectBranches(self, project, tenant):
raise NotImplementedError()
def getProjectOpenChanges(self, project):
"""Get the open changes for a project."""
raise NotImplementedError()
def updateChange(self, change, history=None):
"""Update information for a change."""
raise NotImplementedError()
def getGitUrl(self, project):
"""Get the git url for a project."""
raise NotImplementedError()
def getGitwebUrl(self, project, sha=None):
"""Get the git-web url for a project."""
raise NotImplementedError()
def getRequireFilters(self, config):
f = GitlabRefFilter()
return [f]
def getRejectFilters(self, config):
raise NotImplementedError()
def getRefForChange(self, change):
raise NotImplementedError()
# Require model
def getRequireSchema():
require = {}
return require
def getRejectSchema():
reject = {}
return reject

View File

@ -0,0 +1,33 @@
# Copyright 2019 Red Hat, Inc.
#
# Licensed under the Apache License, Version 2.0 (the "License"); you may
# not use this file except in compliance with the License. You may obtain
# a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
# WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
# License for the specific language governing permissions and limitations
# under the License.
import logging
from zuul.trigger import BaseTrigger
class GitlabTrigger(BaseTrigger):
name = 'gitlab'
log = logging.getLogger("zuul.trigger.GitlabTrigger")
def getEventFilters(self, trigger_config):
efilters = []
return efilters
def onPullRequest(self, payload):
pass
def getSchema():
gitlab_trigger = {}
return gitlab_trigger

View File

@ -28,6 +28,7 @@ import zuul.driver.bubblewrap
import zuul.driver.nullwrap
import zuul.driver.mqtt
import zuul.driver.pagure
import zuul.driver.gitlab
from zuul.connection import BaseConnection
from zuul.driver import SourceInterface
@ -56,6 +57,7 @@ class ConnectionRegistry(object):
self.registerDriver(zuul.driver.nullwrap.NullwrapDriver())
self.registerDriver(zuul.driver.mqtt.MQTTDriver())
self.registerDriver(zuul.driver.pagure.PagureDriver())
self.registerDriver(zuul.driver.gitlab.GitlabDriver())
def registerDriver(self, driver):
if driver.name in self.drivers: